mirror of
https://github.com/fluencelabs/tendermint
synced 2025-06-12 12:51:22 +00:00
Revert "delete everything" (includes everything non-go-crypto)
This reverts commit 96a3502
This commit is contained in:
794
rpc/lib/server/handlers.go
Normal file
794
rpc/lib/server/handlers.go
Normal file
@ -0,0 +1,794 @@
|
||||
package rpcserver
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"encoding/hex"
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"io/ioutil"
|
||||
"net/http"
|
||||
"reflect"
|
||||
"runtime/debug"
|
||||
"sort"
|
||||
"strings"
|
||||
"time"
|
||||
|
||||
"github.com/gorilla/websocket"
|
||||
"github.com/pkg/errors"
|
||||
|
||||
amino "github.com/tendermint/go-amino"
|
||||
types "github.com/tendermint/tendermint/rpc/lib/types"
|
||||
cmn "github.com/tendermint/tmlibs/common"
|
||||
"github.com/tendermint/tmlibs/log"
|
||||
)
|
||||
|
||||
// RegisterRPCFuncs adds a route for each function in the funcMap, as well as general jsonrpc and websocket handlers for all functions.
|
||||
// "result" is the interface on which the result objects are registered, and is popualted with every RPCResponse
|
||||
func RegisterRPCFuncs(mux *http.ServeMux, funcMap map[string]*RPCFunc, cdc *amino.Codec, logger log.Logger) {
|
||||
// HTTP endpoints
|
||||
for funcName, rpcFunc := range funcMap {
|
||||
mux.HandleFunc("/"+funcName, makeHTTPHandler(rpcFunc, cdc, logger))
|
||||
}
|
||||
|
||||
// JSONRPC endpoints
|
||||
mux.HandleFunc("/", handleInvalidJSONRPCPaths(makeJSONRPCHandler(funcMap, cdc, logger)))
|
||||
}
|
||||
|
||||
//-------------------------------------
|
||||
// function introspection
|
||||
|
||||
// RPCFunc contains the introspected type information for a function
|
||||
type RPCFunc struct {
|
||||
f reflect.Value // underlying rpc function
|
||||
args []reflect.Type // type of each function arg
|
||||
returns []reflect.Type // type of each return arg
|
||||
argNames []string // name of each argument
|
||||
ws bool // websocket only
|
||||
}
|
||||
|
||||
// NewRPCFunc wraps a function for introspection.
|
||||
// f is the function, args are comma separated argument names
|
||||
func NewRPCFunc(f interface{}, args string) *RPCFunc {
|
||||
return newRPCFunc(f, args, false)
|
||||
}
|
||||
|
||||
// NewWSRPCFunc wraps a function for introspection and use in the websockets.
|
||||
func NewWSRPCFunc(f interface{}, args string) *RPCFunc {
|
||||
return newRPCFunc(f, args, true)
|
||||
}
|
||||
|
||||
func newRPCFunc(f interface{}, args string, ws bool) *RPCFunc {
|
||||
var argNames []string
|
||||
if args != "" {
|
||||
argNames = strings.Split(args, ",")
|
||||
}
|
||||
return &RPCFunc{
|
||||
f: reflect.ValueOf(f),
|
||||
args: funcArgTypes(f),
|
||||
returns: funcReturnTypes(f),
|
||||
argNames: argNames,
|
||||
ws: ws,
|
||||
}
|
||||
}
|
||||
|
||||
// return a function's argument types
|
||||
func funcArgTypes(f interface{}) []reflect.Type {
|
||||
t := reflect.TypeOf(f)
|
||||
n := t.NumIn()
|
||||
typez := make([]reflect.Type, n)
|
||||
for i := 0; i < n; i++ {
|
||||
typez[i] = t.In(i)
|
||||
}
|
||||
return typez
|
||||
}
|
||||
|
||||
// return a function's return types
|
||||
func funcReturnTypes(f interface{}) []reflect.Type {
|
||||
t := reflect.TypeOf(f)
|
||||
n := t.NumOut()
|
||||
typez := make([]reflect.Type, n)
|
||||
for i := 0; i < n; i++ {
|
||||
typez[i] = t.Out(i)
|
||||
}
|
||||
return typez
|
||||
}
|
||||
|
||||
// function introspection
|
||||
//-----------------------------------------------------------------------------
|
||||
// rpc.json
|
||||
|
||||
// jsonrpc calls grab the given method's function info and runs reflect.Call
|
||||
func makeJSONRPCHandler(funcMap map[string]*RPCFunc, cdc *amino.Codec, logger log.Logger) http.HandlerFunc {
|
||||
return func(w http.ResponseWriter, r *http.Request) {
|
||||
b, err := ioutil.ReadAll(r.Body)
|
||||
if err != nil {
|
||||
WriteRPCResponseHTTP(w, types.RPCInvalidRequestError("", errors.Wrap(err, "Error reading request body")))
|
||||
return
|
||||
}
|
||||
// if its an empty request (like from a browser),
|
||||
// just display a list of functions
|
||||
if len(b) == 0 {
|
||||
writeListOfEndpoints(w, r, funcMap)
|
||||
return
|
||||
}
|
||||
|
||||
var request types.RPCRequest
|
||||
err = json.Unmarshal(b, &request)
|
||||
if err != nil {
|
||||
WriteRPCResponseHTTP(w, types.RPCParseError("", errors.Wrap(err, "Error unmarshalling request")))
|
||||
return
|
||||
}
|
||||
// A Notification is a Request object without an "id" member.
|
||||
// The Server MUST NOT reply to a Notification, including those that are within a batch request.
|
||||
if request.ID == "" {
|
||||
logger.Debug("HTTPJSONRPC received a notification, skipping... (please send a non-empty ID if you want to call a method)")
|
||||
return
|
||||
}
|
||||
if len(r.URL.Path) > 1 {
|
||||
WriteRPCResponseHTTP(w, types.RPCInvalidRequestError(request.ID, errors.Errorf("Path %s is invalid", r.URL.Path)))
|
||||
return
|
||||
}
|
||||
rpcFunc := funcMap[request.Method]
|
||||
if rpcFunc == nil || rpcFunc.ws {
|
||||
WriteRPCResponseHTTP(w, types.RPCMethodNotFoundError(request.ID))
|
||||
return
|
||||
}
|
||||
var args []reflect.Value
|
||||
if len(request.Params) > 0 {
|
||||
args, err = jsonParamsToArgsRPC(rpcFunc, cdc, request.Params)
|
||||
if err != nil {
|
||||
WriteRPCResponseHTTP(w, types.RPCInvalidParamsError(request.ID, errors.Wrap(err, "Error converting json params to arguments")))
|
||||
return
|
||||
}
|
||||
}
|
||||
returns := rpcFunc.f.Call(args)
|
||||
logger.Info("HTTPJSONRPC", "method", request.Method, "args", args, "returns", returns)
|
||||
result, err := unreflectResult(returns)
|
||||
if err != nil {
|
||||
WriteRPCResponseHTTP(w, types.RPCInternalError(request.ID, err))
|
||||
return
|
||||
}
|
||||
WriteRPCResponseHTTP(w, types.NewRPCSuccessResponse(cdc, request.ID, result))
|
||||
}
|
||||
}
|
||||
|
||||
func handleInvalidJSONRPCPaths(next http.HandlerFunc) http.HandlerFunc {
|
||||
return func(w http.ResponseWriter, r *http.Request) {
|
||||
// Since the pattern "/" matches all paths not matched by other registered patterns we check whether the path is indeed
|
||||
// "/", otherwise return a 404 error
|
||||
if r.URL.Path != "/" {
|
||||
http.NotFound(w, r)
|
||||
return
|
||||
}
|
||||
|
||||
next(w, r)
|
||||
}
|
||||
}
|
||||
|
||||
func mapParamsToArgs(rpcFunc *RPCFunc, cdc *amino.Codec, params map[string]json.RawMessage, argsOffset int) ([]reflect.Value, error) {
|
||||
values := make([]reflect.Value, len(rpcFunc.argNames))
|
||||
for i, argName := range rpcFunc.argNames {
|
||||
argType := rpcFunc.args[i+argsOffset]
|
||||
|
||||
if p, ok := params[argName]; ok && p != nil && len(p) > 0 {
|
||||
val := reflect.New(argType)
|
||||
err := cdc.UnmarshalJSON(p, val.Interface())
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
values[i] = val.Elem()
|
||||
} else { // use default for that type
|
||||
values[i] = reflect.Zero(argType)
|
||||
}
|
||||
}
|
||||
|
||||
return values, nil
|
||||
}
|
||||
|
||||
func arrayParamsToArgs(rpcFunc *RPCFunc, cdc *amino.Codec, params []json.RawMessage, argsOffset int) ([]reflect.Value, error) {
|
||||
if len(rpcFunc.argNames) != len(params) {
|
||||
return nil, errors.Errorf("Expected %v parameters (%v), got %v (%v)",
|
||||
len(rpcFunc.argNames), rpcFunc.argNames, len(params), params)
|
||||
}
|
||||
|
||||
values := make([]reflect.Value, len(params))
|
||||
for i, p := range params {
|
||||
argType := rpcFunc.args[i+argsOffset]
|
||||
val := reflect.New(argType)
|
||||
err := cdc.UnmarshalJSON(p, val.Interface())
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
values[i] = val.Elem()
|
||||
}
|
||||
return values, nil
|
||||
}
|
||||
|
||||
// `raw` is unparsed json (from json.RawMessage) encoding either a map or an array.
|
||||
// `argsOffset` should be 0 for RPC calls, and 1 for WS requests, where len(rpcFunc.args) != len(rpcFunc.argNames).
|
||||
//
|
||||
// Example:
|
||||
// rpcFunc.args = [rpctypes.WSRPCContext string]
|
||||
// rpcFunc.argNames = ["arg"]
|
||||
func jsonParamsToArgs(rpcFunc *RPCFunc, cdc *amino.Codec, raw []byte, argsOffset int) ([]reflect.Value, error) {
|
||||
|
||||
// TODO: Make more efficient, perhaps by checking the first character for '{' or '['?
|
||||
// First, try to get the map.
|
||||
var m map[string]json.RawMessage
|
||||
err := json.Unmarshal(raw, &m)
|
||||
if err == nil {
|
||||
return mapParamsToArgs(rpcFunc, cdc, m, argsOffset)
|
||||
}
|
||||
|
||||
// Otherwise, try an array.
|
||||
var a []json.RawMessage
|
||||
err = json.Unmarshal(raw, &a)
|
||||
if err == nil {
|
||||
return arrayParamsToArgs(rpcFunc, cdc, a, argsOffset)
|
||||
}
|
||||
|
||||
// Otherwise, bad format, we cannot parse
|
||||
return nil, errors.Errorf("Unknown type for JSON params: %v. Expected map or array", err)
|
||||
}
|
||||
|
||||
// Convert a []interface{} OR a map[string]interface{} to properly typed values
|
||||
func jsonParamsToArgsRPC(rpcFunc *RPCFunc, cdc *amino.Codec, params json.RawMessage) ([]reflect.Value, error) {
|
||||
return jsonParamsToArgs(rpcFunc, cdc, params, 0)
|
||||
}
|
||||
|
||||
// Same as above, but with the first param the websocket connection
|
||||
func jsonParamsToArgsWS(rpcFunc *RPCFunc, cdc *amino.Codec, params json.RawMessage, wsCtx types.WSRPCContext) ([]reflect.Value, error) {
|
||||
values, err := jsonParamsToArgs(rpcFunc, cdc, params, 1)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return append([]reflect.Value{reflect.ValueOf(wsCtx)}, values...), nil
|
||||
}
|
||||
|
||||
// rpc.json
|
||||
//-----------------------------------------------------------------------------
|
||||
// rpc.http
|
||||
|
||||
// convert from a function name to the http handler
|
||||
func makeHTTPHandler(rpcFunc *RPCFunc, cdc *amino.Codec, logger log.Logger) func(http.ResponseWriter, *http.Request) {
|
||||
// Exception for websocket endpoints
|
||||
if rpcFunc.ws {
|
||||
return func(w http.ResponseWriter, r *http.Request) {
|
||||
WriteRPCResponseHTTP(w, types.RPCMethodNotFoundError(""))
|
||||
}
|
||||
}
|
||||
// All other endpoints
|
||||
return func(w http.ResponseWriter, r *http.Request) {
|
||||
logger.Debug("HTTP HANDLER", "req", r)
|
||||
args, err := httpParamsToArgs(rpcFunc, cdc, r)
|
||||
if err != nil {
|
||||
WriteRPCResponseHTTP(w, types.RPCInvalidParamsError("", errors.Wrap(err, "Error converting http params to arguments")))
|
||||
return
|
||||
}
|
||||
returns := rpcFunc.f.Call(args)
|
||||
logger.Info("HTTPRestRPC", "method", r.URL.Path, "args", args, "returns", returns)
|
||||
result, err := unreflectResult(returns)
|
||||
if err != nil {
|
||||
WriteRPCResponseHTTP(w, types.RPCInternalError("", err))
|
||||
return
|
||||
}
|
||||
WriteRPCResponseHTTP(w, types.NewRPCSuccessResponse(cdc, "", result))
|
||||
}
|
||||
}
|
||||
|
||||
// Covert an http query to a list of properly typed values.
|
||||
// To be properly decoded the arg must be a concrete type from tendermint (if its an interface).
|
||||
func httpParamsToArgs(rpcFunc *RPCFunc, cdc *amino.Codec, r *http.Request) ([]reflect.Value, error) {
|
||||
values := make([]reflect.Value, len(rpcFunc.args))
|
||||
|
||||
for i, name := range rpcFunc.argNames {
|
||||
argType := rpcFunc.args[i]
|
||||
|
||||
values[i] = reflect.Zero(argType) // set default for that type
|
||||
|
||||
arg := GetParam(r, name)
|
||||
// log.Notice("param to arg", "argType", argType, "name", name, "arg", arg)
|
||||
|
||||
if "" == arg {
|
||||
continue
|
||||
}
|
||||
|
||||
v, err, ok := nonJSONToArg(cdc, argType, arg)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if ok {
|
||||
values[i] = v
|
||||
continue
|
||||
}
|
||||
|
||||
values[i], err = _jsonStringToArg(cdc, argType, arg)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
|
||||
return values, nil
|
||||
}
|
||||
|
||||
func _jsonStringToArg(cdc *amino.Codec, ty reflect.Type, arg string) (reflect.Value, error) {
|
||||
v := reflect.New(ty)
|
||||
err := cdc.UnmarshalJSON([]byte(arg), v.Interface())
|
||||
if err != nil {
|
||||
return v, err
|
||||
}
|
||||
v = v.Elem()
|
||||
return v, nil
|
||||
}
|
||||
|
||||
func nonJSONToArg(cdc *amino.Codec, ty reflect.Type, arg string) (reflect.Value, error, bool) {
|
||||
isQuotedString := strings.HasPrefix(arg, `"`) && strings.HasSuffix(arg, `"`)
|
||||
isHexString := strings.HasPrefix(strings.ToLower(arg), "0x")
|
||||
expectingString := ty.Kind() == reflect.String
|
||||
expectingByteSlice := ty.Kind() == reflect.Slice && ty.Elem().Kind() == reflect.Uint8
|
||||
|
||||
if isHexString {
|
||||
if !expectingString && !expectingByteSlice {
|
||||
err := errors.Errorf("Got a hex string arg, but expected '%s'",
|
||||
ty.Kind().String())
|
||||
return reflect.ValueOf(nil), err, false
|
||||
}
|
||||
|
||||
var value []byte
|
||||
value, err := hex.DecodeString(arg[2:])
|
||||
if err != nil {
|
||||
return reflect.ValueOf(nil), err, false
|
||||
}
|
||||
if ty.Kind() == reflect.String {
|
||||
return reflect.ValueOf(string(value)), nil, true
|
||||
}
|
||||
return reflect.ValueOf([]byte(value)), nil, true
|
||||
}
|
||||
|
||||
if isQuotedString && expectingByteSlice {
|
||||
v := reflect.New(reflect.TypeOf(""))
|
||||
err := cdc.UnmarshalJSON([]byte(arg), v.Interface())
|
||||
if err != nil {
|
||||
return reflect.ValueOf(nil), err, false
|
||||
}
|
||||
v = v.Elem()
|
||||
return reflect.ValueOf([]byte(v.String())), nil, true
|
||||
}
|
||||
|
||||
return reflect.ValueOf(nil), nil, false
|
||||
}
|
||||
|
||||
// rpc.http
|
||||
//-----------------------------------------------------------------------------
|
||||
// rpc.websocket
|
||||
|
||||
const (
|
||||
defaultWSWriteChanCapacity = 1000
|
||||
defaultWSWriteWait = 10 * time.Second
|
||||
defaultWSReadWait = 30 * time.Second
|
||||
defaultWSPingPeriod = (defaultWSReadWait * 9) / 10
|
||||
)
|
||||
|
||||
// A single websocket connection contains listener id, underlying ws
|
||||
// connection, and the event switch for subscribing to events.
|
||||
//
|
||||
// In case of an error, the connection is stopped.
|
||||
type wsConnection struct {
|
||||
cmn.BaseService
|
||||
|
||||
remoteAddr string
|
||||
baseConn *websocket.Conn
|
||||
writeChan chan types.RPCResponse
|
||||
|
||||
funcMap map[string]*RPCFunc
|
||||
cdc *amino.Codec
|
||||
|
||||
// write channel capacity
|
||||
writeChanCapacity int
|
||||
|
||||
// each write times out after this.
|
||||
writeWait time.Duration
|
||||
|
||||
// Connection times out if we haven't received *anything* in this long, not even pings.
|
||||
readWait time.Duration
|
||||
|
||||
// Send pings to server with this period. Must be less than readWait, but greater than zero.
|
||||
pingPeriod time.Duration
|
||||
|
||||
// object that is used to subscribe / unsubscribe from events
|
||||
eventSub types.EventSubscriber
|
||||
}
|
||||
|
||||
// NewWSConnection wraps websocket.Conn.
|
||||
//
|
||||
// See the commentary on the func(*wsConnection) functions for a detailed
|
||||
// description of how to configure ping period and pong wait time. NOTE: if the
|
||||
// write buffer is full, pongs may be dropped, which may cause clients to
|
||||
// disconnect. see https://github.com/gorilla/websocket/issues/97
|
||||
func NewWSConnection(baseConn *websocket.Conn, funcMap map[string]*RPCFunc, cdc *amino.Codec, options ...func(*wsConnection)) *wsConnection {
|
||||
wsc := &wsConnection{
|
||||
remoteAddr: baseConn.RemoteAddr().String(),
|
||||
baseConn: baseConn,
|
||||
funcMap: funcMap,
|
||||
cdc: cdc,
|
||||
writeWait: defaultWSWriteWait,
|
||||
writeChanCapacity: defaultWSWriteChanCapacity,
|
||||
readWait: defaultWSReadWait,
|
||||
pingPeriod: defaultWSPingPeriod,
|
||||
}
|
||||
for _, option := range options {
|
||||
option(wsc)
|
||||
}
|
||||
wsc.BaseService = *cmn.NewBaseService(nil, "wsConnection", wsc)
|
||||
return wsc
|
||||
}
|
||||
|
||||
// EventSubscriber sets object that is used to subscribe / unsubscribe from
|
||||
// events - not Goroutine-safe. If none given, default node's eventBus will be
|
||||
// used.
|
||||
func EventSubscriber(eventSub types.EventSubscriber) func(*wsConnection) {
|
||||
return func(wsc *wsConnection) {
|
||||
wsc.eventSub = eventSub
|
||||
}
|
||||
}
|
||||
|
||||
// WriteWait sets the amount of time to wait before a websocket write times out.
|
||||
// It should only be used in the constructor - not Goroutine-safe.
|
||||
func WriteWait(writeWait time.Duration) func(*wsConnection) {
|
||||
return func(wsc *wsConnection) {
|
||||
wsc.writeWait = writeWait
|
||||
}
|
||||
}
|
||||
|
||||
// WriteChanCapacity sets the capacity of the websocket write channel.
|
||||
// It should only be used in the constructor - not Goroutine-safe.
|
||||
func WriteChanCapacity(cap int) func(*wsConnection) {
|
||||
return func(wsc *wsConnection) {
|
||||
wsc.writeChanCapacity = cap
|
||||
}
|
||||
}
|
||||
|
||||
// ReadWait sets the amount of time to wait before a websocket read times out.
|
||||
// It should only be used in the constructor - not Goroutine-safe.
|
||||
func ReadWait(readWait time.Duration) func(*wsConnection) {
|
||||
return func(wsc *wsConnection) {
|
||||
wsc.readWait = readWait
|
||||
}
|
||||
}
|
||||
|
||||
// PingPeriod sets the duration for sending websocket pings.
|
||||
// It should only be used in the constructor - not Goroutine-safe.
|
||||
func PingPeriod(pingPeriod time.Duration) func(*wsConnection) {
|
||||
return func(wsc *wsConnection) {
|
||||
wsc.pingPeriod = pingPeriod
|
||||
}
|
||||
}
|
||||
|
||||
// OnStart implements cmn.Service by starting the read and write routines. It
|
||||
// blocks until the connection closes.
|
||||
func (wsc *wsConnection) OnStart() error {
|
||||
wsc.writeChan = make(chan types.RPCResponse, wsc.writeChanCapacity)
|
||||
|
||||
// Read subscriptions/unsubscriptions to events
|
||||
go wsc.readRoutine()
|
||||
// Write responses, BLOCKING.
|
||||
wsc.writeRoutine()
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// OnStop implements cmn.Service by unsubscribing remoteAddr from all subscriptions.
|
||||
func (wsc *wsConnection) OnStop() {
|
||||
// Both read and write loops close the websocket connection when they exit their loops.
|
||||
// The writeChan is never closed, to allow WriteRPCResponse() to fail.
|
||||
if wsc.eventSub != nil {
|
||||
wsc.eventSub.UnsubscribeAll(context.TODO(), wsc.remoteAddr)
|
||||
}
|
||||
}
|
||||
|
||||
// GetRemoteAddr returns the remote address of the underlying connection.
|
||||
// It implements WSRPCConnection
|
||||
func (wsc *wsConnection) GetRemoteAddr() string {
|
||||
return wsc.remoteAddr
|
||||
}
|
||||
|
||||
// GetEventSubscriber implements WSRPCConnection by returning event subscriber.
|
||||
func (wsc *wsConnection) GetEventSubscriber() types.EventSubscriber {
|
||||
return wsc.eventSub
|
||||
}
|
||||
|
||||
// WriteRPCResponse pushes a response to the writeChan, and blocks until it is accepted.
|
||||
// It implements WSRPCConnection. It is Goroutine-safe.
|
||||
func (wsc *wsConnection) WriteRPCResponse(resp types.RPCResponse) {
|
||||
select {
|
||||
case <-wsc.Quit():
|
||||
return
|
||||
case wsc.writeChan <- resp:
|
||||
}
|
||||
}
|
||||
|
||||
// TryWriteRPCResponse attempts to push a response to the writeChan, but does not block.
|
||||
// It implements WSRPCConnection. It is Goroutine-safe
|
||||
func (wsc *wsConnection) TryWriteRPCResponse(resp types.RPCResponse) bool {
|
||||
select {
|
||||
case <-wsc.Quit():
|
||||
return false
|
||||
case wsc.writeChan <- resp:
|
||||
return true
|
||||
default:
|
||||
return false
|
||||
}
|
||||
}
|
||||
|
||||
// Codec returns an amino codec used to decode parameters and encode results.
|
||||
// It implements WSRPCConnection.
|
||||
func (wsc *wsConnection) Codec() *amino.Codec {
|
||||
return wsc.cdc
|
||||
}
|
||||
|
||||
// Read from the socket and subscribe to or unsubscribe from events
|
||||
func (wsc *wsConnection) readRoutine() {
|
||||
defer func() {
|
||||
if r := recover(); r != nil {
|
||||
err, ok := r.(error)
|
||||
if !ok {
|
||||
err = fmt.Errorf("WSJSONRPC: %v", r)
|
||||
}
|
||||
wsc.Logger.Error("Panic in WSJSONRPC handler", "err", err, "stack", string(debug.Stack()))
|
||||
wsc.WriteRPCResponse(types.RPCInternalError("unknown", err))
|
||||
go wsc.readRoutine()
|
||||
} else {
|
||||
wsc.baseConn.Close() // nolint: errcheck
|
||||
}
|
||||
}()
|
||||
|
||||
wsc.baseConn.SetPongHandler(func(m string) error {
|
||||
return wsc.baseConn.SetReadDeadline(time.Now().Add(wsc.readWait))
|
||||
})
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-wsc.Quit():
|
||||
return
|
||||
default:
|
||||
// reset deadline for every type of message (control or data)
|
||||
if err := wsc.baseConn.SetReadDeadline(time.Now().Add(wsc.readWait)); err != nil {
|
||||
wsc.Logger.Error("failed to set read deadline", "err", err)
|
||||
}
|
||||
var in []byte
|
||||
_, in, err := wsc.baseConn.ReadMessage()
|
||||
if err != nil {
|
||||
if websocket.IsCloseError(err, websocket.CloseNormalClosure) {
|
||||
wsc.Logger.Info("Client closed the connection")
|
||||
} else {
|
||||
wsc.Logger.Error("Failed to read request", "err", err)
|
||||
}
|
||||
wsc.Stop()
|
||||
return
|
||||
}
|
||||
|
||||
var request types.RPCRequest
|
||||
err = json.Unmarshal(in, &request)
|
||||
if err != nil {
|
||||
wsc.WriteRPCResponse(types.RPCParseError("", errors.Wrap(err, "Error unmarshaling request")))
|
||||
continue
|
||||
}
|
||||
|
||||
// A Notification is a Request object without an "id" member.
|
||||
// The Server MUST NOT reply to a Notification, including those that are within a batch request.
|
||||
if request.ID == "" {
|
||||
wsc.Logger.Debug("WSJSONRPC received a notification, skipping... (please send a non-empty ID if you want to call a method)")
|
||||
continue
|
||||
}
|
||||
|
||||
// Now, fetch the RPCFunc and execute it.
|
||||
|
||||
rpcFunc := wsc.funcMap[request.Method]
|
||||
if rpcFunc == nil {
|
||||
wsc.WriteRPCResponse(types.RPCMethodNotFoundError(request.ID))
|
||||
continue
|
||||
}
|
||||
var args []reflect.Value
|
||||
if rpcFunc.ws {
|
||||
wsCtx := types.WSRPCContext{Request: request, WSRPCConnection: wsc}
|
||||
if len(request.Params) > 0 {
|
||||
args, err = jsonParamsToArgsWS(rpcFunc, wsc.cdc, request.Params, wsCtx)
|
||||
}
|
||||
} else {
|
||||
if len(request.Params) > 0 {
|
||||
args, err = jsonParamsToArgsRPC(rpcFunc, wsc.cdc, request.Params)
|
||||
}
|
||||
}
|
||||
if err != nil {
|
||||
wsc.WriteRPCResponse(types.RPCInternalError(request.ID, errors.Wrap(err, "Error converting json params to arguments")))
|
||||
continue
|
||||
}
|
||||
returns := rpcFunc.f.Call(args)
|
||||
|
||||
// TODO: Need to encode args/returns to string if we want to log them
|
||||
wsc.Logger.Info("WSJSONRPC", "method", request.Method)
|
||||
|
||||
result, err := unreflectResult(returns)
|
||||
if err != nil {
|
||||
wsc.WriteRPCResponse(types.RPCInternalError(request.ID, err))
|
||||
continue
|
||||
}
|
||||
|
||||
wsc.WriteRPCResponse(types.NewRPCSuccessResponse(wsc.cdc, request.ID, result))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// receives on a write channel and writes out on the socket
|
||||
func (wsc *wsConnection) writeRoutine() {
|
||||
pingTicker := time.NewTicker(wsc.pingPeriod)
|
||||
defer func() {
|
||||
pingTicker.Stop()
|
||||
if err := wsc.baseConn.Close(); err != nil {
|
||||
wsc.Logger.Error("Error closing connection", "err", err)
|
||||
}
|
||||
}()
|
||||
|
||||
// https://github.com/gorilla/websocket/issues/97
|
||||
pongs := make(chan string, 1)
|
||||
wsc.baseConn.SetPingHandler(func(m string) error {
|
||||
select {
|
||||
case pongs <- m:
|
||||
default:
|
||||
}
|
||||
return nil
|
||||
})
|
||||
|
||||
for {
|
||||
select {
|
||||
case m := <-pongs:
|
||||
err := wsc.writeMessageWithDeadline(websocket.PongMessage, []byte(m))
|
||||
if err != nil {
|
||||
wsc.Logger.Info("Failed to write pong (client may disconnect)", "err", err)
|
||||
}
|
||||
case <-pingTicker.C:
|
||||
err := wsc.writeMessageWithDeadline(websocket.PingMessage, []byte{})
|
||||
if err != nil {
|
||||
wsc.Logger.Error("Failed to write ping", "err", err)
|
||||
wsc.Stop()
|
||||
return
|
||||
}
|
||||
case msg := <-wsc.writeChan:
|
||||
jsonBytes, err := json.MarshalIndent(msg, "", " ")
|
||||
if err != nil {
|
||||
wsc.Logger.Error("Failed to marshal RPCResponse to JSON", "err", err)
|
||||
} else {
|
||||
if err = wsc.writeMessageWithDeadline(websocket.TextMessage, jsonBytes); err != nil {
|
||||
wsc.Logger.Error("Failed to write response", "err", err)
|
||||
wsc.Stop()
|
||||
return
|
||||
}
|
||||
}
|
||||
case <-wsc.Quit():
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// All writes to the websocket must (re)set the write deadline.
|
||||
// If some writes don't set it while others do, they may timeout incorrectly (https://github.com/tendermint/tendermint/issues/553)
|
||||
func (wsc *wsConnection) writeMessageWithDeadline(msgType int, msg []byte) error {
|
||||
if err := wsc.baseConn.SetWriteDeadline(time.Now().Add(wsc.writeWait)); err != nil {
|
||||
return err
|
||||
}
|
||||
return wsc.baseConn.WriteMessage(msgType, msg)
|
||||
}
|
||||
|
||||
//----------------------------------------
|
||||
|
||||
// WebsocketManager provides a WS handler for incoming connections and passes a
|
||||
// map of functions along with any additional params to new connections.
|
||||
// NOTE: The websocket path is defined externally, e.g. in node/node.go
|
||||
type WebsocketManager struct {
|
||||
websocket.Upgrader
|
||||
|
||||
funcMap map[string]*RPCFunc
|
||||
cdc *amino.Codec
|
||||
logger log.Logger
|
||||
wsConnOptions []func(*wsConnection)
|
||||
}
|
||||
|
||||
// NewWebsocketManager returns a new WebsocketManager that passes a map of
|
||||
// functions, connection options and logger to new WS connections.
|
||||
func NewWebsocketManager(funcMap map[string]*RPCFunc, cdc *amino.Codec, wsConnOptions ...func(*wsConnection)) *WebsocketManager {
|
||||
return &WebsocketManager{
|
||||
funcMap: funcMap,
|
||||
cdc: cdc,
|
||||
Upgrader: websocket.Upgrader{
|
||||
CheckOrigin: func(r *http.Request) bool {
|
||||
// TODO ???
|
||||
return true
|
||||
},
|
||||
},
|
||||
logger: log.NewNopLogger(),
|
||||
wsConnOptions: wsConnOptions,
|
||||
}
|
||||
}
|
||||
|
||||
// SetLogger sets the logger.
|
||||
func (wm *WebsocketManager) SetLogger(l log.Logger) {
|
||||
wm.logger = l
|
||||
}
|
||||
|
||||
// WebsocketHandler upgrades the request/response (via http.Hijack) and starts
|
||||
// the wsConnection.
|
||||
func (wm *WebsocketManager) WebsocketHandler(w http.ResponseWriter, r *http.Request) {
|
||||
wsConn, err := wm.Upgrade(w, r, nil)
|
||||
if err != nil {
|
||||
// TODO - return http error
|
||||
wm.logger.Error("Failed to upgrade to websocket connection", "err", err)
|
||||
return
|
||||
}
|
||||
|
||||
// register connection
|
||||
con := NewWSConnection(wsConn, wm.funcMap, wm.cdc, wm.wsConnOptions...)
|
||||
con.SetLogger(wm.logger.With("remote", wsConn.RemoteAddr()))
|
||||
wm.logger.Info("New websocket connection", "remote", con.remoteAddr)
|
||||
err = con.Start() // Blocking
|
||||
if err != nil {
|
||||
wm.logger.Error("Error starting connection", "err", err)
|
||||
}
|
||||
}
|
||||
|
||||
// rpc.websocket
|
||||
//-----------------------------------------------------------------------------
|
||||
|
||||
// NOTE: assume returns is result struct and error. If error is not nil, return it
|
||||
func unreflectResult(returns []reflect.Value) (interface{}, error) {
|
||||
errV := returns[1]
|
||||
if errV.Interface() != nil {
|
||||
return nil, errors.Errorf("%v", errV.Interface())
|
||||
}
|
||||
rv := returns[0]
|
||||
// the result is a registered interface,
|
||||
// we need a pointer to it so we can marshal with type byte
|
||||
rvp := reflect.New(rv.Type())
|
||||
rvp.Elem().Set(rv)
|
||||
return rvp.Interface(), nil
|
||||
}
|
||||
|
||||
// writes a list of available rpc endpoints as an html page
|
||||
func writeListOfEndpoints(w http.ResponseWriter, r *http.Request, funcMap map[string]*RPCFunc) {
|
||||
noArgNames := []string{}
|
||||
argNames := []string{}
|
||||
for name, funcData := range funcMap {
|
||||
if len(funcData.args) == 0 {
|
||||
noArgNames = append(noArgNames, name)
|
||||
} else {
|
||||
argNames = append(argNames, name)
|
||||
}
|
||||
}
|
||||
sort.Strings(noArgNames)
|
||||
sort.Strings(argNames)
|
||||
buf := new(bytes.Buffer)
|
||||
buf.WriteString("<html><body>")
|
||||
buf.WriteString("<br>Available endpoints:<br>")
|
||||
|
||||
for _, name := range noArgNames {
|
||||
link := fmt.Sprintf("//%s/%s", r.Host, name)
|
||||
buf.WriteString(fmt.Sprintf("<a href=\"%s\">%s</a></br>", link, link))
|
||||
}
|
||||
|
||||
buf.WriteString("<br>Endpoints that require arguments:<br>")
|
||||
for _, name := range argNames {
|
||||
link := fmt.Sprintf("//%s/%s?", r.Host, name)
|
||||
funcData := funcMap[name]
|
||||
for i, argName := range funcData.argNames {
|
||||
link += argName + "=_"
|
||||
if i < len(funcData.argNames)-1 {
|
||||
link += "&"
|
||||
}
|
||||
}
|
||||
buf.WriteString(fmt.Sprintf("<a href=\"%s\">%s</a></br>", link, link))
|
||||
}
|
||||
buf.WriteString("</body></html>")
|
||||
w.Header().Set("Content-Type", "text/html")
|
||||
w.WriteHeader(200)
|
||||
w.Write(buf.Bytes()) // nolint: errcheck
|
||||
}
|
110
rpc/lib/server/handlers_test.go
Normal file
110
rpc/lib/server/handlers_test.go
Normal file
@ -0,0 +1,110 @@
|
||||
package rpcserver_test
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"encoding/json"
|
||||
"io/ioutil"
|
||||
"net/http"
|
||||
"net/http/httptest"
|
||||
"strings"
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/require"
|
||||
|
||||
amino "github.com/tendermint/go-amino"
|
||||
rs "github.com/tendermint/tendermint/rpc/lib/server"
|
||||
types "github.com/tendermint/tendermint/rpc/lib/types"
|
||||
"github.com/tendermint/tmlibs/log"
|
||||
)
|
||||
|
||||
func testMux() *http.ServeMux {
|
||||
funcMap := map[string]*rs.RPCFunc{
|
||||
"c": rs.NewRPCFunc(func(s string, i int) (string, error) { return "foo", nil }, "s,i"),
|
||||
}
|
||||
cdc := amino.NewCodec()
|
||||
mux := http.NewServeMux()
|
||||
buf := new(bytes.Buffer)
|
||||
logger := log.NewTMLogger(buf)
|
||||
rs.RegisterRPCFuncs(mux, funcMap, cdc, logger)
|
||||
|
||||
return mux
|
||||
}
|
||||
|
||||
func statusOK(code int) bool { return code >= 200 && code <= 299 }
|
||||
|
||||
// Ensure that nefarious/unintended inputs to `params`
|
||||
// do not crash our RPC handlers.
|
||||
// See Issue https://github.com/tendermint/tendermint/issues/708.
|
||||
func TestRPCParams(t *testing.T) {
|
||||
mux := testMux()
|
||||
tests := []struct {
|
||||
payload string
|
||||
wantErr string
|
||||
}{
|
||||
// bad
|
||||
{`{"jsonrpc": "2.0", "id": "0"}`, "Method not found"},
|
||||
{`{"jsonrpc": "2.0", "method": "y", "id": "0"}`, "Method not found"},
|
||||
{`{"method": "c", "id": "0", "params": a}`, "invalid character"},
|
||||
{`{"method": "c", "id": "0", "params": ["a"]}`, "got 1"},
|
||||
{`{"method": "c", "id": "0", "params": ["a", "b"]}`, "invalid character"},
|
||||
{`{"method": "c", "id": "0", "params": [1, 1]}`, "of type string"},
|
||||
|
||||
// good
|
||||
{`{"jsonrpc": "2.0", "method": "c", "id": "0", "params": null}`, ""},
|
||||
{`{"method": "c", "id": "0", "params": {}}`, ""},
|
||||
{`{"method": "c", "id": "0", "params": ["a", "10"]}`, ""},
|
||||
}
|
||||
|
||||
for i, tt := range tests {
|
||||
req, _ := http.NewRequest("POST", "http://localhost/", strings.NewReader(tt.payload))
|
||||
rec := httptest.NewRecorder()
|
||||
mux.ServeHTTP(rec, req)
|
||||
res := rec.Result()
|
||||
// Always expecting back a JSONRPCResponse
|
||||
assert.True(t, statusOK(res.StatusCode), "#%d: should always return 2XX", i)
|
||||
blob, err := ioutil.ReadAll(res.Body)
|
||||
if err != nil {
|
||||
t.Errorf("#%d: err reading body: %v", i, err)
|
||||
continue
|
||||
}
|
||||
|
||||
recv := new(types.RPCResponse)
|
||||
assert.Nil(t, json.Unmarshal(blob, recv), "#%d: expecting successful parsing of an RPCResponse:\nblob: %s", i, blob)
|
||||
assert.NotEqual(t, recv, new(types.RPCResponse), "#%d: not expecting a blank RPCResponse", i)
|
||||
|
||||
if tt.wantErr == "" {
|
||||
assert.Nil(t, recv.Error, "#%d: not expecting an error", i)
|
||||
} else {
|
||||
assert.True(t, recv.Error.Code < 0, "#%d: not expecting a positive JSONRPC code", i)
|
||||
// The wanted error is either in the message or the data
|
||||
assert.Contains(t, recv.Error.Message+recv.Error.Data, tt.wantErr, "#%d: expected substring", i)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestRPCNotification(t *testing.T) {
|
||||
mux := testMux()
|
||||
body := strings.NewReader(`{"jsonrpc": "2.0"}`)
|
||||
req, _ := http.NewRequest("POST", "http://localhost/", body)
|
||||
rec := httptest.NewRecorder()
|
||||
mux.ServeHTTP(rec, req)
|
||||
res := rec.Result()
|
||||
|
||||
// Always expecting back a JSONRPCResponse
|
||||
require.True(t, statusOK(res.StatusCode), "should always return 2XX")
|
||||
blob, err := ioutil.ReadAll(res.Body)
|
||||
require.Nil(t, err, "reading from the body should not give back an error")
|
||||
require.Equal(t, len(blob), 0, "a notification SHOULD NOT be responded to by the server")
|
||||
}
|
||||
|
||||
func TestUnknownRPCPath(t *testing.T) {
|
||||
mux := testMux()
|
||||
req, _ := http.NewRequest("GET", "http://localhost/unknownrpcpath", nil)
|
||||
rec := httptest.NewRecorder()
|
||||
mux.ServeHTTP(rec, req)
|
||||
res := rec.Result()
|
||||
|
||||
// Always expecting back a 404 error
|
||||
require.Equal(t, http.StatusNotFound, res.StatusCode, "should always return 404")
|
||||
}
|
90
rpc/lib/server/http_params.go
Normal file
90
rpc/lib/server/http_params.go
Normal file
@ -0,0 +1,90 @@
|
||||
package rpcserver
|
||||
|
||||
import (
|
||||
"encoding/hex"
|
||||
"net/http"
|
||||
"regexp"
|
||||
"strconv"
|
||||
|
||||
"github.com/pkg/errors"
|
||||
)
|
||||
|
||||
var (
|
||||
// Parts of regular expressions
|
||||
atom = "[A-Z0-9!#$%&'*+\\-/=?^_`{|}~]+"
|
||||
dotAtom = atom + `(?:\.` + atom + `)*`
|
||||
domain = `[A-Z0-9.-]+\.[A-Z]{2,4}`
|
||||
|
||||
RE_HEX = regexp.MustCompile(`^(?i)[a-f0-9]+$`)
|
||||
RE_EMAIL = regexp.MustCompile(`^(?i)(` + dotAtom + `)@(` + dotAtom + `)$`)
|
||||
RE_ADDRESS = regexp.MustCompile(`^(?i)[a-z0-9]{25,34}$`)
|
||||
RE_HOST = regexp.MustCompile(`^(?i)(` + domain + `)$`)
|
||||
|
||||
//RE_ID12 = regexp.MustCompile(`^[a-zA-Z0-9]{12}$`)
|
||||
)
|
||||
|
||||
func GetParam(r *http.Request, param string) string {
|
||||
s := r.URL.Query().Get(param)
|
||||
if s == "" {
|
||||
s = r.FormValue(param)
|
||||
}
|
||||
return s
|
||||
}
|
||||
|
||||
func GetParamByteSlice(r *http.Request, param string) ([]byte, error) {
|
||||
s := GetParam(r, param)
|
||||
return hex.DecodeString(s)
|
||||
}
|
||||
|
||||
func GetParamInt64(r *http.Request, param string) (int64, error) {
|
||||
s := GetParam(r, param)
|
||||
i, err := strconv.ParseInt(s, 10, 64)
|
||||
if err != nil {
|
||||
return 0, errors.Errorf(param, err.Error())
|
||||
}
|
||||
return i, nil
|
||||
}
|
||||
|
||||
func GetParamInt32(r *http.Request, param string) (int32, error) {
|
||||
s := GetParam(r, param)
|
||||
i, err := strconv.ParseInt(s, 10, 32)
|
||||
if err != nil {
|
||||
return 0, errors.Errorf(param, err.Error())
|
||||
}
|
||||
return int32(i), nil
|
||||
}
|
||||
|
||||
func GetParamUint64(r *http.Request, param string) (uint64, error) {
|
||||
s := GetParam(r, param)
|
||||
i, err := strconv.ParseUint(s, 10, 64)
|
||||
if err != nil {
|
||||
return 0, errors.Errorf(param, err.Error())
|
||||
}
|
||||
return i, nil
|
||||
}
|
||||
|
||||
func GetParamUint(r *http.Request, param string) (uint, error) {
|
||||
s := GetParam(r, param)
|
||||
i, err := strconv.ParseUint(s, 10, 64)
|
||||
if err != nil {
|
||||
return 0, errors.Errorf(param, err.Error())
|
||||
}
|
||||
return uint(i), nil
|
||||
}
|
||||
|
||||
func GetParamRegexp(r *http.Request, param string, re *regexp.Regexp) (string, error) {
|
||||
s := GetParam(r, param)
|
||||
if !re.MatchString(s) {
|
||||
return "", errors.Errorf(param, "Did not match regular expression %v", re.String())
|
||||
}
|
||||
return s, nil
|
||||
}
|
||||
|
||||
func GetParamFloat64(r *http.Request, param string) (float64, error) {
|
||||
s := GetParam(r, param)
|
||||
f, err := strconv.ParseFloat(s, 64)
|
||||
if err != nil {
|
||||
return 0, errors.Errorf(param, err.Error())
|
||||
}
|
||||
return f, nil
|
||||
}
|
156
rpc/lib/server/http_server.go
Normal file
156
rpc/lib/server/http_server.go
Normal file
@ -0,0 +1,156 @@
|
||||
// Commons for HTTP handling
|
||||
package rpcserver
|
||||
|
||||
import (
|
||||
"bufio"
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"net"
|
||||
"net/http"
|
||||
"runtime/debug"
|
||||
"strings"
|
||||
"time"
|
||||
|
||||
"github.com/pkg/errors"
|
||||
|
||||
types "github.com/tendermint/tendermint/rpc/lib/types"
|
||||
"github.com/tendermint/tmlibs/log"
|
||||
)
|
||||
|
||||
func StartHTTPServer(listenAddr string, handler http.Handler, logger log.Logger) (listener net.Listener, err error) {
|
||||
var proto, addr string
|
||||
parts := strings.SplitN(listenAddr, "://", 2)
|
||||
if len(parts) != 2 {
|
||||
return nil, errors.Errorf("Invalid listening address %s (use fully formed addresses, including the tcp:// or unix:// prefix)", listenAddr)
|
||||
}
|
||||
proto, addr = parts[0], parts[1]
|
||||
|
||||
logger.Info(fmt.Sprintf("Starting RPC HTTP server on %s", listenAddr))
|
||||
listener, err = net.Listen(proto, addr)
|
||||
if err != nil {
|
||||
return nil, errors.Errorf("Failed to listen on %v: %v", listenAddr, err)
|
||||
}
|
||||
|
||||
go func() {
|
||||
err := http.Serve(
|
||||
listener,
|
||||
RecoverAndLogHandler(handler, logger),
|
||||
)
|
||||
logger.Error("RPC HTTP server stopped", "err", err)
|
||||
}()
|
||||
return listener, nil
|
||||
}
|
||||
|
||||
func StartHTTPAndTLSServer(listenAddr string, handler http.Handler, certFile, keyFile string, logger log.Logger) (listener net.Listener, err error) {
|
||||
var proto, addr string
|
||||
parts := strings.SplitN(listenAddr, "://", 2)
|
||||
if len(parts) != 2 {
|
||||
return nil, errors.Errorf("Invalid listening address %s (use fully formed addresses, including the tcp:// or unix:// prefix)", listenAddr)
|
||||
}
|
||||
proto, addr = parts[0], parts[1]
|
||||
|
||||
logger.Info(fmt.Sprintf("Starting RPC HTTPS server on %s (cert: %q, key: %q)", listenAddr, certFile, keyFile))
|
||||
listener, err = net.Listen(proto, addr)
|
||||
if err != nil {
|
||||
return nil, errors.Errorf("Failed to listen on %v: %v", listenAddr, err)
|
||||
}
|
||||
|
||||
go func() {
|
||||
err := http.ServeTLS(
|
||||
listener,
|
||||
RecoverAndLogHandler(handler, logger),
|
||||
certFile,
|
||||
keyFile,
|
||||
)
|
||||
logger.Error("RPC HTTPS server stopped", "err", err)
|
||||
}()
|
||||
return listener, nil
|
||||
}
|
||||
|
||||
func WriteRPCResponseHTTPError(w http.ResponseWriter, httpCode int, res types.RPCResponse) {
|
||||
jsonBytes, err := json.MarshalIndent(res, "", " ")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
w.Header().Set("Content-Type", "application/json")
|
||||
w.WriteHeader(httpCode)
|
||||
w.Write(jsonBytes) // nolint: errcheck, gas
|
||||
}
|
||||
|
||||
func WriteRPCResponseHTTP(w http.ResponseWriter, res types.RPCResponse) {
|
||||
jsonBytes, err := json.MarshalIndent(res, "", " ")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
w.Header().Set("Content-Type", "application/json")
|
||||
w.WriteHeader(200)
|
||||
w.Write(jsonBytes) // nolint: errcheck, gas
|
||||
}
|
||||
|
||||
//-----------------------------------------------------------------------------
|
||||
|
||||
// Wraps an HTTP handler, adding error logging.
|
||||
// If the inner function panics, the outer function recovers, logs, sends an
|
||||
// HTTP 500 error response.
|
||||
func RecoverAndLogHandler(handler http.Handler, logger log.Logger) http.Handler {
|
||||
return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
|
||||
// Wrap the ResponseWriter to remember the status
|
||||
rww := &ResponseWriterWrapper{-1, w}
|
||||
begin := time.Now()
|
||||
|
||||
// Common headers
|
||||
origin := r.Header.Get("Origin")
|
||||
rww.Header().Set("Access-Control-Allow-Origin", origin)
|
||||
rww.Header().Set("Access-Control-Allow-Credentials", "true")
|
||||
rww.Header().Set("Access-Control-Expose-Headers", "X-Server-Time")
|
||||
rww.Header().Set("X-Server-Time", fmt.Sprintf("%v", begin.Unix()))
|
||||
|
||||
defer func() {
|
||||
// Send a 500 error if a panic happens during a handler.
|
||||
// Without this, Chrome & Firefox were retrying aborted ajax requests,
|
||||
// at least to my localhost.
|
||||
if e := recover(); e != nil {
|
||||
|
||||
// If RPCResponse
|
||||
if res, ok := e.(types.RPCResponse); ok {
|
||||
WriteRPCResponseHTTP(rww, res)
|
||||
} else {
|
||||
// For the rest,
|
||||
logger.Error("Panic in RPC HTTP handler", "err", e, "stack", string(debug.Stack()))
|
||||
rww.WriteHeader(http.StatusInternalServerError)
|
||||
WriteRPCResponseHTTP(rww, types.RPCInternalError("", e.(error)))
|
||||
}
|
||||
}
|
||||
|
||||
// Finally, log.
|
||||
durationMS := time.Since(begin).Nanoseconds() / 1000000
|
||||
if rww.Status == -1 {
|
||||
rww.Status = 200
|
||||
}
|
||||
logger.Info("Served RPC HTTP response",
|
||||
"method", r.Method, "url", r.URL,
|
||||
"status", rww.Status, "duration", durationMS,
|
||||
"remoteAddr", r.RemoteAddr,
|
||||
)
|
||||
}()
|
||||
|
||||
handler.ServeHTTP(rww, r)
|
||||
})
|
||||
}
|
||||
|
||||
// Remember the status for logging
|
||||
type ResponseWriterWrapper struct {
|
||||
Status int
|
||||
http.ResponseWriter
|
||||
}
|
||||
|
||||
func (w *ResponseWriterWrapper) WriteHeader(status int) {
|
||||
w.Status = status
|
||||
w.ResponseWriter.WriteHeader(status)
|
||||
}
|
||||
|
||||
// implements http.Hijacker
|
||||
func (w *ResponseWriterWrapper) Hijack() (net.Conn, *bufio.ReadWriter, error) {
|
||||
return w.ResponseWriter.(http.Hijacker).Hijack()
|
||||
}
|
176
rpc/lib/server/parse_test.go
Normal file
176
rpc/lib/server/parse_test.go
Normal file
@ -0,0 +1,176 @@
|
||||
package rpcserver
|
||||
|
||||
import (
|
||||
"encoding/json"
|
||||
"strconv"
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
amino "github.com/tendermint/go-amino"
|
||||
cmn "github.com/tendermint/tmlibs/common"
|
||||
)
|
||||
|
||||
func TestParseJSONMap(t *testing.T) {
|
||||
assert := assert.New(t)
|
||||
|
||||
input := []byte(`{"value":"1234","height":22}`)
|
||||
|
||||
// naive is float,string
|
||||
var p1 map[string]interface{}
|
||||
err := json.Unmarshal(input, &p1)
|
||||
if assert.Nil(err) {
|
||||
h, ok := p1["height"].(float64)
|
||||
if assert.True(ok, "%#v", p1["height"]) {
|
||||
assert.EqualValues(22, h)
|
||||
}
|
||||
v, ok := p1["value"].(string)
|
||||
if assert.True(ok, "%#v", p1["value"]) {
|
||||
assert.EqualValues("1234", v)
|
||||
}
|
||||
}
|
||||
|
||||
// preloading map with values doesn't help
|
||||
tmp := 0
|
||||
p2 := map[string]interface{}{
|
||||
"value": &cmn.HexBytes{},
|
||||
"height": &tmp,
|
||||
}
|
||||
err = json.Unmarshal(input, &p2)
|
||||
if assert.Nil(err) {
|
||||
h, ok := p2["height"].(float64)
|
||||
if assert.True(ok, "%#v", p2["height"]) {
|
||||
assert.EqualValues(22, h)
|
||||
}
|
||||
v, ok := p2["value"].(string)
|
||||
if assert.True(ok, "%#v", p2["value"]) {
|
||||
assert.EqualValues("1234", v)
|
||||
}
|
||||
}
|
||||
|
||||
// preload here with *pointers* to the desired types
|
||||
// struct has unknown types, but hard-coded keys
|
||||
tmp = 0
|
||||
p3 := struct {
|
||||
Value interface{} `json:"value"`
|
||||
Height interface{} `json:"height"`
|
||||
}{
|
||||
Height: &tmp,
|
||||
Value: &cmn.HexBytes{},
|
||||
}
|
||||
err = json.Unmarshal(input, &p3)
|
||||
if assert.Nil(err) {
|
||||
h, ok := p3.Height.(*int)
|
||||
if assert.True(ok, "%#v", p3.Height) {
|
||||
assert.Equal(22, *h)
|
||||
}
|
||||
v, ok := p3.Value.(*cmn.HexBytes)
|
||||
if assert.True(ok, "%#v", p3.Value) {
|
||||
assert.EqualValues([]byte{0x12, 0x34}, *v)
|
||||
}
|
||||
}
|
||||
|
||||
// simplest solution, but hard-coded
|
||||
p4 := struct {
|
||||
Value cmn.HexBytes `json:"value"`
|
||||
Height int `json:"height"`
|
||||
}{}
|
||||
err = json.Unmarshal(input, &p4)
|
||||
if assert.Nil(err) {
|
||||
assert.EqualValues(22, p4.Height)
|
||||
assert.EqualValues([]byte{0x12, 0x34}, p4.Value)
|
||||
}
|
||||
|
||||
// so, let's use this trick...
|
||||
// dynamic keys on map, and we can deserialize to the desired types
|
||||
var p5 map[string]*json.RawMessage
|
||||
err = json.Unmarshal(input, &p5)
|
||||
if assert.Nil(err) {
|
||||
var h int
|
||||
err = json.Unmarshal(*p5["height"], &h)
|
||||
if assert.Nil(err) {
|
||||
assert.Equal(22, h)
|
||||
}
|
||||
|
||||
var v cmn.HexBytes
|
||||
err = json.Unmarshal(*p5["value"], &v)
|
||||
if assert.Nil(err) {
|
||||
assert.Equal(cmn.HexBytes{0x12, 0x34}, v)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestParseJSONArray(t *testing.T) {
|
||||
assert := assert.New(t)
|
||||
|
||||
input := []byte(`["1234",22]`)
|
||||
|
||||
// naive is float,string
|
||||
var p1 []interface{}
|
||||
err := json.Unmarshal(input, &p1)
|
||||
if assert.Nil(err) {
|
||||
v, ok := p1[0].(string)
|
||||
if assert.True(ok, "%#v", p1[0]) {
|
||||
assert.EqualValues("1234", v)
|
||||
}
|
||||
h, ok := p1[1].(float64)
|
||||
if assert.True(ok, "%#v", p1[1]) {
|
||||
assert.EqualValues(22, h)
|
||||
}
|
||||
}
|
||||
|
||||
// preloading map with values helps here (unlike map - p2 above)
|
||||
tmp := 0
|
||||
p2 := []interface{}{&cmn.HexBytes{}, &tmp}
|
||||
err = json.Unmarshal(input, &p2)
|
||||
if assert.Nil(err) {
|
||||
v, ok := p2[0].(*cmn.HexBytes)
|
||||
if assert.True(ok, "%#v", p2[0]) {
|
||||
assert.EqualValues([]byte{0x12, 0x34}, *v)
|
||||
}
|
||||
h, ok := p2[1].(*int)
|
||||
if assert.True(ok, "%#v", p2[1]) {
|
||||
assert.EqualValues(22, *h)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestParseRPC(t *testing.T) {
|
||||
assert := assert.New(t)
|
||||
|
||||
demo := func(height int, name string) {}
|
||||
call := NewRPCFunc(demo, "height,name")
|
||||
cdc := amino.NewCodec()
|
||||
|
||||
cases := []struct {
|
||||
raw string
|
||||
height int64
|
||||
name string
|
||||
fail bool
|
||||
}{
|
||||
// should parse
|
||||
{`["7", "flew"]`, 7, "flew", false},
|
||||
{`{"name": "john", "height": "22"}`, 22, "john", false},
|
||||
// defaults
|
||||
{`{"name": "solo", "unused": "stuff"}`, 0, "solo", false},
|
||||
// should fail - wrong types/length
|
||||
{`["flew", 7]`, 0, "", true},
|
||||
{`[7,"flew",100]`, 0, "", true},
|
||||
{`{"name": -12, "height": "fred"}`, 0, "", true},
|
||||
}
|
||||
for idx, tc := range cases {
|
||||
i := strconv.Itoa(idx)
|
||||
data := []byte(tc.raw)
|
||||
vals, err := jsonParamsToArgs(call, cdc, data, 0)
|
||||
if tc.fail {
|
||||
assert.NotNil(err, i)
|
||||
} else {
|
||||
assert.Nil(err, "%s: %+v", i, err)
|
||||
if assert.Equal(2, len(vals), i) {
|
||||
assert.Equal(tc.height, vals[0].Int(), i)
|
||||
assert.Equal(tc.name, vals[1].String(), i)
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
Reference in New Issue
Block a user