tendermint/rpc/lib/client/ws_client.go

484 lines
13 KiB
Go
Raw Normal View History

2016-01-12 13:50:06 -08:00
package rpcclient
import (
"context"
2016-01-13 15:37:35 -08:00
"encoding/json"
"fmt"
"math/rand"
"net"
2016-01-12 13:50:06 -08:00
"net/http"
"sync"
2016-01-12 13:50:06 -08:00
"time"
"github.com/gorilla/websocket"
2017-03-09 07:00:05 -08:00
"github.com/pkg/errors"
2017-08-03 11:19:39 -07:00
metrics "github.com/rcrowley/go-metrics"
types "github.com/tendermint/tendermint/rpc/lib/types"
cmn "github.com/tendermint/tmlibs/common"
2016-01-12 13:50:06 -08:00
)
const (
defaultMaxReconnectAttempts = 25
defaultWriteWait = 0
defaultReadWait = 0
defaultPingPeriod = 0
2016-01-12 13:50:06 -08:00
)
2017-08-07 14:56:38 -07:00
// WSClient is a WebSocket client. The methods of WSClient are safe for use by
// multiple goroutines.
2016-01-12 13:50:06 -08:00
type WSClient struct {
cmn.BaseService
conn *websocket.Conn
2016-02-18 18:05:24 -08:00
Address string // IP:PORT or /path/to/socket
Endpoint string // /websocket/url/endpoint
Dialer func(string, string) (net.Conn, error)
// Time between sending a ping and receiving a pong. See
// https://godoc.org/github.com/rcrowley/go-metrics#Timer.
2017-08-03 11:19:39 -07:00
PingPongLatencyTimer metrics.Timer
2017-10-24 09:38:12 -07:00
// Single user facing channel to read RPCResponses from, closed only when the client is being stopped.
ResponsesCh chan types.RPCResponse
// Callback, which will be called each time after successful reconnect.
onReconnect func()
// internal channels
send chan types.RPCRequest // user requests
backlog chan types.RPCRequest // stores a single user request received during a conn failure
reconnectAfter chan error // reconnect requests
readRoutineQuit chan struct{} // a way for readRoutine to close writeRoutine
2017-08-05 09:07:02 -07:00
wg sync.WaitGroup
mtx sync.RWMutex
sentLastPingAt time.Time
2017-08-07 14:56:38 -07:00
reconnecting bool
// Maximum reconnect attempts (0 or greater; default: 25).
maxReconnectAttempts int
// Time allowed to write a message to the server. 0 means block until operation succeeds.
writeWait time.Duration
// Time allowed to read the next message from the server. 0 means block until operation succeeds.
readWait time.Duration
// Send pings to server with this period. Must be less than readWait. If 0, no pings will be sent.
pingPeriod time.Duration
2016-01-12 13:50:06 -08:00
}
2017-08-07 14:56:38 -07:00
// NewWSClient returns a new client. See the commentary on the func(*WSClient)
// functions for a detailed description of how to configure ping period and
2017-08-08 14:33:17 -07:00
// pong wait time. The endpoint argument must begin with a `/`.
func NewWSClient(remoteAddr, endpoint string, options ...func(*WSClient)) *WSClient {
addr, dialer := makeHTTPDialer(remoteAddr)
c := &WSClient{
2017-08-03 11:19:39 -07:00
Address: addr,
Dialer: dialer,
Endpoint: endpoint,
PingPongLatencyTimer: metrics.NewTimer(),
maxReconnectAttempts: defaultMaxReconnectAttempts,
readWait: defaultReadWait,
writeWait: defaultWriteWait,
pingPeriod: defaultPingPeriod,
}
c.BaseService = *cmn.NewBaseService(nil, "WSClient", c)
for _, option := range options {
option(c)
}
return c
}
2017-08-24 13:25:56 -07:00
// MaxReconnectAttempts sets the maximum number of reconnect attempts before returning an error.
// It should only be used in the constructor and is not Goroutine-safe.
func MaxReconnectAttempts(max int) func(*WSClient) {
return func(c *WSClient) {
c.maxReconnectAttempts = max
}
}
2017-08-24 13:25:56 -07:00
// ReadWait sets the amount of time to wait before a websocket read times out.
// It should only be used in the constructor and is not Goroutine-safe.
func ReadWait(readWait time.Duration) func(*WSClient) {
return func(c *WSClient) {
c.readWait = readWait
}
}
2017-08-24 13:25:56 -07:00
// WriteWait sets the amount of time to wait before a websocket write times out.
// It should only be used in the constructor and is not Goroutine-safe.
func WriteWait(writeWait time.Duration) func(*WSClient) {
return func(c *WSClient) {
c.writeWait = writeWait
}
}
2017-08-24 13:25:56 -07:00
// 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(*WSClient) {
return func(c *WSClient) {
c.pingPeriod = pingPeriod
2016-01-12 13:50:06 -08:00
}
}
// OnReconnect sets the callback, which will be called every time after
// successful reconnect.
func OnReconnect(cb func()) func(*WSClient) {
return func(c *WSClient) {
c.onReconnect = cb
}
}
// String returns WS client full address.
func (c *WSClient) String() string {
return fmt.Sprintf("%s (%s)", c.Address, c.Endpoint)
2016-02-02 23:01:28 -08:00
}
// OnStart implements cmn.Service by dialing a server and creating read and
// write routines.
func (c *WSClient) OnStart() error {
err := c.dial()
2016-01-12 13:50:06 -08:00
if err != nil {
return err
}
2017-10-24 09:38:12 -07:00
c.ResponsesCh = make(chan types.RPCResponse)
c.send = make(chan types.RPCRequest)
// 1 additional error may come from the read/write
// goroutine depending on which failed first.
c.reconnectAfter = make(chan error, 1)
// capacity for 1 request. a user won't be able to send more because the send
// channel is unbuffered.
c.backlog = make(chan types.RPCRequest, 1)
c.startReadWriteRoutines()
go c.reconnectRoutine()
2016-01-12 13:50:06 -08:00
return nil
}
// OnStop implements cmn.Service.
func (c *WSClient) OnStop() {}
// Stop overrides cmn.Service#Stop. There is no other way to wait until Quit
// channel is closed.
func (c *WSClient) Stop() bool {
success := c.BaseService.Stop()
// only close user-facing channels when we can't write to them
c.wg.Wait()
2017-10-24 09:38:12 -07:00
close(c.ResponsesCh)
return success
2016-01-12 13:50:06 -08:00
}
// IsReconnecting returns true if the client is reconnecting right now.
func (c *WSClient) IsReconnecting() bool {
2017-08-07 14:56:38 -07:00
c.mtx.RLock()
defer c.mtx.RUnlock()
return c.reconnecting
}
// IsActive returns true if the client is running and not reconnecting.
func (c *WSClient) IsActive() bool {
return c.IsRunning() && !c.IsReconnecting()
}
2017-08-05 09:07:02 -07:00
// Send the given RPC request to the server. Results will be available on
2017-10-24 09:38:12 -07:00
// ResponsesCh, errors, if any, on ErrorsCh. Will block until send succeeds or
2017-08-05 09:07:02 -07:00
// ctx.Done is closed.
func (c *WSClient) Send(ctx context.Context, request types.RPCRequest) error {
select {
case c.send <- request:
c.Logger.Info("sent a request", "req", request)
return nil
case <-ctx.Done():
return ctx.Err()
}
}
2016-02-18 18:05:24 -08:00
2017-08-05 09:07:02 -07:00
// Call the given method. See Send description.
func (c *WSClient) Call(ctx context.Context, method string, params map[string]interface{}) error {
request, err := types.MapToRequest("ws-client", method, params)
if err != nil {
return err
}
return c.Send(ctx, request)
}
2017-08-05 09:07:02 -07:00
// CallWithArrayParams the given method with params in a form of array. See
// Send description.
func (c *WSClient) CallWithArrayParams(ctx context.Context, method string, params []interface{}) error {
request, err := types.ArrayToRequest("ws-client", method, params)
if err != nil {
return err
}
return c.Send(ctx, request)
}
///////////////////////////////////////////////////////////////////////////////
// Private methods
func (c *WSClient) dial() error {
2016-02-18 18:05:24 -08:00
dialer := &websocket.Dialer{
NetDial: c.Dialer,
2016-02-18 18:05:24 -08:00
Proxy: http.ProxyFromEnvironment,
}
2016-01-12 13:50:06 -08:00
rHeader := http.Header{}
conn, _, err := dialer.Dial("ws://"+c.Address+c.Endpoint, rHeader)
2016-01-12 13:50:06 -08:00
if err != nil {
return err
}
c.conn = conn
2016-01-12 13:50:06 -08:00
return nil
}
// reconnect tries to redial up to maxReconnectAttempts with exponential
// backoff.
func (c *WSClient) reconnect() error {
attempt := 0
2016-01-12 13:50:06 -08:00
2017-08-07 14:56:38 -07:00
c.mtx.Lock()
c.reconnecting = true
2017-08-07 14:56:38 -07:00
c.mtx.Unlock()
defer func() {
2017-08-07 14:56:38 -07:00
c.mtx.Lock()
c.reconnecting = false
2017-08-07 14:56:38 -07:00
c.mtx.Unlock()
}()
// 1s == (1e9 ns) == (1 Billion ns)
billionNs := float64(time.Second.Nanoseconds())
2016-01-12 13:50:06 -08:00
for {
jitterSeconds := time.Duration(rand.Float64() * billionNs)
backoffDuration := jitterSeconds + ((1 << uint(attempt)) * time.Second)
c.Logger.Info("reconnecting", "attempt", attempt+1, "backoff_duration", backoffDuration)
time.Sleep(backoffDuration)
err := c.dial()
2016-01-12 13:50:06 -08:00
if err != nil {
c.Logger.Error("failed to redial", "err", err)
2016-01-12 13:50:06 -08:00
} else {
c.Logger.Info("reconnected")
if c.onReconnect != nil {
go c.onReconnect()
}
return nil
}
attempt++
if attempt > c.maxReconnectAttempts {
return errors.Wrap(err, "reached maximum reconnect attempts")
}
}
}
func (c *WSClient) startReadWriteRoutines() {
c.wg.Add(2)
2017-08-05 09:07:02 -07:00
c.readRoutineQuit = make(chan struct{})
go c.readRoutine()
go c.writeRoutine()
}
func (c *WSClient) processBacklog() error {
select {
case request := <-c.backlog:
if c.writeWait > 0 {
2017-09-06 08:50:43 -07:00
if err := c.conn.SetWriteDeadline(time.Now().Add(c.writeWait)); err != nil {
2017-09-21 06:55:06 -07:00
c.Logger.Error("failed to set write deadline", "err", err)
2017-09-06 08:50:43 -07:00
}
}
2017-09-06 08:50:43 -07:00
if err := c.conn.WriteJSON(request); err != nil {
c.Logger.Error("failed to resend request", "err", err)
c.reconnectAfter <- err
// requeue request
c.backlog <- request
return err
}
c.Logger.Info("resend a request", "req", request)
default:
}
return nil
}
func (c *WSClient) reconnectRoutine() {
for {
select {
case originalError := <-c.reconnectAfter:
2017-08-05 09:07:02 -07:00
// wait until writeRoutine and readRoutine finish
c.wg.Wait()
2017-09-06 08:50:43 -07:00
if err := c.reconnect(); err != nil {
c.Logger.Error("failed to reconnect", "err", err, "original_err", originalError)
c.Stop()
return
} else {
// drain reconnectAfter
LOOP:
for {
select {
case <-c.reconnectAfter:
default:
break LOOP
}
}
err = c.processBacklog()
if err == nil {
c.startReadWriteRoutines()
}
}
case <-c.Quit:
return
}
}
}
// The client ensures that there is at most one writer to a connection by
// executing all writes from this goroutine.
func (c *WSClient) writeRoutine() {
var ticker *time.Ticker
if c.pingPeriod > 0 {
// ticker with a predefined period
ticker = time.NewTicker(c.pingPeriod)
} else {
// ticker that never fires
ticker = &time.Ticker{C: make(<-chan time.Time)}
}
defer func() {
ticker.Stop()
2017-09-06 08:50:43 -07:00
if err := c.conn.Close(); err != nil {
2017-09-21 07:56:42 -07:00
// ignore error; it will trigger in tests
2017-10-03 15:49:20 -07:00
// likely because it's closing an already closed connection
2017-09-06 08:50:43 -07:00
}
c.wg.Done()
}()
for {
select {
case request := <-c.send:
if c.writeWait > 0 {
2017-09-06 08:50:43 -07:00
if err := c.conn.SetWriteDeadline(time.Now().Add(c.writeWait)); err != nil {
2017-09-21 06:55:06 -07:00
c.Logger.Error("failed to set write deadline", "err", err)
2017-09-06 08:50:43 -07:00
}
}
2017-09-06 08:50:43 -07:00
if err := c.conn.WriteJSON(request); err != nil {
c.Logger.Error("failed to send request", "err", err)
c.reconnectAfter <- err
// add request to the backlog, so we don't lose it
c.backlog <- request
return
}
case <-ticker.C:
if c.writeWait > 0 {
2017-09-06 08:50:43 -07:00
if err := c.conn.SetWriteDeadline(time.Now().Add(c.writeWait)); err != nil {
2017-09-21 06:55:06 -07:00
c.Logger.Error("failed to set write deadline", "err", err)
2017-09-06 08:50:43 -07:00
}
}
2017-09-06 08:50:43 -07:00
if err := c.conn.WriteMessage(websocket.PingMessage, []byte{}); err != nil {
c.Logger.Error("failed to write ping", "err", err)
c.reconnectAfter <- err
return
}
2017-08-03 11:19:39 -07:00
c.mtx.Lock()
c.sentLastPingAt = time.Now()
c.mtx.Unlock()
c.Logger.Debug("sent ping")
2017-08-05 09:07:02 -07:00
case <-c.readRoutineQuit:
return
case <-c.Quit:
2017-09-06 08:50:43 -07:00
if err := c.conn.WriteMessage(websocket.CloseMessage, websocket.FormatCloseMessage(websocket.CloseNormalClosure, "")); err != nil {
2017-09-21 06:55:06 -07:00
c.Logger.Error("failed to write message", "err", err)
2017-09-06 08:50:43 -07:00
}
return
2016-01-12 13:50:06 -08:00
}
}
}
// The client ensures that there is at most one reader to a connection by
// executing all reads from this goroutine.
2017-08-05 09:07:02 -07:00
func (c *WSClient) readRoutine() {
defer func() {
2017-09-06 08:50:43 -07:00
if err := c.conn.Close(); err != nil {
2017-09-21 07:56:42 -07:00
// ignore error; it will trigger in tests
2017-10-03 15:49:20 -07:00
// likely because it's closing an already closed connection
2017-09-06 08:50:43 -07:00
}
c.wg.Done()
}()
2016-01-12 13:50:06 -08:00
c.conn.SetPongHandler(func(string) error {
// gather latency stats
2017-08-03 11:19:39 -07:00
c.mtx.RLock()
t := c.sentLastPingAt
2017-08-03 11:19:39 -07:00
c.mtx.RUnlock()
c.PingPongLatencyTimer.UpdateSince(t)
c.Logger.Debug("got pong")
return nil
})
2017-08-05 09:07:02 -07:00
for {
// reset deadline for every message type (control or data)
if c.readWait > 0 {
2017-09-06 08:50:43 -07:00
if err := c.conn.SetReadDeadline(time.Now().Add(c.readWait)); err != nil {
2017-09-21 06:55:06 -07:00
c.Logger.Error("failed to set read deadline", "err", err)
2017-09-06 08:50:43 -07:00
}
}
_, data, err := c.conn.ReadMessage()
if err != nil {
if !websocket.IsUnexpectedCloseError(err, websocket.CloseNormalClosure) {
return
}
c.Logger.Error("failed to read response", "err", err)
2017-08-05 09:07:02 -07:00
close(c.readRoutineQuit)
c.reconnectAfter <- err
return
}
var response types.RPCResponse
err = json.Unmarshal(data, &response)
if err != nil {
c.Logger.Error("failed to parse response", "err", err, "data", string(data))
continue
}
c.Logger.Info("got response", "resp", response.Result)
// Combine a non-blocking read on BaseService.Quit with a non-blocking write on ResponsesCh to avoid blocking
// c.wg.Wait() in c.Stop(). Note we rely on Quit being closed so that it sends unlimited Quit signals to stop
// both readRoutine and writeRoutine
select {
case <-c.Quit:
2017-10-24 09:38:12 -07:00
case c.ResponsesCh <- response:
}
}
2016-01-12 13:50:06 -08:00
}
///////////////////////////////////////////////////////////////////////////////
// Predefined methods
new pubsub package comment out failing consensus tests for now rewrite rpc httpclient to use new pubsub package import pubsub as tmpubsub, query as tmquery make event IDs constants EventKey -> EventTypeKey rename EventsPubsub to PubSub mempool does not use pubsub rename eventsSub to pubsub new subscribe API fix channel size issues and consensus tests bugs refactor rpc client add missing discardFromChan method add mutex rename pubsub to eventBus remove IsRunning from WSRPCConnection interface (not needed) add a comment in broadcastNewRoundStepsAndVotes rename registerEventCallbacks to broadcastNewRoundStepsAndVotes See https://dave.cheney.net/2014/03/19/channel-axioms stop eventBuses after reactor tests remove unnecessary Unsubscribe return subscribe helper function move discardFromChan to where it is used subscribe now returns an err this gives us ability to refuse to subscribe if pubsub is at its max capacity. use context for control overflow cache queries handle err when subscribing in replay_test rename testClientID to testSubscriber extract var set channel buffer capacity to 1 in replay_file fix byzantine_test unsubscribe from single event, not all events refactor httpclient to return events to appropriate channels return failing testReplayCrashBeforeWriteVote test fix TestValidatorSetChanges refactor code a bit fix testReplayCrashBeforeWriteVote add comment fix TestValidatorSetChanges fixes from Bucky's review update comment [ci skip] test TxEventBuffer update changelog fix TestValidatorSetChanges (2nd attempt) only do wg.Done when no errors benchmark event bus create pubsub server inside NewEventBus only expose config params (later if needed) set buffer capacity to 0 so we are not testing cache new tx event format: key = "Tx" plus a tag {"tx.hash": XYZ} This should allow to subscribe to all transactions! or a specific one using a query: "tm.events.type = Tx and tx.hash = '013ABF99434...'" use TimeoutCommit instead of afterPublishEventNewBlockTimeout TimeoutCommit is the time a node waits after committing a block, before it goes into the next height. So it will finish everything from the last block, but then wait a bit. The idea is this gives it time to hear more votes from other validators, to strengthen the commit it includes in the next block. But it also gives it time to hear about new transactions. waitForBlockWithUpdatedVals rewrite WAL crash tests Task: test that we can recover from any WAL crash. Solution: the old tests were relying on event hub being run in the same thread (we were injecting the private validator's last signature). when considering a rewrite, we considered two possible solutions: write a "fuzzy" testing system where WAL is crashing upon receiving a new message, or inject failures and trigger them in tests using something like https://github.com/coreos/gofail. remove sleep no cs.Lock around wal.Save test different cases (empty block, non-empty block, ...) comments add comments test 4 cases: empty block, non-empty block, non-empty block with smaller part size, many blocks fixes as per Bucky's last review reset subscriptions on UnsubscribeAll use a simple counter to track message for which we panicked also, set a smaller part size for all test cases
2017-06-26 08:00:30 -07:00
// Subscribe to a query. Note the server must have a "subscribe" route
// defined.
new pubsub package comment out failing consensus tests for now rewrite rpc httpclient to use new pubsub package import pubsub as tmpubsub, query as tmquery make event IDs constants EventKey -> EventTypeKey rename EventsPubsub to PubSub mempool does not use pubsub rename eventsSub to pubsub new subscribe API fix channel size issues and consensus tests bugs refactor rpc client add missing discardFromChan method add mutex rename pubsub to eventBus remove IsRunning from WSRPCConnection interface (not needed) add a comment in broadcastNewRoundStepsAndVotes rename registerEventCallbacks to broadcastNewRoundStepsAndVotes See https://dave.cheney.net/2014/03/19/channel-axioms stop eventBuses after reactor tests remove unnecessary Unsubscribe return subscribe helper function move discardFromChan to where it is used subscribe now returns an err this gives us ability to refuse to subscribe if pubsub is at its max capacity. use context for control overflow cache queries handle err when subscribing in replay_test rename testClientID to testSubscriber extract var set channel buffer capacity to 1 in replay_file fix byzantine_test unsubscribe from single event, not all events refactor httpclient to return events to appropriate channels return failing testReplayCrashBeforeWriteVote test fix TestValidatorSetChanges refactor code a bit fix testReplayCrashBeforeWriteVote add comment fix TestValidatorSetChanges fixes from Bucky's review update comment [ci skip] test TxEventBuffer update changelog fix TestValidatorSetChanges (2nd attempt) only do wg.Done when no errors benchmark event bus create pubsub server inside NewEventBus only expose config params (later if needed) set buffer capacity to 0 so we are not testing cache new tx event format: key = "Tx" plus a tag {"tx.hash": XYZ} This should allow to subscribe to all transactions! or a specific one using a query: "tm.events.type = Tx and tx.hash = '013ABF99434...'" use TimeoutCommit instead of afterPublishEventNewBlockTimeout TimeoutCommit is the time a node waits after committing a block, before it goes into the next height. So it will finish everything from the last block, but then wait a bit. The idea is this gives it time to hear more votes from other validators, to strengthen the commit it includes in the next block. But it also gives it time to hear about new transactions. waitForBlockWithUpdatedVals rewrite WAL crash tests Task: test that we can recover from any WAL crash. Solution: the old tests were relying on event hub being run in the same thread (we were injecting the private validator's last signature). when considering a rewrite, we considered two possible solutions: write a "fuzzy" testing system where WAL is crashing upon receiving a new message, or inject failures and trigger them in tests using something like https://github.com/coreos/gofail. remove sleep no cs.Lock around wal.Save test different cases (empty block, non-empty block, ...) comments add comments test 4 cases: empty block, non-empty block, non-empty block with smaller part size, many blocks fixes as per Bucky's last review reset subscriptions on UnsubscribeAll use a simple counter to track message for which we panicked also, set a smaller part size for all test cases
2017-06-26 08:00:30 -07:00
func (c *WSClient) Subscribe(ctx context.Context, query string) error {
params := map[string]interface{}{"query": query}
return c.Call(ctx, "subscribe", params)
2016-01-12 13:50:06 -08:00
}
new pubsub package comment out failing consensus tests for now rewrite rpc httpclient to use new pubsub package import pubsub as tmpubsub, query as tmquery make event IDs constants EventKey -> EventTypeKey rename EventsPubsub to PubSub mempool does not use pubsub rename eventsSub to pubsub new subscribe API fix channel size issues and consensus tests bugs refactor rpc client add missing discardFromChan method add mutex rename pubsub to eventBus remove IsRunning from WSRPCConnection interface (not needed) add a comment in broadcastNewRoundStepsAndVotes rename registerEventCallbacks to broadcastNewRoundStepsAndVotes See https://dave.cheney.net/2014/03/19/channel-axioms stop eventBuses after reactor tests remove unnecessary Unsubscribe return subscribe helper function move discardFromChan to where it is used subscribe now returns an err this gives us ability to refuse to subscribe if pubsub is at its max capacity. use context for control overflow cache queries handle err when subscribing in replay_test rename testClientID to testSubscriber extract var set channel buffer capacity to 1 in replay_file fix byzantine_test unsubscribe from single event, not all events refactor httpclient to return events to appropriate channels return failing testReplayCrashBeforeWriteVote test fix TestValidatorSetChanges refactor code a bit fix testReplayCrashBeforeWriteVote add comment fix TestValidatorSetChanges fixes from Bucky's review update comment [ci skip] test TxEventBuffer update changelog fix TestValidatorSetChanges (2nd attempt) only do wg.Done when no errors benchmark event bus create pubsub server inside NewEventBus only expose config params (later if needed) set buffer capacity to 0 so we are not testing cache new tx event format: key = "Tx" plus a tag {"tx.hash": XYZ} This should allow to subscribe to all transactions! or a specific one using a query: "tm.events.type = Tx and tx.hash = '013ABF99434...'" use TimeoutCommit instead of afterPublishEventNewBlockTimeout TimeoutCommit is the time a node waits after committing a block, before it goes into the next height. So it will finish everything from the last block, but then wait a bit. The idea is this gives it time to hear more votes from other validators, to strengthen the commit it includes in the next block. But it also gives it time to hear about new transactions. waitForBlockWithUpdatedVals rewrite WAL crash tests Task: test that we can recover from any WAL crash. Solution: the old tests were relying on event hub being run in the same thread (we were injecting the private validator's last signature). when considering a rewrite, we considered two possible solutions: write a "fuzzy" testing system where WAL is crashing upon receiving a new message, or inject failures and trigger them in tests using something like https://github.com/coreos/gofail. remove sleep no cs.Lock around wal.Save test different cases (empty block, non-empty block, ...) comments add comments test 4 cases: empty block, non-empty block, non-empty block with smaller part size, many blocks fixes as per Bucky's last review reset subscriptions on UnsubscribeAll use a simple counter to track message for which we panicked also, set a smaller part size for all test cases
2017-06-26 08:00:30 -07:00
// Unsubscribe from a query. Note the server must have a "unsubscribe" route
// defined.
new pubsub package comment out failing consensus tests for now rewrite rpc httpclient to use new pubsub package import pubsub as tmpubsub, query as tmquery make event IDs constants EventKey -> EventTypeKey rename EventsPubsub to PubSub mempool does not use pubsub rename eventsSub to pubsub new subscribe API fix channel size issues and consensus tests bugs refactor rpc client add missing discardFromChan method add mutex rename pubsub to eventBus remove IsRunning from WSRPCConnection interface (not needed) add a comment in broadcastNewRoundStepsAndVotes rename registerEventCallbacks to broadcastNewRoundStepsAndVotes See https://dave.cheney.net/2014/03/19/channel-axioms stop eventBuses after reactor tests remove unnecessary Unsubscribe return subscribe helper function move discardFromChan to where it is used subscribe now returns an err this gives us ability to refuse to subscribe if pubsub is at its max capacity. use context for control overflow cache queries handle err when subscribing in replay_test rename testClientID to testSubscriber extract var set channel buffer capacity to 1 in replay_file fix byzantine_test unsubscribe from single event, not all events refactor httpclient to return events to appropriate channels return failing testReplayCrashBeforeWriteVote test fix TestValidatorSetChanges refactor code a bit fix testReplayCrashBeforeWriteVote add comment fix TestValidatorSetChanges fixes from Bucky's review update comment [ci skip] test TxEventBuffer update changelog fix TestValidatorSetChanges (2nd attempt) only do wg.Done when no errors benchmark event bus create pubsub server inside NewEventBus only expose config params (later if needed) set buffer capacity to 0 so we are not testing cache new tx event format: key = "Tx" plus a tag {"tx.hash": XYZ} This should allow to subscribe to all transactions! or a specific one using a query: "tm.events.type = Tx and tx.hash = '013ABF99434...'" use TimeoutCommit instead of afterPublishEventNewBlockTimeout TimeoutCommit is the time a node waits after committing a block, before it goes into the next height. So it will finish everything from the last block, but then wait a bit. The idea is this gives it time to hear more votes from other validators, to strengthen the commit it includes in the next block. But it also gives it time to hear about new transactions. waitForBlockWithUpdatedVals rewrite WAL crash tests Task: test that we can recover from any WAL crash. Solution: the old tests were relying on event hub being run in the same thread (we were injecting the private validator's last signature). when considering a rewrite, we considered two possible solutions: write a "fuzzy" testing system where WAL is crashing upon receiving a new message, or inject failures and trigger them in tests using something like https://github.com/coreos/gofail. remove sleep no cs.Lock around wal.Save test different cases (empty block, non-empty block, ...) comments add comments test 4 cases: empty block, non-empty block, non-empty block with smaller part size, many blocks fixes as per Bucky's last review reset subscriptions on UnsubscribeAll use a simple counter to track message for which we panicked also, set a smaller part size for all test cases
2017-06-26 08:00:30 -07:00
func (c *WSClient) Unsubscribe(ctx context.Context, query string) error {
params := map[string]interface{}{"query": query}
return c.Call(ctx, "unsubscribe", params)
2016-01-12 13:50:06 -08:00
}
// UnsubscribeAll from all. Note the server must have a "unsubscribe_all" route
// defined.
func (c *WSClient) UnsubscribeAll(ctx context.Context) error {
params := map[string]interface{}{}
return c.Call(ctx, "unsubscribe_all", params)
}