Browse Source

light: correctly handle contexts (#6687)

pull/6696/head
Callum Waters 3 years ago
committed by GitHub
parent
commit
051e127d38
No known key found for this signature in database GPG Key ID: 4AEE18F83AFDEB23
7 changed files with 77 additions and 25 deletions
  1. +1
    -0
      CHANGELOG_PENDING.md
  2. +4
    -0
      internal/statesync/dispatcher.go
  3. +0
    -1
      internal/statesync/reactor.go
  4. +19
    -1
      internal/statesync/syncer.go
  5. +15
    -1
      light/client.go
  6. +20
    -2
      light/provider/http/http.go
  7. +18
    -20
      light/provider/http/http_test.go

+ 1
- 0
CHANGELOG_PENDING.md View File

@ -149,3 +149,4 @@ Friendly reminder: We have a [bug bounty program](https://hackerone.com/tendermi
- [statesync] \#6463 Adds Reverse Sync feature to fetch historical light blocks after state sync in order to verify any evidence (@cmwaters)
- [fastsync] \#6590 Update the metrics during fast-sync (@JayT106)
- [gitignore] \#6668 Fix gitignore of abci-cli (@tanyabouman)
- [light] \#6687 Fix bug with incorrecly handled contexts in the light client (@cmwaters)

+ 4
- 0
internal/statesync/dispatcher.go View File

@ -45,6 +45,8 @@ func newDispatcher(requestCh chan<- p2p.Envelope, timeout time.Duration) *dispat
}
}
// LightBlock uses the request channel to fetch a light block from the next peer
// in a list, tracks the call and waits for the reactor to pass along the response
func (d *dispatcher) LightBlock(ctx context.Context, height int64) (*types.LightBlock, types.NodeID, error) {
d.mtx.Lock()
outgoingCalls := len(d.calls)
@ -62,6 +64,8 @@ func (d *dispatcher) LightBlock(ctx context.Context, height int64) (*types.Light
return lb, peer, err
}
// Providers turns the dispatcher into a set of providers (per peer) which can
// be used by a light client
func (d *dispatcher) Providers(chainID string, timeout time.Duration) []provider.Provider {
d.mtx.Lock()
defer d.mtx.Unlock()


+ 0
- 1
internal/statesync/reactor.go View File

@ -628,7 +628,6 @@ func (r *Reactor) handleLightBlockMessage(envelope p2p.Envelope) error {
case *ssproto.LightBlockResponse:
if err := r.dispatcher.respond(msg.LightBlock, envelope.From); err != nil {
r.Logger.Error("error processing light block response", "err", err)
return err
}
default:


+ 19
- 1
internal/statesync/syncer.go View File

@ -38,12 +38,16 @@ var (
errRejectFormat = errors.New("snapshot format was rejected")
// errRejectSender is returned by Sync() when the snapshot sender is rejected.
errRejectSender = errors.New("snapshot sender was rejected")
// errVerifyFailed is returned by Sync() when app hash or last height verification fails.
// errVerifyFailed is returned by Sync() when app hash or last height
// verification fails.
errVerifyFailed = errors.New("verification failed")
// errTimeout is returned by Sync() when we've waited too long to receive a chunk.
errTimeout = errors.New("timed out waiting for chunk")
// errNoSnapshots is returned by SyncAny() if no snapshots are found and discovery is disabled.
errNoSnapshots = errors.New("no suitable snapshots found")
// errStateCommitTimeout is returned by Sync() when the timeout for retrieving
// tendermint state or the commit is exceeded
errStateCommitTimeout = errors.New("timed out trying to retrieve state and commit")
)
// syncer runs a state sync against an ABCI app. Use either SyncAny() to automatically attempt to
@ -226,6 +230,10 @@ func (s *syncer) SyncAny(
s.logger.Info("Snapshot sender rejected", "peer", peer)
}
case errors.Is(err, errStateCommitTimeout):
s.logger.Info("Timed out retrieving state and commit, rejecting and retrying...", "height", snapshot.Height)
s.snapshots.Reject(snapshot)
default:
return sm.State{}, nil, fmt.Errorf("snapshot restoration failed: %w", err)
}
@ -275,10 +283,20 @@ func (s *syncer) Sync(ctx context.Context, snapshot *snapshot, chunks *chunkQueu
// Optimistically build new state, so we don't discover any light client failures at the end.
state, err := s.stateProvider.State(pctx, snapshot.Height)
if err != nil {
// check if the provider context exceeded the 10 second deadline
if err == context.DeadlineExceeded && ctx.Err() == nil {
return sm.State{}, nil, errStateCommitTimeout
}
return sm.State{}, nil, fmt.Errorf("failed to build new state: %w", err)
}
commit, err := s.stateProvider.Commit(pctx, snapshot.Height)
if err != nil {
// check if the provider context exceeded the 10 second deadline
if err == context.DeadlineExceeded && ctx.Err() == nil {
return sm.State{}, nil, errStateCommitTimeout
}
return sm.State{}, nil, fmt.Errorf("failed to fetch commit: %w", err)
}


+ 15
- 1
light/client.go View File

@ -909,6 +909,10 @@ func (c *Client) lightBlockFromPrimary(ctx context.Context, height int64) (*type
// Everything went smoothly. We reset the lightBlockRequests and return the light block
return l, nil
// catch canceled contexts or deadlines
case context.Canceled, context.DeadlineExceeded:
return nil, err
case provider.ErrNoResponse, provider.ErrLightBlockNotFound, provider.ErrHeightTooHigh:
// we find a new witness to replace the primary
c.logger.Debug("error from light block request from primary, replacing...",
@ -1011,6 +1015,10 @@ func (c *Client) findNewPrimary(ctx context.Context, height int64, remove bool)
// return the light block that new primary responded with
return response.lb, nil
// catch canceled contexts or deadlines
case context.Canceled, context.DeadlineExceeded:
return nil, response.err
// process benign errors by logging them only
case provider.ErrNoResponse, provider.ErrLightBlockNotFound, provider.ErrHeightTooHigh:
lastError = response.err
@ -1067,7 +1075,13 @@ and remove witness. Otherwise, use a different primary`, e.WitnessIndex), "witne
"witness", c.witnesses[e.WitnessIndex],
"err", err)
witnessesToRemove = append(witnessesToRemove, e.WitnessIndex)
default: // the witness either didn't respond or didn't have the block. We ignore it.
default:
// check for canceled contexts or deadlines
if errors.Is(err, context.Canceled) || errors.Is(err, context.DeadlineExceeded) {
return err
}
// the witness either didn't respond or didn't have the block. We ignore it.
c.logger.Debug("unable to compare first header with witness",
"err", err)
}


+ 20
- 2
light/provider/http/http.go View File

@ -19,7 +19,7 @@ import (
var defaultOptions = Options{
MaxRetryAttempts: 5,
Timeout: 3 * time.Second,
Timeout: 5 * time.Second,
NoBlockThreshold: 5,
NoResponseThreshold: 5,
}
@ -125,7 +125,7 @@ func (p *http) LightBlock(ctx context.Context, height int64) (*types.LightBlock,
if sh.Header == nil {
return nil, provider.ErrBadLightBlock{
Reason: errors.New("header is nil unexpectedly"),
Reason: errors.New("returned header is nil unexpectedly"),
}
}
@ -205,6 +205,11 @@ func (p *http) validatorSet(ctx context.Context, height *int64) (*types.Validato
return nil, p.parseRPCError(e)
default:
// check if the error stems from the context
if errors.Is(err, context.Canceled) || errors.Is(err, context.DeadlineExceeded) {
return nil, err
}
// If we don't know the error then by default we return an unreliable provider error and
// terminate the connection with the peer.
return nil, provider.ErrUnreliableProvider{Reason: e.Error()}
@ -236,11 +241,19 @@ func (p *http) signedHeader(ctx context.Context, height *int64) (*types.SignedHe
return &commit.SignedHeader, nil
case *url.Error:
// check if the request timed out
if e.Timeout() {
// we wait and try again with exponential backoff
time.Sleep(backoffTimeout(attempt))
continue
}
// check if the connection was refused or dropped
if strings.Contains(e.Error(), "connection refused") {
return nil, provider.ErrConnectionClosed
}
// else, as a catch all, we return the error as a bad light block response
return nil, provider.ErrBadLightBlock{Reason: e}
case *rpctypes.RPCError:
@ -248,6 +261,11 @@ func (p *http) signedHeader(ctx context.Context, height *int64) (*types.SignedHe
return nil, p.parseRPCError(e)
default:
// check if the error stems from the context
if errors.Is(err, context.Canceled) || errors.Is(err, context.DeadlineExceeded) {
return nil, err
}
// If we don't know the error then by default we return an unreliable provider error and
// terminate the connection with the peer.
return nil, provider.ErrUnreliableProvider{Reason: e.Error()}


+ 18
- 20
light/provider/http/http_test.go View File

@ -4,13 +4,12 @@ import (
"context"
"fmt"
"testing"
"time"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"github.com/tendermint/tendermint/abci/example/kvstore"
"github.com/tendermint/tendermint/config"
"github.com/tendermint/tendermint/libs/service"
"github.com/tendermint/tendermint/light/provider"
lighthttp "github.com/tendermint/tendermint/light/provider/http"
rpcclient "github.com/tendermint/tendermint/rpc/client"
@ -33,30 +32,17 @@ func TestNewProvider(t *testing.T) {
require.Equal(t, fmt.Sprintf("%s", c), "http{http://153.200.0.1}")
}
// NodeSuite initiates and runs a full node instance in the
// background, stopping it once the test is completed
func NodeSuite(t *testing.T) (service.Service, *config.Config) {
t.Helper()
func TestProvider(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
conf := rpctest.CreateConfig(t.Name())
defer cancel()
cfg := rpctest.CreateConfig(t.Name())
// start a tendermint node in the background to test against
app := kvstore.NewApplication()
app.RetainBlocks = 9
node, closer, err := rpctest.StartTendermint(ctx, conf, app)
_, closer, err := rpctest.StartTendermint(ctx, cfg, app)
require.NoError(t, err)
t.Cleanup(func() {
_ = closer(ctx)
cancel()
})
return node, conf
}
func TestProvider(t *testing.T) {
_, cfg := NodeSuite(t)
rpcAddr := cfg.RPC.ListenAddress
genDoc, err := types.GenesisDocFromFile(cfg.GenesisFile())
require.NoError(t, err)
@ -95,8 +81,9 @@ func TestProvider(t *testing.T) {
require.Nil(t, lb)
assert.Equal(t, provider.ErrHeightTooHigh, err)
_, err = p.LightBlock(context.Background(), 1)
lb, err = p.LightBlock(context.Background(), 1)
require.Error(t, err)
require.Nil(t, lb)
assert.Equal(t, provider.ErrLightBlockNotFound, err)
// if the provider is unable to provide four more blocks then we should return
@ -105,4 +92,15 @@ func TestProvider(t *testing.T) {
_, err = p.LightBlock(context.Background(), 1)
}
assert.IsType(t, provider.ErrUnreliableProvider{}, err)
// shut down tendermint node
require.NoError(t, closer(ctx))
cancel()
time.Sleep(10 * time.Second)
lb, err = p.LightBlock(context.Background(), lower+2)
// we should see a connection refused
require.Error(t, err)
require.Nil(t, lb)
assert.Equal(t, provider.ErrConnectionClosed, err)
}

Loading…
Cancel
Save