mirror of
https://github.com/tendermint/tendermint.git
synced 2026-01-03 11:45:18 +00:00
add state sync reactor (#4705)
Fixes #828. Adds state sync, as outlined in [ADR-053](https://github.com/tendermint/tendermint/blob/master/docs/architecture/adr-053-state-sync-prototype.md). See related PRs in Cosmos SDK (https://github.com/cosmos/cosmos-sdk/pull/5803) and Gaia (https://github.com/cosmos/gaia/pull/327). This is split out of the previous PR #4645, and branched off of the ABCI interface in #4704. * Adds a new P2P reactor which exchanges snapshots with peers, and bootstraps an empty local node from remote snapshots when requested. * Adds a new configuration section `[statesync]` that enables state sync and configures the light client. Also enables `statesync:info` logging by default. * Integrates state sync into node startup. Does not support the v2 blockchain reactor, since it needs some reorganization to defer startup.
This commit is contained in:
321
statesync/chunks.go
Normal file
321
statesync/chunks.go
Normal file
@@ -0,0 +1,321 @@
|
||||
package statesync
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"fmt"
|
||||
"io/ioutil"
|
||||
"os"
|
||||
"path/filepath"
|
||||
"strconv"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/tendermint/tendermint/p2p"
|
||||
)
|
||||
|
||||
// errDone is returned by chunkQueue.Next() when all chunks have been returned.
|
||||
var errDone = errors.New("chunk queue has completed")
|
||||
|
||||
// chunk contains data for a chunk.
|
||||
type chunk struct {
|
||||
Height uint64
|
||||
Format uint32
|
||||
Index uint32
|
||||
Chunk []byte
|
||||
Sender p2p.ID
|
||||
}
|
||||
|
||||
// chunkQueue manages chunks for a state sync process, ordering them if requested. It acts as an
|
||||
// iterator over all chunks, but callers can request chunks to be retried, optionally after
|
||||
// refetching.
|
||||
type chunkQueue struct {
|
||||
sync.Mutex
|
||||
snapshot *snapshot // if this is nil, the queue has been closed
|
||||
dir string // temp dir for on-disk chunk storage
|
||||
chunkFiles map[uint32]string // path to temporary chunk file
|
||||
chunkSenders map[uint32]p2p.ID // the peer who sent the given chunk
|
||||
chunkAllocated map[uint32]bool // chunks that have been allocated via Allocate()
|
||||
chunkReturned map[uint32]bool // chunks returned via Next()
|
||||
waiters map[uint32][]chan<- uint32 // signals WaitFor() waiters about chunk arrival
|
||||
}
|
||||
|
||||
// newChunkQueue creates a new chunk queue for a snapshot, using a temp dir for storage.
|
||||
// Callers must call Close() when done.
|
||||
func newChunkQueue(snapshot *snapshot, tempDir string) (*chunkQueue, error) {
|
||||
dir, err := ioutil.TempDir(tempDir, "tm-statesync")
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("unable to create temp dir for state sync chunks: %w", err)
|
||||
}
|
||||
if snapshot.Chunks == 0 {
|
||||
return nil, errors.New("snapshot has no chunks")
|
||||
}
|
||||
return &chunkQueue{
|
||||
snapshot: snapshot,
|
||||
dir: dir,
|
||||
chunkFiles: make(map[uint32]string, snapshot.Chunks),
|
||||
chunkSenders: make(map[uint32]p2p.ID, snapshot.Chunks),
|
||||
chunkAllocated: make(map[uint32]bool, snapshot.Chunks),
|
||||
chunkReturned: make(map[uint32]bool, snapshot.Chunks),
|
||||
waiters: make(map[uint32][]chan<- uint32),
|
||||
}, nil
|
||||
}
|
||||
|
||||
// Add adds a chunk to the queue. It ignores chunks that already exist, returning false.
|
||||
func (q *chunkQueue) Add(chunk *chunk) (bool, error) {
|
||||
if chunk == nil || chunk.Chunk == nil {
|
||||
return false, errors.New("cannot add nil chunk")
|
||||
}
|
||||
q.Lock()
|
||||
defer q.Unlock()
|
||||
if q.snapshot == nil {
|
||||
return false, nil // queue is closed
|
||||
}
|
||||
if chunk.Height != q.snapshot.Height {
|
||||
return false, fmt.Errorf("invalid chunk height %v, expected %v", chunk.Height, q.snapshot.Height)
|
||||
}
|
||||
if chunk.Format != q.snapshot.Format {
|
||||
return false, fmt.Errorf("invalid chunk format %v, expected %v", chunk.Format, q.snapshot.Format)
|
||||
}
|
||||
if chunk.Index >= q.snapshot.Chunks {
|
||||
return false, fmt.Errorf("received unexpected chunk %v", chunk.Index)
|
||||
}
|
||||
if q.chunkFiles[chunk.Index] != "" {
|
||||
return false, nil
|
||||
}
|
||||
|
||||
path := filepath.Join(q.dir, strconv.FormatUint(uint64(chunk.Index), 10))
|
||||
err := ioutil.WriteFile(path, chunk.Chunk, 0644)
|
||||
if err != nil {
|
||||
return false, fmt.Errorf("failed to save chunk %v to file %v: %w", chunk.Index, path, err)
|
||||
}
|
||||
q.chunkFiles[chunk.Index] = path
|
||||
q.chunkSenders[chunk.Index] = chunk.Sender
|
||||
|
||||
// Signal any waiters that the chunk has arrived.
|
||||
for _, waiter := range q.waiters[chunk.Index] {
|
||||
waiter <- chunk.Index
|
||||
close(waiter)
|
||||
}
|
||||
delete(q.waiters, chunk.Index)
|
||||
|
||||
return true, nil
|
||||
}
|
||||
|
||||
// Allocate allocates a chunk to the caller, making it responsible for fetching it. Returns
|
||||
// errDone once no chunks are left or the queue is closed.
|
||||
func (q *chunkQueue) Allocate() (uint32, error) {
|
||||
q.Lock()
|
||||
defer q.Unlock()
|
||||
if q.snapshot == nil {
|
||||
return 0, errDone
|
||||
}
|
||||
if uint32(len(q.chunkAllocated)) >= q.snapshot.Chunks {
|
||||
return 0, errDone
|
||||
}
|
||||
for i := uint32(0); i < q.snapshot.Chunks; i++ {
|
||||
if !q.chunkAllocated[i] {
|
||||
q.chunkAllocated[i] = true
|
||||
return i, nil
|
||||
}
|
||||
}
|
||||
return 0, errDone
|
||||
}
|
||||
|
||||
// Close closes the chunk queue, cleaning up all temporary files.
|
||||
func (q *chunkQueue) Close() error {
|
||||
q.Lock()
|
||||
defer q.Unlock()
|
||||
if q.snapshot == nil {
|
||||
return nil
|
||||
}
|
||||
for _, waiters := range q.waiters {
|
||||
for _, waiter := range waiters {
|
||||
close(waiter)
|
||||
}
|
||||
}
|
||||
q.waiters = nil
|
||||
q.snapshot = nil
|
||||
err := os.RemoveAll(q.dir)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to clean up state sync tempdir %v: %w", q.dir, err)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// Discard discards a chunk. It will be removed from the queue, available for allocation, and can
|
||||
// be added and returned via Next() again. If the chunk is not already in the queue this does
|
||||
// nothing, to avoid it being allocated to multiple fetchers.
|
||||
func (q *chunkQueue) Discard(index uint32) error {
|
||||
q.Lock()
|
||||
defer q.Unlock()
|
||||
return q.discard(index)
|
||||
}
|
||||
|
||||
// discard discards a chunk, scheduling it for refetching. The caller must hold the mutex lock.
|
||||
func (q *chunkQueue) discard(index uint32) error {
|
||||
if q.snapshot == nil {
|
||||
return nil
|
||||
}
|
||||
path := q.chunkFiles[index]
|
||||
if path == "" {
|
||||
return nil
|
||||
}
|
||||
err := os.Remove(path)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to remove chunk %v: %w", index, err)
|
||||
}
|
||||
delete(q.chunkFiles, index)
|
||||
delete(q.chunkReturned, index)
|
||||
delete(q.chunkAllocated, index)
|
||||
return nil
|
||||
}
|
||||
|
||||
// DiscardSender discards all *unreturned* chunks from a given sender. If the caller wants to
|
||||
// discard already returned chunks, this can be done via Discard().
|
||||
func (q *chunkQueue) DiscardSender(peerID p2p.ID) error {
|
||||
q.Lock()
|
||||
defer q.Unlock()
|
||||
|
||||
for index, sender := range q.chunkSenders {
|
||||
if sender == peerID && !q.chunkReturned[index] {
|
||||
err := q.discard(index)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
delete(q.chunkSenders, index)
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// GetSender returns the sender of the chunk with the given index, or empty if not found.
|
||||
func (q *chunkQueue) GetSender(index uint32) p2p.ID {
|
||||
q.Lock()
|
||||
defer q.Unlock()
|
||||
return q.chunkSenders[index]
|
||||
}
|
||||
|
||||
// Has checks whether a chunk exists in the queue.
|
||||
func (q *chunkQueue) Has(index uint32) bool {
|
||||
q.Lock()
|
||||
defer q.Unlock()
|
||||
return q.chunkFiles[index] != ""
|
||||
}
|
||||
|
||||
// load loads a chunk from disk, or nil if the chunk is not in the queue. The caller must hold the
|
||||
// mutex lock.
|
||||
func (q *chunkQueue) load(index uint32) (*chunk, error) {
|
||||
path, ok := q.chunkFiles[index]
|
||||
if !ok {
|
||||
return nil, nil
|
||||
}
|
||||
body, err := ioutil.ReadFile(path)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("failed to load chunk %v: %w", index, err)
|
||||
}
|
||||
return &chunk{
|
||||
Height: q.snapshot.Height,
|
||||
Format: q.snapshot.Format,
|
||||
Index: index,
|
||||
Chunk: body,
|
||||
Sender: q.chunkSenders[index],
|
||||
}, nil
|
||||
}
|
||||
|
||||
// Next returns the next chunk from the queue, or errDone if all chunks have been returned. It
|
||||
// blocks until the chunk is available. Concurrent Next() calls may return the same chunk.
|
||||
func (q *chunkQueue) Next() (*chunk, error) {
|
||||
q.Lock()
|
||||
var chunk *chunk
|
||||
index, err := q.nextUp()
|
||||
if err == nil {
|
||||
chunk, err = q.load(index)
|
||||
if err == nil {
|
||||
q.chunkReturned[index] = true
|
||||
}
|
||||
}
|
||||
q.Unlock()
|
||||
if chunk != nil || err != nil {
|
||||
return chunk, err
|
||||
}
|
||||
|
||||
select {
|
||||
case _, ok := <-q.WaitFor(index):
|
||||
if !ok {
|
||||
return nil, errDone // queue closed
|
||||
}
|
||||
case <-time.After(chunkTimeout):
|
||||
return nil, errTimeout
|
||||
}
|
||||
|
||||
q.Lock()
|
||||
defer q.Unlock()
|
||||
chunk, err = q.load(index)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
q.chunkReturned[index] = true
|
||||
return chunk, nil
|
||||
}
|
||||
|
||||
// nextUp returns the next chunk to be returned, or errDone if all chunks have been returned. The
|
||||
// caller must hold the mutex lock.
|
||||
func (q *chunkQueue) nextUp() (uint32, error) {
|
||||
if q.snapshot == nil {
|
||||
return 0, errDone
|
||||
}
|
||||
for i := uint32(0); i < q.snapshot.Chunks; i++ {
|
||||
if !q.chunkReturned[i] {
|
||||
return i, nil
|
||||
}
|
||||
}
|
||||
return 0, errDone
|
||||
}
|
||||
|
||||
// Retry schedules a chunk to be retried, without refetching it.
|
||||
func (q *chunkQueue) Retry(index uint32) {
|
||||
q.Lock()
|
||||
defer q.Unlock()
|
||||
delete(q.chunkReturned, index)
|
||||
}
|
||||
|
||||
// RetryAll schedules all chunks to be retried, without refetching them.
|
||||
func (q *chunkQueue) RetryAll() {
|
||||
q.Lock()
|
||||
defer q.Unlock()
|
||||
q.chunkReturned = make(map[uint32]bool)
|
||||
}
|
||||
|
||||
// Size returns the total number of chunks for the snapshot and queue, or 0 when closed.
|
||||
func (q *chunkQueue) Size() uint32 {
|
||||
q.Lock()
|
||||
defer q.Unlock()
|
||||
if q.snapshot == nil {
|
||||
return 0
|
||||
}
|
||||
return q.snapshot.Chunks
|
||||
}
|
||||
|
||||
// WaitFor returns a channel that receives a chunk index when it arrives in the queue, or
|
||||
// immediately if it has already arrived. The channel is closed without a value if the queue is
|
||||
// closed or if the chunk index is not valid.
|
||||
func (q *chunkQueue) WaitFor(index uint32) <-chan uint32 {
|
||||
q.Lock()
|
||||
defer q.Unlock()
|
||||
ch := make(chan uint32, 1)
|
||||
switch {
|
||||
case q.snapshot == nil:
|
||||
close(ch)
|
||||
case index >= q.snapshot.Chunks:
|
||||
close(ch)
|
||||
case q.chunkFiles[index] != "":
|
||||
ch <- index
|
||||
close(ch)
|
||||
default:
|
||||
if q.waiters[index] == nil {
|
||||
q.waiters[index] = make([]chan<- uint32, 0)
|
||||
}
|
||||
q.waiters[index] = append(q.waiters[index], ch)
|
||||
}
|
||||
return ch
|
||||
}
|
||||
551
statesync/chunks_test.go
Normal file
551
statesync/chunks_test.go
Normal file
@@ -0,0 +1,551 @@
|
||||
package statesync
|
||||
|
||||
import (
|
||||
"io/ioutil"
|
||||
"os"
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/require"
|
||||
|
||||
"github.com/tendermint/tendermint/p2p"
|
||||
)
|
||||
|
||||
func setupChunkQueue(t *testing.T) (*chunkQueue, func()) {
|
||||
snapshot := &snapshot{
|
||||
Height: 3,
|
||||
Format: 1,
|
||||
Chunks: 5,
|
||||
Hash: []byte{7},
|
||||
Metadata: nil,
|
||||
}
|
||||
queue, err := newChunkQueue(snapshot, "")
|
||||
require.NoError(t, err)
|
||||
teardown := func() {
|
||||
err := queue.Close()
|
||||
require.NoError(t, err)
|
||||
}
|
||||
return queue, teardown
|
||||
}
|
||||
|
||||
func TestNewChunkQueue_TempDir(t *testing.T) {
|
||||
snapshot := &snapshot{
|
||||
Height: 3,
|
||||
Format: 1,
|
||||
Chunks: 5,
|
||||
Hash: []byte{7},
|
||||
Metadata: nil,
|
||||
}
|
||||
dir, err := ioutil.TempDir("", "newchunkqueue")
|
||||
require.NoError(t, err)
|
||||
defer os.RemoveAll(dir)
|
||||
queue, err := newChunkQueue(snapshot, dir)
|
||||
require.NoError(t, err)
|
||||
|
||||
files, err := ioutil.ReadDir(dir)
|
||||
require.NoError(t, err)
|
||||
assert.Len(t, files, 1)
|
||||
|
||||
err = queue.Close()
|
||||
require.NoError(t, err)
|
||||
|
||||
files, err = ioutil.ReadDir(dir)
|
||||
require.NoError(t, err)
|
||||
assert.Len(t, files, 0)
|
||||
}
|
||||
|
||||
func TestChunkQueue(t *testing.T) {
|
||||
queue, teardown := setupChunkQueue(t)
|
||||
defer teardown()
|
||||
|
||||
// Adding the first chunk should be fine
|
||||
added, err := queue.Add(&chunk{Height: 3, Format: 1, Index: 0, Chunk: []byte{3, 1, 0}})
|
||||
require.NoError(t, err)
|
||||
assert.True(t, added)
|
||||
|
||||
// Adding the last chunk should also be fine
|
||||
added, err = queue.Add(&chunk{Height: 3, Format: 1, Index: 4, Chunk: []byte{3, 1, 4}})
|
||||
require.NoError(t, err)
|
||||
assert.True(t, added)
|
||||
|
||||
// Adding the first or last chunks again should return false
|
||||
added, err = queue.Add(&chunk{Height: 3, Format: 1, Index: 0, Chunk: []byte{3, 1, 0}})
|
||||
require.NoError(t, err)
|
||||
assert.False(t, added)
|
||||
|
||||
added, err = queue.Add(&chunk{Height: 3, Format: 1, Index: 4, Chunk: []byte{3, 1, 4}})
|
||||
require.NoError(t, err)
|
||||
assert.False(t, added)
|
||||
|
||||
// Adding the remaining chunks in reverse should be fine
|
||||
added, err = queue.Add(&chunk{Height: 3, Format: 1, Index: 3, Chunk: []byte{3, 1, 3}})
|
||||
require.NoError(t, err)
|
||||
assert.True(t, added)
|
||||
|
||||
added, err = queue.Add(&chunk{Height: 3, Format: 1, Index: 2, Chunk: []byte{3, 1, 2}})
|
||||
require.NoError(t, err)
|
||||
assert.True(t, added)
|
||||
|
||||
added, err = queue.Add(&chunk{Height: 3, Format: 1, Index: 1, Chunk: []byte{3, 1, 1}})
|
||||
require.NoError(t, err)
|
||||
assert.True(t, added)
|
||||
|
||||
// At this point, we should be able to retrieve them all via Next
|
||||
for i := 0; i < 5; i++ {
|
||||
c, err := queue.Next()
|
||||
require.NoError(t, err)
|
||||
assert.Equal(t, &chunk{Height: 3, Format: 1, Index: uint32(i), Chunk: []byte{3, 1, byte(i)}}, c)
|
||||
}
|
||||
_, err = queue.Next()
|
||||
require.Error(t, err)
|
||||
assert.Equal(t, errDone, err)
|
||||
|
||||
// It should still be possible to try to add chunks (which will be ignored)
|
||||
added, err = queue.Add(&chunk{Height: 3, Format: 1, Index: 0, Chunk: []byte{3, 1, 0}})
|
||||
require.NoError(t, err)
|
||||
assert.False(t, added)
|
||||
|
||||
// After closing the queue it will also return false
|
||||
err = queue.Close()
|
||||
require.NoError(t, err)
|
||||
added, err = queue.Add(&chunk{Height: 3, Format: 1, Index: 0, Chunk: []byte{3, 1, 0}})
|
||||
require.NoError(t, err)
|
||||
assert.False(t, added)
|
||||
|
||||
// Closing the queue again should also be fine
|
||||
err = queue.Close()
|
||||
require.NoError(t, err)
|
||||
}
|
||||
|
||||
func TestChunkQueue_Add_ChunkErrors(t *testing.T) {
|
||||
testcases := map[string]struct {
|
||||
chunk *chunk
|
||||
}{
|
||||
"nil chunk": {nil},
|
||||
"nil body": {&chunk{Height: 3, Format: 1, Index: 0, Chunk: nil}},
|
||||
"wrong height": {&chunk{Height: 9, Format: 1, Index: 0, Chunk: []byte{3, 1, 0}}},
|
||||
"wrong format": {&chunk{Height: 3, Format: 9, Index: 0, Chunk: []byte{3, 1, 0}}},
|
||||
"invalid index": {&chunk{Height: 3, Format: 1, Index: 5, Chunk: []byte{3, 1, 0}}},
|
||||
}
|
||||
for name, tc := range testcases {
|
||||
tc := tc
|
||||
t.Run(name, func(t *testing.T) {
|
||||
queue, teardown := setupChunkQueue(t)
|
||||
defer teardown()
|
||||
_, err := queue.Add(tc.chunk)
|
||||
require.Error(t, err)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestChunkQueue_Allocate(t *testing.T) {
|
||||
queue, teardown := setupChunkQueue(t)
|
||||
defer teardown()
|
||||
|
||||
for i := uint32(0); i < queue.Size(); i++ {
|
||||
index, err := queue.Allocate()
|
||||
require.NoError(t, err)
|
||||
assert.EqualValues(t, i, index)
|
||||
}
|
||||
|
||||
_, err := queue.Allocate()
|
||||
require.Error(t, err)
|
||||
assert.Equal(t, errDone, err)
|
||||
|
||||
for i := uint32(0); i < queue.Size(); i++ {
|
||||
_, err = queue.Add(&chunk{Height: 3, Format: 1, Index: i, Chunk: []byte{byte(i)}})
|
||||
require.NoError(t, err)
|
||||
}
|
||||
|
||||
// After all chunks have been allocated and retrieved, discarding a chunk will reallocate it.
|
||||
err = queue.Discard(2)
|
||||
require.NoError(t, err)
|
||||
|
||||
index, err := queue.Allocate()
|
||||
require.NoError(t, err)
|
||||
assert.EqualValues(t, 2, index)
|
||||
_, err = queue.Allocate()
|
||||
require.Error(t, err)
|
||||
assert.Equal(t, errDone, err)
|
||||
|
||||
// Discarding a chunk the closing the queue will return errDone.
|
||||
err = queue.Discard(2)
|
||||
require.NoError(t, err)
|
||||
err = queue.Close()
|
||||
require.NoError(t, err)
|
||||
_, err = queue.Allocate()
|
||||
require.Error(t, err)
|
||||
assert.Equal(t, errDone, err)
|
||||
}
|
||||
|
||||
func TestChunkQueue_Discard(t *testing.T) {
|
||||
queue, teardown := setupChunkQueue(t)
|
||||
defer teardown()
|
||||
|
||||
// Add a few chunks to the queue and fetch a couple
|
||||
_, err := queue.Add(&chunk{Height: 3, Format: 1, Index: 0, Chunk: []byte{byte(0)}})
|
||||
require.NoError(t, err)
|
||||
_, err = queue.Add(&chunk{Height: 3, Format: 1, Index: 1, Chunk: []byte{byte(1)}})
|
||||
require.NoError(t, err)
|
||||
_, err = queue.Add(&chunk{Height: 3, Format: 1, Index: 2, Chunk: []byte{byte(2)}})
|
||||
require.NoError(t, err)
|
||||
|
||||
c, err := queue.Next()
|
||||
require.NoError(t, err)
|
||||
assert.EqualValues(t, 0, c.Index)
|
||||
c, err = queue.Next()
|
||||
require.NoError(t, err)
|
||||
assert.EqualValues(t, 1, c.Index)
|
||||
|
||||
// Discarding the first chunk and re-adding it should cause it to be returned
|
||||
// immediately by Next(), before procceeding with chunk 2
|
||||
err = queue.Discard(0)
|
||||
require.NoError(t, err)
|
||||
added, err := queue.Add(&chunk{Height: 3, Format: 1, Index: 0, Chunk: []byte{byte(0)}})
|
||||
require.NoError(t, err)
|
||||
assert.True(t, added)
|
||||
c, err = queue.Next()
|
||||
require.NoError(t, err)
|
||||
assert.EqualValues(t, 0, c.Index)
|
||||
c, err = queue.Next()
|
||||
require.NoError(t, err)
|
||||
assert.EqualValues(t, 2, c.Index)
|
||||
|
||||
// Discard then allocate, add and fetch all chunks
|
||||
for i := uint32(0); i < queue.Size(); i++ {
|
||||
err := queue.Discard(i)
|
||||
require.NoError(t, err)
|
||||
}
|
||||
for i := uint32(0); i < queue.Size(); i++ {
|
||||
_, err := queue.Allocate()
|
||||
require.NoError(t, err)
|
||||
_, err = queue.Add(&chunk{Height: 3, Format: 1, Index: i, Chunk: []byte{byte(i)}})
|
||||
require.NoError(t, err)
|
||||
c, err = queue.Next()
|
||||
require.NoError(t, err)
|
||||
assert.EqualValues(t, i, c.Index)
|
||||
}
|
||||
|
||||
// Discarding a non-existent chunk does nothing.
|
||||
err = queue.Discard(99)
|
||||
require.NoError(t, err)
|
||||
|
||||
// When discard a couple of chunks, we should be able to allocate, add, and fetch them again.
|
||||
err = queue.Discard(3)
|
||||
require.NoError(t, err)
|
||||
err = queue.Discard(1)
|
||||
require.NoError(t, err)
|
||||
|
||||
index, err := queue.Allocate()
|
||||
require.NoError(t, err)
|
||||
assert.EqualValues(t, 1, index)
|
||||
index, err = queue.Allocate()
|
||||
require.NoError(t, err)
|
||||
assert.EqualValues(t, 3, index)
|
||||
|
||||
added, err = queue.Add(&chunk{Height: 3, Format: 1, Index: 3, Chunk: []byte{3}})
|
||||
require.NoError(t, err)
|
||||
assert.True(t, added)
|
||||
added, err = queue.Add(&chunk{Height: 3, Format: 1, Index: 1, Chunk: []byte{1}})
|
||||
require.NoError(t, err)
|
||||
assert.True(t, added)
|
||||
|
||||
chunk, err := queue.Next()
|
||||
require.NoError(t, err)
|
||||
assert.EqualValues(t, 1, chunk.Index)
|
||||
|
||||
chunk, err = queue.Next()
|
||||
require.NoError(t, err)
|
||||
assert.EqualValues(t, 3, chunk.Index)
|
||||
|
||||
_, err = queue.Next()
|
||||
require.Error(t, err)
|
||||
assert.Equal(t, errDone, err)
|
||||
|
||||
// After closing the queue, discarding does nothing
|
||||
err = queue.Close()
|
||||
require.NoError(t, err)
|
||||
err = queue.Discard(2)
|
||||
require.NoError(t, err)
|
||||
}
|
||||
|
||||
func TestChunkQueue_DiscardSender(t *testing.T) {
|
||||
queue, teardown := setupChunkQueue(t)
|
||||
defer teardown()
|
||||
|
||||
// Allocate and add all chunks to the queue
|
||||
senders := []p2p.ID{"a", "b", "c"}
|
||||
for i := uint32(0); i < queue.Size(); i++ {
|
||||
_, err := queue.Allocate()
|
||||
require.NoError(t, err)
|
||||
_, err = queue.Add(&chunk{
|
||||
Height: 3,
|
||||
Format: 1,
|
||||
Index: i,
|
||||
Chunk: []byte{byte(i)},
|
||||
Sender: senders[int(i)%len(senders)],
|
||||
})
|
||||
require.NoError(t, err)
|
||||
}
|
||||
|
||||
// Fetch the first three chunks
|
||||
for i := uint32(0); i < 3; i++ {
|
||||
_, err := queue.Next()
|
||||
require.NoError(t, err)
|
||||
}
|
||||
|
||||
// Discarding an unknown sender should do nothing
|
||||
err := queue.DiscardSender("x")
|
||||
require.NoError(t, err)
|
||||
_, err = queue.Allocate()
|
||||
assert.Equal(t, errDone, err)
|
||||
|
||||
// Discarding sender b should discard chunk 4, but not chunk 1 which has already been
|
||||
// returned.
|
||||
err = queue.DiscardSender("b")
|
||||
require.NoError(t, err)
|
||||
index, err := queue.Allocate()
|
||||
require.NoError(t, err)
|
||||
assert.EqualValues(t, 4, index)
|
||||
_, err = queue.Allocate()
|
||||
assert.Equal(t, errDone, err)
|
||||
}
|
||||
|
||||
func TestChunkQueue_GetSender(t *testing.T) {
|
||||
queue, teardown := setupChunkQueue(t)
|
||||
defer teardown()
|
||||
|
||||
_, err := queue.Add(&chunk{Height: 3, Format: 1, Index: 0, Chunk: []byte{1}, Sender: p2p.ID("a")})
|
||||
require.NoError(t, err)
|
||||
_, err = queue.Add(&chunk{Height: 3, Format: 1, Index: 1, Chunk: []byte{2}, Sender: p2p.ID("b")})
|
||||
require.NoError(t, err)
|
||||
|
||||
assert.EqualValues(t, "a", queue.GetSender(0))
|
||||
assert.EqualValues(t, "b", queue.GetSender(1))
|
||||
assert.EqualValues(t, "", queue.GetSender(2))
|
||||
|
||||
// After the chunk has been processed, we should still know who the sender was
|
||||
chunk, err := queue.Next()
|
||||
require.NoError(t, err)
|
||||
require.NotNil(t, chunk)
|
||||
require.EqualValues(t, 0, chunk.Index)
|
||||
assert.EqualValues(t, "a", queue.GetSender(0))
|
||||
}
|
||||
|
||||
func TestChunkQueue_Next(t *testing.T) {
|
||||
queue, teardown := setupChunkQueue(t)
|
||||
defer teardown()
|
||||
|
||||
// Next should block waiting for the next chunks, even when given out of order.
|
||||
chNext := make(chan *chunk, 10)
|
||||
go func() {
|
||||
for {
|
||||
c, err := queue.Next()
|
||||
if err == errDone {
|
||||
close(chNext)
|
||||
break
|
||||
}
|
||||
require.NoError(t, err)
|
||||
chNext <- c
|
||||
}
|
||||
}()
|
||||
|
||||
assert.Empty(t, chNext)
|
||||
_, err := queue.Add(&chunk{Height: 3, Format: 1, Index: 1, Chunk: []byte{3, 1, 1}, Sender: p2p.ID("b")})
|
||||
require.NoError(t, err)
|
||||
select {
|
||||
case <-chNext:
|
||||
assert.Fail(t, "channel should be empty")
|
||||
default:
|
||||
}
|
||||
|
||||
_, err = queue.Add(&chunk{Height: 3, Format: 1, Index: 0, Chunk: []byte{3, 1, 0}, Sender: p2p.ID("a")})
|
||||
require.NoError(t, err)
|
||||
|
||||
assert.Equal(t,
|
||||
&chunk{Height: 3, Format: 1, Index: 0, Chunk: []byte{3, 1, 0}, Sender: p2p.ID("a")},
|
||||
<-chNext)
|
||||
assert.Equal(t,
|
||||
&chunk{Height: 3, Format: 1, Index: 1, Chunk: []byte{3, 1, 1}, Sender: p2p.ID("b")},
|
||||
<-chNext)
|
||||
|
||||
_, err = queue.Add(&chunk{Height: 3, Format: 1, Index: 4, Chunk: []byte{3, 1, 4}, Sender: p2p.ID("e")})
|
||||
require.NoError(t, err)
|
||||
select {
|
||||
case <-chNext:
|
||||
assert.Fail(t, "channel should be empty")
|
||||
default:
|
||||
}
|
||||
|
||||
_, err = queue.Add(&chunk{Height: 3, Format: 1, Index: 2, Chunk: []byte{3, 1, 2}, Sender: p2p.ID("c")})
|
||||
require.NoError(t, err)
|
||||
_, err = queue.Add(&chunk{Height: 3, Format: 1, Index: 3, Chunk: []byte{3, 1, 3}, Sender: p2p.ID("d")})
|
||||
require.NoError(t, err)
|
||||
|
||||
assert.Equal(t,
|
||||
&chunk{Height: 3, Format: 1, Index: 2, Chunk: []byte{3, 1, 2}, Sender: p2p.ID("c")},
|
||||
<-chNext)
|
||||
assert.Equal(t,
|
||||
&chunk{Height: 3, Format: 1, Index: 3, Chunk: []byte{3, 1, 3}, Sender: p2p.ID("d")},
|
||||
<-chNext)
|
||||
assert.Equal(t,
|
||||
&chunk{Height: 3, Format: 1, Index: 4, Chunk: []byte{3, 1, 4}, Sender: p2p.ID("e")},
|
||||
<-chNext)
|
||||
|
||||
_, ok := <-chNext
|
||||
assert.False(t, ok, "channel should be closed")
|
||||
|
||||
// Calling next on a finished queue should return done
|
||||
_, err = queue.Next()
|
||||
assert.Equal(t, errDone, err)
|
||||
}
|
||||
|
||||
func TestChunkQueue_Next_Closed(t *testing.T) {
|
||||
queue, teardown := setupChunkQueue(t)
|
||||
defer teardown()
|
||||
|
||||
// Calling Next on a closed queue should return done
|
||||
_, err := queue.Add(&chunk{Height: 3, Format: 1, Index: 1, Chunk: []byte{3, 1, 1}})
|
||||
require.NoError(t, err)
|
||||
err = queue.Close()
|
||||
require.NoError(t, err)
|
||||
|
||||
_, err = queue.Next()
|
||||
assert.Equal(t, errDone, err)
|
||||
}
|
||||
|
||||
func TestChunkQueue_Retry(t *testing.T) {
|
||||
queue, teardown := setupChunkQueue(t)
|
||||
defer teardown()
|
||||
|
||||
// Allocate and add all chunks to the queue
|
||||
for i := uint32(0); i < queue.Size(); i++ {
|
||||
_, err := queue.Allocate()
|
||||
require.NoError(t, err)
|
||||
_, err = queue.Add(&chunk{Height: 3, Format: 1, Index: i, Chunk: []byte{byte(i)}})
|
||||
require.NoError(t, err)
|
||||
_, err = queue.Next()
|
||||
require.NoError(t, err)
|
||||
}
|
||||
|
||||
// Retrying a couple of chunks makes Next() return them, but they are not allocatable
|
||||
queue.Retry(3)
|
||||
queue.Retry(1)
|
||||
|
||||
_, err := queue.Allocate()
|
||||
assert.Equal(t, errDone, err)
|
||||
|
||||
chunk, err := queue.Next()
|
||||
require.NoError(t, err)
|
||||
assert.EqualValues(t, 1, chunk.Index)
|
||||
|
||||
chunk, err = queue.Next()
|
||||
require.NoError(t, err)
|
||||
assert.EqualValues(t, 3, chunk.Index)
|
||||
|
||||
_, err = queue.Next()
|
||||
assert.Equal(t, errDone, err)
|
||||
}
|
||||
|
||||
func TestChunkQueue_RetryAll(t *testing.T) {
|
||||
queue, teardown := setupChunkQueue(t)
|
||||
defer teardown()
|
||||
|
||||
// Allocate and add all chunks to the queue
|
||||
for i := uint32(0); i < queue.Size(); i++ {
|
||||
_, err := queue.Allocate()
|
||||
require.NoError(t, err)
|
||||
_, err = queue.Add(&chunk{Height: 3, Format: 1, Index: i, Chunk: []byte{byte(i)}})
|
||||
require.NoError(t, err)
|
||||
_, err = queue.Next()
|
||||
require.NoError(t, err)
|
||||
}
|
||||
|
||||
_, err := queue.Next()
|
||||
assert.Equal(t, errDone, err)
|
||||
|
||||
queue.RetryAll()
|
||||
|
||||
_, err = queue.Allocate()
|
||||
assert.Equal(t, errDone, err)
|
||||
|
||||
for i := uint32(0); i < queue.Size(); i++ {
|
||||
chunk, err := queue.Next()
|
||||
require.NoError(t, err)
|
||||
assert.EqualValues(t, i, chunk.Index)
|
||||
}
|
||||
|
||||
_, err = queue.Next()
|
||||
assert.Equal(t, errDone, err)
|
||||
}
|
||||
|
||||
func TestChunkQueue_Size(t *testing.T) {
|
||||
queue, teardown := setupChunkQueue(t)
|
||||
defer teardown()
|
||||
|
||||
assert.EqualValues(t, 5, queue.Size())
|
||||
|
||||
err := queue.Close()
|
||||
require.NoError(t, err)
|
||||
assert.EqualValues(t, 0, queue.Size())
|
||||
}
|
||||
|
||||
func TestChunkQueue_WaitFor(t *testing.T) {
|
||||
queue, teardown := setupChunkQueue(t)
|
||||
defer teardown()
|
||||
|
||||
waitFor1 := queue.WaitFor(1)
|
||||
waitFor4 := queue.WaitFor(4)
|
||||
|
||||
// Adding 0 and 2 should not trigger waiters
|
||||
_, err := queue.Add(&chunk{Height: 3, Format: 1, Index: 0, Chunk: []byte{3, 1, 0}})
|
||||
require.NoError(t, err)
|
||||
_, err = queue.Add(&chunk{Height: 3, Format: 1, Index: 2, Chunk: []byte{3, 1, 2}})
|
||||
require.NoError(t, err)
|
||||
select {
|
||||
case <-waitFor1:
|
||||
require.Fail(t, "WaitFor(1) should not trigger on 0 or 2")
|
||||
case <-waitFor4:
|
||||
require.Fail(t, "WaitFor(4) should not trigger on 0 or 2")
|
||||
default:
|
||||
}
|
||||
|
||||
// Adding 1 should trigger WaitFor(1), but not WaitFor(4). The channel should be closed.
|
||||
_, err = queue.Add(&chunk{Height: 3, Format: 1, Index: 1, Chunk: []byte{3, 1, 1}})
|
||||
require.NoError(t, err)
|
||||
assert.EqualValues(t, 1, <-waitFor1)
|
||||
_, ok := <-waitFor1
|
||||
assert.False(t, ok)
|
||||
select {
|
||||
case <-waitFor4:
|
||||
require.Fail(t, "WaitFor(4) should not trigger on 0 or 2")
|
||||
default:
|
||||
}
|
||||
|
||||
// Fetch the first chunk. At this point, waiting for either 0 (retrieved from pool) or 1
|
||||
// (queued in pool) should immediately return true.
|
||||
c, err := queue.Next()
|
||||
require.NoError(t, err)
|
||||
assert.EqualValues(t, 0, c.Index)
|
||||
|
||||
w := queue.WaitFor(0)
|
||||
assert.EqualValues(t, 0, <-w)
|
||||
_, ok = <-w
|
||||
assert.False(t, ok)
|
||||
|
||||
w = queue.WaitFor(1)
|
||||
assert.EqualValues(t, 1, <-w)
|
||||
_, ok = <-w
|
||||
assert.False(t, ok)
|
||||
|
||||
// Close the queue. This should cause the waiter for 4 to close, and also cause any future
|
||||
// waiters to get closed channels.
|
||||
err = queue.Close()
|
||||
require.NoError(t, err)
|
||||
_, ok = <-waitFor4
|
||||
assert.False(t, ok)
|
||||
|
||||
w = queue.WaitFor(3)
|
||||
_, ok = <-w
|
||||
assert.False(t, ok)
|
||||
}
|
||||
129
statesync/messages.go
Normal file
129
statesync/messages.go
Normal file
@@ -0,0 +1,129 @@
|
||||
package statesync
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"fmt"
|
||||
|
||||
amino "github.com/tendermint/go-amino"
|
||||
|
||||
"github.com/tendermint/tendermint/types"
|
||||
)
|
||||
|
||||
const (
|
||||
// snapshotMsgSize is the maximum size of a snapshotResponseMessage
|
||||
snapshotMsgSize = int(4e6)
|
||||
// chunkMsgSize is the maximum size of a chunkResponseMessage
|
||||
chunkMsgSize = int(16e6)
|
||||
// maxMsgSize is the maximum size of any message
|
||||
maxMsgSize = chunkMsgSize
|
||||
)
|
||||
|
||||
var cdc = amino.NewCodec()
|
||||
|
||||
func init() {
|
||||
cdc.RegisterInterface((*Message)(nil), nil)
|
||||
cdc.RegisterConcrete(&snapshotsRequestMessage{}, "tendermint/SnapshotsRequestMessage", nil)
|
||||
cdc.RegisterConcrete(&snapshotsResponseMessage{}, "tendermint/SnapshotsResponseMessage", nil)
|
||||
cdc.RegisterConcrete(&chunkRequestMessage{}, "tendermint/ChunkRequestMessage", nil)
|
||||
cdc.RegisterConcrete(&chunkResponseMessage{}, "tendermint/ChunkResponseMessage", nil)
|
||||
types.RegisterBlockAmino(cdc)
|
||||
}
|
||||
|
||||
// decodeMsg decodes a message.
|
||||
func decodeMsg(bz []byte) (Message, error) {
|
||||
if len(bz) > maxMsgSize {
|
||||
return nil, fmt.Errorf("msg exceeds max size (%d > %d)", len(bz), maxMsgSize)
|
||||
}
|
||||
var msg Message
|
||||
err := cdc.UnmarshalBinaryBare(bz, &msg)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return msg, nil
|
||||
}
|
||||
|
||||
// Message is a message sent and received by the reactor.
|
||||
type Message interface {
|
||||
ValidateBasic() error
|
||||
}
|
||||
|
||||
// snapshotsRequestMessage requests recent snapshots from a peer.
|
||||
type snapshotsRequestMessage struct{}
|
||||
|
||||
// ValidateBasic implements Message.
|
||||
func (m *snapshotsRequestMessage) ValidateBasic() error {
|
||||
if m == nil {
|
||||
return errors.New("nil message")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// SnapshotResponseMessage contains information about a single snapshot.
|
||||
type snapshotsResponseMessage struct {
|
||||
Height uint64
|
||||
Format uint32
|
||||
Chunks uint32
|
||||
Hash []byte
|
||||
Metadata []byte
|
||||
}
|
||||
|
||||
// ValidateBasic implements Message.
|
||||
func (m *snapshotsResponseMessage) ValidateBasic() error {
|
||||
if m == nil {
|
||||
return errors.New("nil message")
|
||||
}
|
||||
if m.Height == 0 {
|
||||
return errors.New("height cannot be 0")
|
||||
}
|
||||
if len(m.Hash) == 0 {
|
||||
return errors.New("snapshot has no hash")
|
||||
}
|
||||
if m.Chunks == 0 {
|
||||
return errors.New("snapshot has no chunks")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// chunkRequestMessage requests a single chunk from a peer.
|
||||
type chunkRequestMessage struct {
|
||||
Height uint64
|
||||
Format uint32
|
||||
Index uint32
|
||||
}
|
||||
|
||||
// ValidateBasic implements Message.
|
||||
func (m *chunkRequestMessage) ValidateBasic() error {
|
||||
if m == nil {
|
||||
return errors.New("nil message")
|
||||
}
|
||||
if m.Height == 0 {
|
||||
return errors.New("height cannot be 0")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// chunkResponseMessage contains a single chunk from a peer.
|
||||
type chunkResponseMessage struct {
|
||||
Height uint64
|
||||
Format uint32
|
||||
Index uint32
|
||||
Chunk []byte
|
||||
Missing bool
|
||||
}
|
||||
|
||||
// ValidateBasic implements Message.
|
||||
func (m *chunkResponseMessage) ValidateBasic() error {
|
||||
if m == nil {
|
||||
return errors.New("nil message")
|
||||
}
|
||||
if m.Height == 0 {
|
||||
return errors.New("height cannot be 0")
|
||||
}
|
||||
if m.Missing && len(m.Chunk) > 0 {
|
||||
return errors.New("missing chunk cannot have contents")
|
||||
}
|
||||
if !m.Missing && m.Chunk == nil {
|
||||
return errors.New("chunk cannot be nil")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
106
statesync/messages_test.go
Normal file
106
statesync/messages_test.go
Normal file
@@ -0,0 +1,106 @@
|
||||
package statesync
|
||||
|
||||
import (
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/require"
|
||||
)
|
||||
|
||||
func TestSnapshotsRequestMessage_ValidateBasic(t *testing.T) {
|
||||
testcases := map[string]struct {
|
||||
msg *snapshotsRequestMessage
|
||||
valid bool
|
||||
}{
|
||||
"nil": {nil, false},
|
||||
"valid": {&snapshotsRequestMessage{}, true},
|
||||
}
|
||||
for name, tc := range testcases {
|
||||
tc := tc
|
||||
t.Run(name, func(t *testing.T) {
|
||||
err := tc.msg.ValidateBasic()
|
||||
if tc.valid {
|
||||
require.NoError(t, err)
|
||||
} else {
|
||||
require.Error(t, err)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestSnapshotsResponseMessage_ValidateBasic(t *testing.T) {
|
||||
testcases := map[string]struct {
|
||||
msg *snapshotsResponseMessage
|
||||
valid bool
|
||||
}{
|
||||
"nil": {nil, false},
|
||||
"valid": {&snapshotsResponseMessage{Height: 1, Format: 1, Chunks: 2, Hash: []byte{1}}, true},
|
||||
"0 height": {&snapshotsResponseMessage{Height: 0, Format: 1, Chunks: 2, Hash: []byte{1}}, false},
|
||||
"0 format": {&snapshotsResponseMessage{Height: 1, Format: 0, Chunks: 2, Hash: []byte{1}}, true},
|
||||
"0 chunks": {&snapshotsResponseMessage{Height: 1, Format: 1, Hash: []byte{1}}, false},
|
||||
"no hash": {&snapshotsResponseMessage{Height: 1, Format: 1, Chunks: 2, Hash: []byte{}}, false},
|
||||
}
|
||||
for name, tc := range testcases {
|
||||
tc := tc
|
||||
t.Run(name, func(t *testing.T) {
|
||||
err := tc.msg.ValidateBasic()
|
||||
if tc.valid {
|
||||
require.NoError(t, err)
|
||||
} else {
|
||||
require.Error(t, err)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestChunkRequestMessage_ValidateBasic(t *testing.T) {
|
||||
testcases := map[string]struct {
|
||||
msg *chunkRequestMessage
|
||||
valid bool
|
||||
}{
|
||||
"nil": {nil, false},
|
||||
"valid": {&chunkRequestMessage{Height: 1, Format: 1, Index: 1}, true},
|
||||
"0 height": {&chunkRequestMessage{Height: 0, Format: 1, Index: 1}, false},
|
||||
"0 format": {&chunkRequestMessage{Height: 1, Format: 0, Index: 1}, true},
|
||||
"0 chunk": {&chunkRequestMessage{Height: 1, Format: 1, Index: 0}, true},
|
||||
}
|
||||
for name, tc := range testcases {
|
||||
tc := tc
|
||||
t.Run(name, func(t *testing.T) {
|
||||
err := tc.msg.ValidateBasic()
|
||||
if tc.valid {
|
||||
require.NoError(t, err)
|
||||
} else {
|
||||
require.Error(t, err)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestChunkResponseMessage_ValidateBasic(t *testing.T) {
|
||||
testcases := map[string]struct {
|
||||
msg *chunkResponseMessage
|
||||
valid bool
|
||||
}{
|
||||
"nil message": {nil, false},
|
||||
"valid": {&chunkResponseMessage{Height: 1, Format: 1, Index: 1, Chunk: []byte{1}}, true},
|
||||
"0 height": {&chunkResponseMessage{Height: 0, Format: 1, Index: 1, Chunk: []byte{1}}, false},
|
||||
"0 format": {&chunkResponseMessage{Height: 1, Format: 0, Index: 1, Chunk: []byte{1}}, true},
|
||||
"0 chunk": {&chunkResponseMessage{Height: 1, Format: 1, Index: 0, Chunk: []byte{1}}, true},
|
||||
"empty body": {&chunkResponseMessage{Height: 1, Format: 1, Index: 1, Chunk: []byte{}}, true},
|
||||
"nil body": {&chunkResponseMessage{Height: 1, Format: 1, Index: 1, Chunk: nil}, false},
|
||||
"missing": {&chunkResponseMessage{Height: 1, Format: 1, Index: 1, Missing: true}, true},
|
||||
"missing with empty": {&chunkResponseMessage{Height: 1, Format: 1, Index: 1, Missing: true, Chunk: []byte{}}, true},
|
||||
"missing with body": {&chunkResponseMessage{Height: 1, Format: 1, Index: 1, Missing: true, Chunk: []byte{1}}, false},
|
||||
}
|
||||
for name, tc := range testcases {
|
||||
tc := tc
|
||||
t.Run(name, func(t *testing.T) {
|
||||
err := tc.msg.ValidateBasic()
|
||||
if tc.valid {
|
||||
require.NoError(t, err)
|
||||
} else {
|
||||
require.Error(t, err)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
82
statesync/mocks/state_provider.go
Normal file
82
statesync/mocks/state_provider.go
Normal file
@@ -0,0 +1,82 @@
|
||||
// Code generated by mockery v1.0.0. DO NOT EDIT.
|
||||
|
||||
package mocks
|
||||
|
||||
import (
|
||||
mock "github.com/stretchr/testify/mock"
|
||||
state "github.com/tendermint/tendermint/state"
|
||||
|
||||
types "github.com/tendermint/tendermint/types"
|
||||
)
|
||||
|
||||
// StateProvider is an autogenerated mock type for the StateProvider type
|
||||
type StateProvider struct {
|
||||
mock.Mock
|
||||
}
|
||||
|
||||
// AppHash provides a mock function with given fields: height
|
||||
func (_m *StateProvider) AppHash(height uint64) ([]byte, error) {
|
||||
ret := _m.Called(height)
|
||||
|
||||
var r0 []byte
|
||||
if rf, ok := ret.Get(0).(func(uint64) []byte); ok {
|
||||
r0 = rf(height)
|
||||
} else {
|
||||
if ret.Get(0) != nil {
|
||||
r0 = ret.Get(0).([]byte)
|
||||
}
|
||||
}
|
||||
|
||||
var r1 error
|
||||
if rf, ok := ret.Get(1).(func(uint64) error); ok {
|
||||
r1 = rf(height)
|
||||
} else {
|
||||
r1 = ret.Error(1)
|
||||
}
|
||||
|
||||
return r0, r1
|
||||
}
|
||||
|
||||
// Commit provides a mock function with given fields: height
|
||||
func (_m *StateProvider) Commit(height uint64) (*types.Commit, error) {
|
||||
ret := _m.Called(height)
|
||||
|
||||
var r0 *types.Commit
|
||||
if rf, ok := ret.Get(0).(func(uint64) *types.Commit); ok {
|
||||
r0 = rf(height)
|
||||
} else {
|
||||
if ret.Get(0) != nil {
|
||||
r0 = ret.Get(0).(*types.Commit)
|
||||
}
|
||||
}
|
||||
|
||||
var r1 error
|
||||
if rf, ok := ret.Get(1).(func(uint64) error); ok {
|
||||
r1 = rf(height)
|
||||
} else {
|
||||
r1 = ret.Error(1)
|
||||
}
|
||||
|
||||
return r0, r1
|
||||
}
|
||||
|
||||
// State provides a mock function with given fields: height
|
||||
func (_m *StateProvider) State(height uint64) (state.State, error) {
|
||||
ret := _m.Called(height)
|
||||
|
||||
var r0 state.State
|
||||
if rf, ok := ret.Get(0).(func(uint64) state.State); ok {
|
||||
r0 = rf(height)
|
||||
} else {
|
||||
r0 = ret.Get(0).(state.State)
|
||||
}
|
||||
|
||||
var r1 error
|
||||
if rf, ok := ret.Get(1).(func(uint64) error); ok {
|
||||
r1 = rf(height)
|
||||
} else {
|
||||
r1 = ret.Error(1)
|
||||
}
|
||||
|
||||
return r0, r1
|
||||
}
|
||||
261
statesync/reactor.go
Normal file
261
statesync/reactor.go
Normal file
@@ -0,0 +1,261 @@
|
||||
package statesync
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"sort"
|
||||
"sync"
|
||||
|
||||
abci "github.com/tendermint/tendermint/abci/types"
|
||||
"github.com/tendermint/tendermint/p2p"
|
||||
"github.com/tendermint/tendermint/proxy"
|
||||
sm "github.com/tendermint/tendermint/state"
|
||||
"github.com/tendermint/tendermint/types"
|
||||
)
|
||||
|
||||
const (
|
||||
// SnapshotChannel exchanges snapshot metadata
|
||||
SnapshotChannel = byte(0x60)
|
||||
// ChunkChannel exchanges chunk contents
|
||||
ChunkChannel = byte(0x61)
|
||||
// recentSnapshots is the number of recent snapshots to send and receive per peer.
|
||||
recentSnapshots = 10
|
||||
)
|
||||
|
||||
// Reactor handles state sync, both restoring snapshots for the local node and serving snapshots
|
||||
// for other nodes.
|
||||
type Reactor struct {
|
||||
p2p.BaseReactor
|
||||
|
||||
conn proxy.AppConnSnapshot
|
||||
connQuery proxy.AppConnQuery
|
||||
tempDir string
|
||||
|
||||
// This will only be set when a state sync is in progress. It is used to feed received
|
||||
// snapshots and chunks into the sync.
|
||||
mtx sync.RWMutex
|
||||
syncer *syncer
|
||||
}
|
||||
|
||||
// NewReactor creates a new state sync reactor.
|
||||
func NewReactor(conn proxy.AppConnSnapshot, connQuery proxy.AppConnQuery, tempDir string) *Reactor {
|
||||
r := &Reactor{
|
||||
conn: conn,
|
||||
connQuery: connQuery,
|
||||
}
|
||||
r.BaseReactor = *p2p.NewBaseReactor("StateSync", r)
|
||||
return r
|
||||
}
|
||||
|
||||
// GetChannels implements p2p.Reactor.
|
||||
func (r *Reactor) GetChannels() []*p2p.ChannelDescriptor {
|
||||
return []*p2p.ChannelDescriptor{
|
||||
{
|
||||
ID: SnapshotChannel,
|
||||
Priority: 3,
|
||||
SendQueueCapacity: 10,
|
||||
RecvMessageCapacity: snapshotMsgSize,
|
||||
},
|
||||
{
|
||||
ID: ChunkChannel,
|
||||
Priority: 1,
|
||||
SendQueueCapacity: 4,
|
||||
RecvMessageCapacity: chunkMsgSize,
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
// OnStart implements p2p.Reactor.
|
||||
func (r *Reactor) OnStart() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
// AddPeer implements p2p.Reactor.
|
||||
func (r *Reactor) AddPeer(peer p2p.Peer) {
|
||||
r.mtx.RLock()
|
||||
defer r.mtx.RUnlock()
|
||||
if r.syncer != nil {
|
||||
r.syncer.AddPeer(peer)
|
||||
}
|
||||
}
|
||||
|
||||
// RemovePeer implements p2p.Reactor.
|
||||
func (r *Reactor) RemovePeer(peer p2p.Peer, reason interface{}) {
|
||||
r.mtx.RLock()
|
||||
defer r.mtx.RUnlock()
|
||||
if r.syncer != nil {
|
||||
r.syncer.RemovePeer(peer)
|
||||
}
|
||||
}
|
||||
|
||||
// Receive implements p2p.Reactor.
|
||||
func (r *Reactor) Receive(chID byte, src p2p.Peer, msgBytes []byte) {
|
||||
if !r.IsRunning() {
|
||||
return
|
||||
}
|
||||
|
||||
msg, err := decodeMsg(msgBytes)
|
||||
if err != nil {
|
||||
r.Logger.Error("Error decoding message", "src", src, "chId", chID, "msg", msg, "err", err, "bytes", msgBytes)
|
||||
r.Switch.StopPeerForError(src, err)
|
||||
return
|
||||
}
|
||||
err = msg.ValidateBasic()
|
||||
if err != nil {
|
||||
r.Logger.Error("Invalid message", "peer", src, "msg", msg, "err", err)
|
||||
r.Switch.StopPeerForError(src, err)
|
||||
return
|
||||
}
|
||||
|
||||
switch chID {
|
||||
case SnapshotChannel:
|
||||
switch msg := msg.(type) {
|
||||
case *snapshotsRequestMessage:
|
||||
snapshots, err := r.recentSnapshots(recentSnapshots)
|
||||
if err != nil {
|
||||
r.Logger.Error("Failed to fetch snapshots", "err", err)
|
||||
return
|
||||
}
|
||||
for _, snapshot := range snapshots {
|
||||
r.Logger.Debug("Advertising snapshot", "height", snapshot.Height,
|
||||
"format", snapshot.Format, "peer", src.ID())
|
||||
src.Send(chID, cdc.MustMarshalBinaryBare(&snapshotsResponseMessage{
|
||||
Height: snapshot.Height,
|
||||
Format: snapshot.Format,
|
||||
Chunks: snapshot.Chunks,
|
||||
Hash: snapshot.Hash,
|
||||
Metadata: snapshot.Metadata,
|
||||
}))
|
||||
}
|
||||
|
||||
case *snapshotsResponseMessage:
|
||||
r.mtx.RLock()
|
||||
defer r.mtx.RUnlock()
|
||||
if r.syncer == nil {
|
||||
r.Logger.Debug("Received unexpected snapshot, no state sync in progress")
|
||||
return
|
||||
}
|
||||
r.Logger.Debug("Received snapshot", "height", msg.Height, "format", msg.Format, "peer", src.ID())
|
||||
_, err := r.syncer.AddSnapshot(src, &snapshot{
|
||||
Height: msg.Height,
|
||||
Format: msg.Format,
|
||||
Chunks: msg.Chunks,
|
||||
Hash: msg.Hash,
|
||||
Metadata: msg.Metadata,
|
||||
})
|
||||
if err != nil {
|
||||
r.Logger.Error("Failed to add snapshot", "height", msg.Height, "format", msg.Format,
|
||||
"peer", src.ID(), "err", err)
|
||||
return
|
||||
}
|
||||
|
||||
default:
|
||||
r.Logger.Error("Received unknown message %T", msg)
|
||||
}
|
||||
|
||||
case ChunkChannel:
|
||||
switch msg := msg.(type) {
|
||||
case *chunkRequestMessage:
|
||||
r.Logger.Debug("Received chunk request", "height", msg.Height, "format", msg.Format,
|
||||
"chunk", msg.Index, "peer", src.ID())
|
||||
resp, err := r.conn.LoadSnapshotChunkSync(abci.RequestLoadSnapshotChunk{
|
||||
Height: msg.Height,
|
||||
Format: msg.Format,
|
||||
Chunk: msg.Index,
|
||||
})
|
||||
if err != nil {
|
||||
r.Logger.Error("Failed to load chunk", "height", msg.Height, "format", msg.Format,
|
||||
"chunk", msg.Index, "err", err)
|
||||
return
|
||||
}
|
||||
r.Logger.Debug("Sending chunk", "height", msg.Height, "format", msg.Format,
|
||||
"chunk", msg.Index, "peer", src.ID())
|
||||
src.Send(ChunkChannel, cdc.MustMarshalBinaryBare(&chunkResponseMessage{
|
||||
Height: msg.Height,
|
||||
Format: msg.Format,
|
||||
Index: msg.Index,
|
||||
Chunk: resp.Chunk,
|
||||
Missing: resp.Chunk == nil,
|
||||
}))
|
||||
|
||||
case *chunkResponseMessage:
|
||||
r.mtx.RLock()
|
||||
defer r.mtx.RUnlock()
|
||||
if r.syncer == nil {
|
||||
r.Logger.Debug("Received unexpected chunk, no state sync in progress", "peer", src.ID())
|
||||
return
|
||||
}
|
||||
r.Logger.Debug("Received chunk, adding to sync", "height", msg.Height, "format", msg.Format,
|
||||
"chunk", msg.Index, "peer", src.ID())
|
||||
_, err := r.syncer.AddChunk(&chunk{
|
||||
Height: msg.Height,
|
||||
Format: msg.Format,
|
||||
Index: msg.Index,
|
||||
Chunk: msg.Chunk,
|
||||
Sender: src.ID(),
|
||||
})
|
||||
if err != nil {
|
||||
r.Logger.Error("Failed to add chunk", "height", msg.Height, "format", msg.Format,
|
||||
"chunk", msg.Index, "err", err)
|
||||
return
|
||||
}
|
||||
|
||||
default:
|
||||
r.Logger.Error("Received unknown message %T", msg)
|
||||
}
|
||||
|
||||
default:
|
||||
r.Logger.Error("Received message on invalid channel %x", chID)
|
||||
}
|
||||
}
|
||||
|
||||
// recentSnapshots fetches the n most recent snapshots from the app
|
||||
func (r *Reactor) recentSnapshots(n uint32) ([]*snapshot, error) {
|
||||
resp, err := r.conn.ListSnapshotsSync(abci.RequestListSnapshots{})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
sort.Slice(resp.Snapshots, func(i, j int) bool {
|
||||
a := resp.Snapshots[i]
|
||||
b := resp.Snapshots[j]
|
||||
switch {
|
||||
case a.Height > b.Height:
|
||||
return true
|
||||
case a.Height == b.Height && a.Format > b.Format:
|
||||
return true
|
||||
default:
|
||||
return false
|
||||
}
|
||||
})
|
||||
snapshots := make([]*snapshot, 0, n)
|
||||
for i, s := range resp.Snapshots {
|
||||
if i >= recentSnapshots {
|
||||
break
|
||||
}
|
||||
snapshots = append(snapshots, &snapshot{
|
||||
Height: s.Height,
|
||||
Format: s.Format,
|
||||
Chunks: s.Chunks,
|
||||
Hash: s.Hash,
|
||||
Metadata: s.Metadata,
|
||||
})
|
||||
}
|
||||
return snapshots, nil
|
||||
}
|
||||
|
||||
// Sync runs a state sync, returning the new state and last commit at the snapshot height.
|
||||
// The caller must store the state and commit in the state database and block store.
|
||||
func (r *Reactor) Sync(stateProvider StateProvider) (sm.State, *types.Commit, error) {
|
||||
r.mtx.Lock()
|
||||
if r.syncer != nil {
|
||||
r.mtx.Unlock()
|
||||
return sm.State{}, nil, errors.New("a state sync is already in progress")
|
||||
}
|
||||
r.syncer = newSyncer(r.Logger, r.conn, r.connQuery, stateProvider, r.tempDir)
|
||||
r.mtx.Unlock()
|
||||
|
||||
state, commit, err := r.syncer.SyncAny(defaultDiscoveryTime)
|
||||
r.mtx.Lock()
|
||||
r.syncer = nil
|
||||
r.mtx.Unlock()
|
||||
return state, commit, err
|
||||
}
|
||||
148
statesync/reactor_test.go
Normal file
148
statesync/reactor_test.go
Normal file
@@ -0,0 +1,148 @@
|
||||
package statesync
|
||||
|
||||
import (
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/mock"
|
||||
"github.com/stretchr/testify/require"
|
||||
|
||||
abci "github.com/tendermint/tendermint/abci/types"
|
||||
"github.com/tendermint/tendermint/p2p"
|
||||
p2pmocks "github.com/tendermint/tendermint/p2p/mocks"
|
||||
proxymocks "github.com/tendermint/tendermint/proxy/mocks"
|
||||
)
|
||||
|
||||
func TestReactor_Receive_ChunkRequestMessage(t *testing.T) {
|
||||
testcases := map[string]struct {
|
||||
request *chunkRequestMessage
|
||||
chunk []byte
|
||||
expectResponse *chunkResponseMessage
|
||||
}{
|
||||
"chunk is returned": {
|
||||
&chunkRequestMessage{Height: 1, Format: 1, Index: 1},
|
||||
[]byte{1, 2, 3},
|
||||
&chunkResponseMessage{Height: 1, Format: 1, Index: 1, Chunk: []byte{1, 2, 3}}},
|
||||
"empty chunk is returned, as nil": {
|
||||
&chunkRequestMessage{Height: 1, Format: 1, Index: 1},
|
||||
[]byte{},
|
||||
&chunkResponseMessage{Height: 1, Format: 1, Index: 1, Chunk: nil}},
|
||||
"nil (missing) chunk is returned as missing": {
|
||||
&chunkRequestMessage{Height: 1, Format: 1, Index: 1},
|
||||
nil,
|
||||
&chunkResponseMessage{Height: 1, Format: 1, Index: 1, Missing: true},
|
||||
},
|
||||
}
|
||||
|
||||
for name, tc := range testcases {
|
||||
tc := tc
|
||||
t.Run(name, func(t *testing.T) {
|
||||
// Mock ABCI connection to return local snapshots
|
||||
conn := &proxymocks.AppConnSnapshot{}
|
||||
conn.On("LoadSnapshotChunkSync", abci.RequestLoadSnapshotChunk{
|
||||
Height: tc.request.Height,
|
||||
Format: tc.request.Format,
|
||||
Chunk: tc.request.Index,
|
||||
}).Return(&abci.ResponseLoadSnapshotChunk{Chunk: tc.chunk}, nil)
|
||||
|
||||
// Mock peer to store response, if found
|
||||
peer := &p2pmocks.Peer{}
|
||||
peer.On("ID").Return(p2p.ID("id"))
|
||||
var response *chunkResponseMessage
|
||||
if tc.expectResponse != nil {
|
||||
peer.On("Send", ChunkChannel, mock.Anything).Run(func(args mock.Arguments) {
|
||||
msg, err := decodeMsg(args[1].([]byte))
|
||||
require.NoError(t, err)
|
||||
response = msg.(*chunkResponseMessage)
|
||||
}).Return(true)
|
||||
}
|
||||
|
||||
// Start a reactor and send a chunkRequestMessage, then wait for and check response
|
||||
r := NewReactor(conn, nil, "")
|
||||
err := r.Start()
|
||||
require.NoError(t, err)
|
||||
defer r.Stop()
|
||||
|
||||
r.Receive(ChunkChannel, peer, cdc.MustMarshalBinaryBare(tc.request))
|
||||
time.Sleep(100 * time.Millisecond)
|
||||
assert.Equal(t, tc.expectResponse, response)
|
||||
|
||||
conn.AssertExpectations(t)
|
||||
peer.AssertExpectations(t)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestReactor_Receive_SnapshotRequestMessage(t *testing.T) {
|
||||
testcases := map[string]struct {
|
||||
snapshots []*abci.Snapshot
|
||||
expectResponses []*snapshotsResponseMessage
|
||||
}{
|
||||
"no snapshots": {nil, []*snapshotsResponseMessage{}},
|
||||
">10 unordered snapshots": {
|
||||
[]*abci.Snapshot{
|
||||
{Height: 1, Format: 2, Chunks: 7, Hash: []byte{1, 2}, Metadata: []byte{1}},
|
||||
{Height: 2, Format: 2, Chunks: 7, Hash: []byte{2, 2}, Metadata: []byte{2}},
|
||||
{Height: 3, Format: 2, Chunks: 7, Hash: []byte{3, 2}, Metadata: []byte{3}},
|
||||
{Height: 1, Format: 1, Chunks: 7, Hash: []byte{1, 1}, Metadata: []byte{4}},
|
||||
{Height: 2, Format: 1, Chunks: 7, Hash: []byte{2, 1}, Metadata: []byte{5}},
|
||||
{Height: 3, Format: 1, Chunks: 7, Hash: []byte{3, 1}, Metadata: []byte{6}},
|
||||
{Height: 1, Format: 4, Chunks: 7, Hash: []byte{1, 4}, Metadata: []byte{7}},
|
||||
{Height: 2, Format: 4, Chunks: 7, Hash: []byte{2, 4}, Metadata: []byte{8}},
|
||||
{Height: 3, Format: 4, Chunks: 7, Hash: []byte{3, 4}, Metadata: []byte{9}},
|
||||
{Height: 1, Format: 3, Chunks: 7, Hash: []byte{1, 3}, Metadata: []byte{10}},
|
||||
{Height: 2, Format: 3, Chunks: 7, Hash: []byte{2, 3}, Metadata: []byte{11}},
|
||||
{Height: 3, Format: 3, Chunks: 7, Hash: []byte{3, 3}, Metadata: []byte{12}},
|
||||
},
|
||||
[]*snapshotsResponseMessage{
|
||||
{Height: 3, Format: 4, Chunks: 7, Hash: []byte{3, 4}, Metadata: []byte{9}},
|
||||
{Height: 3, Format: 3, Chunks: 7, Hash: []byte{3, 3}, Metadata: []byte{12}},
|
||||
{Height: 3, Format: 2, Chunks: 7, Hash: []byte{3, 2}, Metadata: []byte{3}},
|
||||
{Height: 3, Format: 1, Chunks: 7, Hash: []byte{3, 1}, Metadata: []byte{6}},
|
||||
{Height: 2, Format: 4, Chunks: 7, Hash: []byte{2, 4}, Metadata: []byte{8}},
|
||||
{Height: 2, Format: 3, Chunks: 7, Hash: []byte{2, 3}, Metadata: []byte{11}},
|
||||
{Height: 2, Format: 2, Chunks: 7, Hash: []byte{2, 2}, Metadata: []byte{2}},
|
||||
{Height: 2, Format: 1, Chunks: 7, Hash: []byte{2, 1}, Metadata: []byte{5}},
|
||||
{Height: 1, Format: 4, Chunks: 7, Hash: []byte{1, 4}, Metadata: []byte{7}},
|
||||
{Height: 1, Format: 3, Chunks: 7, Hash: []byte{1, 3}, Metadata: []byte{10}},
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
for name, tc := range testcases {
|
||||
tc := tc
|
||||
t.Run(name, func(t *testing.T) {
|
||||
// Mock ABCI connection to return local snapshots
|
||||
conn := &proxymocks.AppConnSnapshot{}
|
||||
conn.On("ListSnapshotsSync", abci.RequestListSnapshots{}).Return(&abci.ResponseListSnapshots{
|
||||
Snapshots: tc.snapshots,
|
||||
}, nil)
|
||||
|
||||
// Mock peer to catch responses and store them in a slice
|
||||
responses := []*snapshotsResponseMessage{}
|
||||
peer := &p2pmocks.Peer{}
|
||||
if len(tc.expectResponses) > 0 {
|
||||
peer.On("ID").Return(p2p.ID("id"))
|
||||
peer.On("Send", SnapshotChannel, mock.Anything).Run(func(args mock.Arguments) {
|
||||
msg, err := decodeMsg(args[1].([]byte))
|
||||
require.NoError(t, err)
|
||||
responses = append(responses, msg.(*snapshotsResponseMessage))
|
||||
}).Return(true)
|
||||
}
|
||||
|
||||
// Start a reactor and send a SnapshotsRequestMessage, then wait for and check responses
|
||||
r := NewReactor(conn, nil, "")
|
||||
err := r.Start()
|
||||
require.NoError(t, err)
|
||||
defer r.Stop()
|
||||
|
||||
r.Receive(SnapshotChannel, peer, cdc.MustMarshalBinaryBare(&snapshotsRequestMessage{}))
|
||||
time.Sleep(100 * time.Millisecond)
|
||||
assert.Equal(t, tc.expectResponses, responses)
|
||||
|
||||
conn.AssertExpectations(t)
|
||||
peer.AssertExpectations(t)
|
||||
})
|
||||
}
|
||||
}
|
||||
263
statesync/snapshots.go
Normal file
263
statesync/snapshots.go
Normal file
@@ -0,0 +1,263 @@
|
||||
package statesync
|
||||
|
||||
import (
|
||||
"crypto/sha256"
|
||||
"fmt"
|
||||
"math/rand"
|
||||
"sort"
|
||||
"sync"
|
||||
|
||||
"github.com/tendermint/tendermint/p2p"
|
||||
)
|
||||
|
||||
// snapshotKey is a snapshot key used for lookups.
|
||||
type snapshotKey [sha256.Size]byte
|
||||
|
||||
// snapshot contains data about a snapshot.
|
||||
type snapshot struct {
|
||||
Height uint64
|
||||
Format uint32
|
||||
Chunks uint32
|
||||
Hash []byte
|
||||
Metadata []byte
|
||||
|
||||
trustedAppHash []byte // populated by light client
|
||||
}
|
||||
|
||||
// Key generates a snapshot key, used for lookups. It takes into account not only the height and
|
||||
// format, but also the chunks, hash, and metadata in case peers have generated snapshots in a
|
||||
// non-deterministic manner. All fields must be equal for the snapshot to be considered the same.
|
||||
func (s *snapshot) Key() snapshotKey {
|
||||
// Hash.Write() never returns an error.
|
||||
hasher := sha256.New()
|
||||
hasher.Write([]byte(fmt.Sprintf("%v:%v:%v", s.Height, s.Format, s.Chunks)))
|
||||
hasher.Write(s.Hash)
|
||||
hasher.Write(s.Metadata)
|
||||
var key snapshotKey
|
||||
copy(key[:], hasher.Sum(nil))
|
||||
return key
|
||||
}
|
||||
|
||||
// snapshotPool discovers and aggregates snapshots across peers.
|
||||
type snapshotPool struct {
|
||||
stateProvider StateProvider
|
||||
|
||||
sync.Mutex
|
||||
snapshots map[snapshotKey]*snapshot
|
||||
snapshotPeers map[snapshotKey]map[p2p.ID]p2p.Peer
|
||||
|
||||
// indexes for fast searches
|
||||
formatIndex map[uint32]map[snapshotKey]bool
|
||||
heightIndex map[uint64]map[snapshotKey]bool
|
||||
peerIndex map[p2p.ID]map[snapshotKey]bool
|
||||
|
||||
// blacklists for rejected items
|
||||
formatBlacklist map[uint32]bool
|
||||
peerBlacklist map[p2p.ID]bool
|
||||
snapshotBlacklist map[snapshotKey]bool
|
||||
}
|
||||
|
||||
// newSnapshotPool creates a new snapshot pool. The state source is used for
|
||||
func newSnapshotPool(stateProvider StateProvider) *snapshotPool {
|
||||
return &snapshotPool{
|
||||
stateProvider: stateProvider,
|
||||
snapshots: make(map[snapshotKey]*snapshot),
|
||||
snapshotPeers: make(map[snapshotKey]map[p2p.ID]p2p.Peer),
|
||||
formatIndex: make(map[uint32]map[snapshotKey]bool),
|
||||
heightIndex: make(map[uint64]map[snapshotKey]bool),
|
||||
peerIndex: make(map[p2p.ID]map[snapshotKey]bool),
|
||||
formatBlacklist: make(map[uint32]bool),
|
||||
peerBlacklist: make(map[p2p.ID]bool),
|
||||
snapshotBlacklist: make(map[snapshotKey]bool),
|
||||
}
|
||||
}
|
||||
|
||||
// Add adds a snapshot to the pool, unless the peer has already sent recentSnapshots snapshots. It
|
||||
// returns true if this was a new, non-blacklisted snapshot. The snapshot height is verified using
|
||||
// the light client, and the expected app hash is set for the snapshot.
|
||||
func (p *snapshotPool) Add(peer p2p.Peer, snapshot *snapshot) (bool, error) {
|
||||
appHash, err := p.stateProvider.AppHash(snapshot.Height)
|
||||
if err != nil {
|
||||
return false, err
|
||||
}
|
||||
snapshot.trustedAppHash = appHash
|
||||
key := snapshot.Key()
|
||||
|
||||
p.Lock()
|
||||
defer p.Unlock()
|
||||
|
||||
switch {
|
||||
case p.formatBlacklist[snapshot.Format]:
|
||||
return false, nil
|
||||
case p.peerBlacklist[peer.ID()]:
|
||||
return false, nil
|
||||
case p.snapshotBlacklist[key]:
|
||||
return false, nil
|
||||
case len(p.peerIndex[peer.ID()]) >= recentSnapshots:
|
||||
return false, nil
|
||||
}
|
||||
|
||||
if p.snapshotPeers[key] == nil {
|
||||
p.snapshotPeers[key] = make(map[p2p.ID]p2p.Peer)
|
||||
}
|
||||
p.snapshotPeers[key][peer.ID()] = peer
|
||||
|
||||
if p.peerIndex[peer.ID()] == nil {
|
||||
p.peerIndex[peer.ID()] = make(map[snapshotKey]bool)
|
||||
}
|
||||
p.peerIndex[peer.ID()][key] = true
|
||||
|
||||
if p.snapshots[key] != nil {
|
||||
return false, nil
|
||||
}
|
||||
p.snapshots[key] = snapshot
|
||||
|
||||
if p.formatIndex[snapshot.Format] == nil {
|
||||
p.formatIndex[snapshot.Format] = make(map[snapshotKey]bool)
|
||||
}
|
||||
p.formatIndex[snapshot.Format][key] = true
|
||||
|
||||
if p.heightIndex[snapshot.Height] == nil {
|
||||
p.heightIndex[snapshot.Height] = make(map[snapshotKey]bool)
|
||||
}
|
||||
p.heightIndex[snapshot.Height][key] = true
|
||||
|
||||
return true, nil
|
||||
}
|
||||
|
||||
// Best returns the "best" currently known snapshot, if any.
|
||||
func (p *snapshotPool) Best() *snapshot {
|
||||
ranked := p.Ranked()
|
||||
if len(ranked) == 0 {
|
||||
return nil
|
||||
}
|
||||
return ranked[0]
|
||||
}
|
||||
|
||||
// GetPeer returns a random peer for a snapshot, if any.
|
||||
func (p *snapshotPool) GetPeer(snapshot *snapshot) p2p.Peer {
|
||||
peers := p.GetPeers(snapshot)
|
||||
if len(peers) == 0 {
|
||||
return nil
|
||||
}
|
||||
return peers[rand.Intn(len(peers))]
|
||||
}
|
||||
|
||||
// GetPeers returns the peers for a snapshot.
|
||||
func (p *snapshotPool) GetPeers(snapshot *snapshot) []p2p.Peer {
|
||||
key := snapshot.Key()
|
||||
p.Lock()
|
||||
defer p.Unlock()
|
||||
|
||||
peers := make([]p2p.Peer, 0, len(p.snapshotPeers[key]))
|
||||
for _, peer := range p.snapshotPeers[key] {
|
||||
peers = append(peers, peer)
|
||||
}
|
||||
// sort results, for testability (otherwise order is random, so tests randomly fail)
|
||||
sort.Slice(peers, func(a int, b int) bool {
|
||||
return peers[a].ID() < peers[b].ID()
|
||||
})
|
||||
return peers
|
||||
}
|
||||
|
||||
// Ranked returns a list of snapshots ranked by preference. The current heuristic is very naïve,
|
||||
// preferring the snapshot with the greatest height, then greatest format, then greatest number of
|
||||
// peers. This can be improved quite a lot.
|
||||
func (p *snapshotPool) Ranked() []*snapshot {
|
||||
p.Lock()
|
||||
defer p.Unlock()
|
||||
|
||||
candidates := make([]*snapshot, 0, len(p.snapshots))
|
||||
for _, snapshot := range p.snapshots {
|
||||
candidates = append(candidates, snapshot)
|
||||
}
|
||||
|
||||
sort.Slice(candidates, func(i, j int) bool {
|
||||
a := candidates[i]
|
||||
b := candidates[j]
|
||||
|
||||
switch {
|
||||
case a.Height > b.Height:
|
||||
return true
|
||||
case a.Height < b.Height:
|
||||
return false
|
||||
case a.Format > b.Format:
|
||||
return true
|
||||
case a.Format < b.Format:
|
||||
return false
|
||||
case len(p.snapshotPeers[a.Key()]) > len(p.snapshotPeers[b.Key()]):
|
||||
return true
|
||||
default:
|
||||
return false
|
||||
}
|
||||
})
|
||||
|
||||
return candidates
|
||||
}
|
||||
|
||||
// Reject rejects a snapshot. Rejected snapshots will never be used again.
|
||||
func (p *snapshotPool) Reject(snapshot *snapshot) {
|
||||
key := snapshot.Key()
|
||||
p.Lock()
|
||||
defer p.Unlock()
|
||||
|
||||
p.snapshotBlacklist[key] = true
|
||||
p.removeSnapshot(key)
|
||||
}
|
||||
|
||||
// RejectFormat rejects a snapshot format. It will never be used again.
|
||||
func (p *snapshotPool) RejectFormat(format uint32) {
|
||||
p.Lock()
|
||||
defer p.Unlock()
|
||||
|
||||
p.formatBlacklist[format] = true
|
||||
for key := range p.formatIndex[format] {
|
||||
p.removeSnapshot(key)
|
||||
}
|
||||
}
|
||||
|
||||
// RejectPeer rejects a peer. It will never be used again.
|
||||
func (p *snapshotPool) RejectPeer(peerID p2p.ID) {
|
||||
if peerID == "" {
|
||||
return
|
||||
}
|
||||
p.Lock()
|
||||
defer p.Unlock()
|
||||
|
||||
p.removePeer(peerID)
|
||||
p.peerBlacklist[peerID] = true
|
||||
}
|
||||
|
||||
// RemovePeer removes a peer from the pool, and any snapshots that no longer have peers.
|
||||
func (p *snapshotPool) RemovePeer(peerID p2p.ID) {
|
||||
p.Lock()
|
||||
defer p.Unlock()
|
||||
p.removePeer(peerID)
|
||||
}
|
||||
|
||||
// removePeer removes a peer. The caller must hold the mutex lock.
|
||||
func (p *snapshotPool) removePeer(peerID p2p.ID) {
|
||||
for key := range p.peerIndex[peerID] {
|
||||
delete(p.snapshotPeers[key], peerID)
|
||||
if len(p.snapshotPeers[key]) == 0 {
|
||||
p.removeSnapshot(key)
|
||||
}
|
||||
}
|
||||
delete(p.peerIndex, peerID)
|
||||
}
|
||||
|
||||
// removeSnapshot removes a snapshot. The caller must hold the mutex lock.
|
||||
func (p *snapshotPool) removeSnapshot(key snapshotKey) {
|
||||
snapshot := p.snapshots[key]
|
||||
if snapshot == nil {
|
||||
return
|
||||
}
|
||||
|
||||
delete(p.snapshots, key)
|
||||
delete(p.formatIndex[snapshot.Format], key)
|
||||
delete(p.heightIndex[snapshot.Height], key)
|
||||
for peerID := range p.snapshotPeers[key] {
|
||||
delete(p.peerIndex[peerID], key)
|
||||
}
|
||||
delete(p.snapshotPeers, key)
|
||||
}
|
||||
326
statesync/snapshots_test.go
Normal file
326
statesync/snapshots_test.go
Normal file
@@ -0,0 +1,326 @@
|
||||
package statesync
|
||||
|
||||
import (
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/mock"
|
||||
"github.com/stretchr/testify/require"
|
||||
|
||||
"github.com/tendermint/tendermint/p2p"
|
||||
p2pmocks "github.com/tendermint/tendermint/p2p/mocks"
|
||||
"github.com/tendermint/tendermint/statesync/mocks"
|
||||
)
|
||||
|
||||
func TestSnapshot_Key(t *testing.T) {
|
||||
testcases := map[string]struct {
|
||||
modify func(*snapshot)
|
||||
}{
|
||||
"new height": {func(s *snapshot) { s.Height = 9 }},
|
||||
"new format": {func(s *snapshot) { s.Format = 9 }},
|
||||
"new chunk count": {func(s *snapshot) { s.Chunks = 9 }},
|
||||
"new hash": {func(s *snapshot) { s.Hash = []byte{9} }},
|
||||
"no metadata": {func(s *snapshot) { s.Metadata = nil }},
|
||||
}
|
||||
for name, tc := range testcases {
|
||||
tc := tc
|
||||
t.Run(name, func(t *testing.T) {
|
||||
s := snapshot{
|
||||
Height: 3,
|
||||
Format: 1,
|
||||
Chunks: 7,
|
||||
Hash: []byte{1, 2, 3},
|
||||
Metadata: []byte{255},
|
||||
}
|
||||
before := s.Key()
|
||||
tc.modify(&s)
|
||||
after := s.Key()
|
||||
assert.NotEqual(t, before, after)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestSnapshotPool_Add(t *testing.T) {
|
||||
stateProvider := &mocks.StateProvider{}
|
||||
stateProvider.On("AppHash", uint64(1)).Return([]byte("app_hash"), nil)
|
||||
|
||||
peer := &p2pmocks.Peer{}
|
||||
peer.On("ID").Return(p2p.ID("id"))
|
||||
|
||||
// Adding to the pool should work
|
||||
pool := newSnapshotPool(stateProvider)
|
||||
added, err := pool.Add(peer, &snapshot{
|
||||
Height: 1,
|
||||
Format: 1,
|
||||
Chunks: 1,
|
||||
Hash: []byte{1},
|
||||
})
|
||||
require.NoError(t, err)
|
||||
assert.True(t, added)
|
||||
|
||||
// Adding again from a different peer should return false
|
||||
otherPeer := &p2pmocks.Peer{}
|
||||
otherPeer.On("ID").Return(p2p.ID("other"))
|
||||
added, err = pool.Add(peer, &snapshot{
|
||||
Height: 1,
|
||||
Format: 1,
|
||||
Chunks: 1,
|
||||
Hash: []byte{1},
|
||||
})
|
||||
require.NoError(t, err)
|
||||
assert.False(t, added)
|
||||
|
||||
// The pool should have populated the snapshot with the trusted app hash
|
||||
snapshot := pool.Best()
|
||||
require.NotNil(t, snapshot)
|
||||
assert.Equal(t, []byte("app_hash"), snapshot.trustedAppHash)
|
||||
|
||||
stateProvider.AssertExpectations(t)
|
||||
}
|
||||
|
||||
func TestSnapshotPool_GetPeer(t *testing.T) {
|
||||
stateProvider := &mocks.StateProvider{}
|
||||
stateProvider.On("AppHash", mock.Anything).Return([]byte("app_hash"), nil)
|
||||
pool := newSnapshotPool(stateProvider)
|
||||
|
||||
s := &snapshot{Height: 1, Format: 1, Chunks: 1, Hash: []byte{1}}
|
||||
peerA := &p2pmocks.Peer{}
|
||||
peerA.On("ID").Return(p2p.ID("a"))
|
||||
peerB := &p2pmocks.Peer{}
|
||||
peerB.On("ID").Return(p2p.ID("b"))
|
||||
|
||||
_, err := pool.Add(peerA, s)
|
||||
require.NoError(t, err)
|
||||
_, err = pool.Add(peerB, s)
|
||||
require.NoError(t, err)
|
||||
_, err = pool.Add(peerA, &snapshot{Height: 2, Format: 1, Chunks: 1, Hash: []byte{1}})
|
||||
require.NoError(t, err)
|
||||
|
||||
// GetPeer currently picks a random peer, so lets run it until we've seen both.
|
||||
seenA := false
|
||||
seenB := false
|
||||
for !seenA || !seenB {
|
||||
peer := pool.GetPeer(s)
|
||||
switch peer.ID() {
|
||||
case p2p.ID("a"):
|
||||
seenA = true
|
||||
case p2p.ID("b"):
|
||||
seenB = true
|
||||
}
|
||||
}
|
||||
|
||||
// GetPeer should return nil for an unknown snapshot
|
||||
peer := pool.GetPeer(&snapshot{Height: 9, Format: 9})
|
||||
assert.Nil(t, peer)
|
||||
}
|
||||
|
||||
func TestSnapshotPool_GetPeers(t *testing.T) {
|
||||
stateProvider := &mocks.StateProvider{}
|
||||
stateProvider.On("AppHash", mock.Anything).Return([]byte("app_hash"), nil)
|
||||
pool := newSnapshotPool(stateProvider)
|
||||
|
||||
s := &snapshot{Height: 1, Format: 1, Chunks: 1, Hash: []byte{1}}
|
||||
peerA := &p2pmocks.Peer{}
|
||||
peerA.On("ID").Return(p2p.ID("a"))
|
||||
peerB := &p2pmocks.Peer{}
|
||||
peerB.On("ID").Return(p2p.ID("b"))
|
||||
|
||||
_, err := pool.Add(peerA, s)
|
||||
require.NoError(t, err)
|
||||
_, err = pool.Add(peerB, s)
|
||||
require.NoError(t, err)
|
||||
_, err = pool.Add(peerA, &snapshot{Height: 2, Format: 1, Chunks: 1, Hash: []byte{2}})
|
||||
require.NoError(t, err)
|
||||
|
||||
peers := pool.GetPeers(s)
|
||||
assert.Len(t, peers, 2)
|
||||
assert.EqualValues(t, "a", peers[0].ID())
|
||||
assert.EqualValues(t, "b", peers[1].ID())
|
||||
}
|
||||
|
||||
func TestSnapshotPool_Ranked_Best(t *testing.T) {
|
||||
stateProvider := &mocks.StateProvider{}
|
||||
stateProvider.On("AppHash", mock.Anything).Return([]byte("app_hash"), nil)
|
||||
pool := newSnapshotPool(stateProvider)
|
||||
|
||||
// snapshots in expected order (best to worst). Highest height wins, then highest format.
|
||||
// Snapshots with different chunk hashes are considered different, and the most peers is
|
||||
// tie-breaker.
|
||||
expectSnapshots := []struct {
|
||||
snapshot *snapshot
|
||||
peers []string
|
||||
}{
|
||||
{&snapshot{Height: 2, Format: 2, Chunks: 4, Hash: []byte{1, 3}}, []string{"a", "b", "c"}},
|
||||
{&snapshot{Height: 2, Format: 2, Chunks: 5, Hash: []byte{1, 2}}, []string{"a"}},
|
||||
{&snapshot{Height: 2, Format: 1, Chunks: 3, Hash: []byte{1, 2}}, []string{"a", "b"}},
|
||||
{&snapshot{Height: 1, Format: 2, Chunks: 5, Hash: []byte{1, 2}}, []string{"a", "b"}},
|
||||
{&snapshot{Height: 1, Format: 1, Chunks: 4, Hash: []byte{1, 2}}, []string{"a", "b", "c"}},
|
||||
}
|
||||
|
||||
// Add snapshots in reverse order, to make sure the pool enforces some order.
|
||||
for i := len(expectSnapshots) - 1; i >= 0; i-- {
|
||||
for _, peerID := range expectSnapshots[i].peers {
|
||||
peer := &p2pmocks.Peer{}
|
||||
peer.On("ID").Return(p2p.ID(peerID))
|
||||
_, err := pool.Add(peer, expectSnapshots[i].snapshot)
|
||||
require.NoError(t, err)
|
||||
}
|
||||
}
|
||||
|
||||
// Ranked should return the snapshots in the same order
|
||||
ranked := pool.Ranked()
|
||||
assert.Len(t, ranked, len(expectSnapshots))
|
||||
for i := range ranked {
|
||||
assert.Equal(t, expectSnapshots[i].snapshot, ranked[i])
|
||||
}
|
||||
|
||||
// Check that best snapshots are returned in expected order
|
||||
for i := range expectSnapshots {
|
||||
snapshot := expectSnapshots[i].snapshot
|
||||
require.Equal(t, snapshot, pool.Best())
|
||||
pool.Reject(snapshot)
|
||||
}
|
||||
assert.Nil(t, pool.Best())
|
||||
}
|
||||
|
||||
func TestSnapshotPool_Reject(t *testing.T) {
|
||||
stateProvider := &mocks.StateProvider{}
|
||||
stateProvider.On("AppHash", mock.Anything).Return([]byte("app_hash"), nil)
|
||||
pool := newSnapshotPool(stateProvider)
|
||||
peer := &p2pmocks.Peer{}
|
||||
peer.On("ID").Return(p2p.ID("id"))
|
||||
|
||||
snapshots := []*snapshot{
|
||||
{Height: 2, Format: 2, Chunks: 1, Hash: []byte{1, 2}},
|
||||
{Height: 2, Format: 1, Chunks: 1, Hash: []byte{1, 2}},
|
||||
{Height: 1, Format: 2, Chunks: 1, Hash: []byte{1, 2}},
|
||||
{Height: 1, Format: 1, Chunks: 1, Hash: []byte{1, 2}},
|
||||
}
|
||||
for _, s := range snapshots {
|
||||
_, err := pool.Add(peer, s)
|
||||
require.NoError(t, err)
|
||||
}
|
||||
|
||||
pool.Reject(snapshots[0])
|
||||
assert.Equal(t, snapshots[1:], pool.Ranked())
|
||||
|
||||
added, err := pool.Add(peer, snapshots[0])
|
||||
require.NoError(t, err)
|
||||
assert.False(t, added)
|
||||
|
||||
added, err = pool.Add(peer, &snapshot{Height: 3, Format: 3, Chunks: 1, Hash: []byte{1}})
|
||||
require.NoError(t, err)
|
||||
assert.True(t, added)
|
||||
}
|
||||
|
||||
// nolint: dupl
|
||||
func TestSnapshotPool_RejectFormat(t *testing.T) {
|
||||
stateProvider := &mocks.StateProvider{}
|
||||
stateProvider.On("AppHash", mock.Anything).Return([]byte("app_hash"), nil)
|
||||
pool := newSnapshotPool(stateProvider)
|
||||
peer := &p2pmocks.Peer{}
|
||||
peer.On("ID").Return(p2p.ID("id"))
|
||||
|
||||
snapshots := []*snapshot{
|
||||
{Height: 2, Format: 2, Chunks: 1, Hash: []byte{1, 2}},
|
||||
{Height: 2, Format: 1, Chunks: 1, Hash: []byte{1, 2}},
|
||||
{Height: 1, Format: 2, Chunks: 1, Hash: []byte{1, 2}},
|
||||
{Height: 1, Format: 1, Chunks: 1, Hash: []byte{1, 2}},
|
||||
}
|
||||
for _, s := range snapshots {
|
||||
_, err := pool.Add(peer, s)
|
||||
require.NoError(t, err)
|
||||
}
|
||||
|
||||
pool.RejectFormat(1)
|
||||
assert.Equal(t, []*snapshot{snapshots[0], snapshots[2]}, pool.Ranked())
|
||||
|
||||
added, err := pool.Add(peer, &snapshot{Height: 3, Format: 1, Chunks: 1, Hash: []byte{1}})
|
||||
require.NoError(t, err)
|
||||
assert.False(t, added)
|
||||
assert.Equal(t, []*snapshot{snapshots[0], snapshots[2]}, pool.Ranked())
|
||||
|
||||
added, err = pool.Add(peer, &snapshot{Height: 3, Format: 3, Chunks: 1, Hash: []byte{1}})
|
||||
require.NoError(t, err)
|
||||
assert.True(t, added)
|
||||
}
|
||||
|
||||
func TestSnapshotPool_RejectPeer(t *testing.T) {
|
||||
stateProvider := &mocks.StateProvider{}
|
||||
stateProvider.On("AppHash", mock.Anything).Return([]byte("app_hash"), nil)
|
||||
pool := newSnapshotPool(stateProvider)
|
||||
|
||||
peerA := &p2pmocks.Peer{}
|
||||
peerA.On("ID").Return(p2p.ID("a"))
|
||||
peerB := &p2pmocks.Peer{}
|
||||
peerB.On("ID").Return(p2p.ID("b"))
|
||||
|
||||
s1 := &snapshot{Height: 1, Format: 1, Chunks: 1, Hash: []byte{1}}
|
||||
s2 := &snapshot{Height: 2, Format: 1, Chunks: 1, Hash: []byte{2}}
|
||||
s3 := &snapshot{Height: 3, Format: 1, Chunks: 1, Hash: []byte{2}}
|
||||
|
||||
_, err := pool.Add(peerA, s1)
|
||||
require.NoError(t, err)
|
||||
_, err = pool.Add(peerA, s2)
|
||||
require.NoError(t, err)
|
||||
|
||||
_, err = pool.Add(peerB, s2)
|
||||
require.NoError(t, err)
|
||||
_, err = pool.Add(peerB, s3)
|
||||
require.NoError(t, err)
|
||||
|
||||
pool.RejectPeer(peerA.ID())
|
||||
|
||||
assert.Empty(t, pool.GetPeers(s1))
|
||||
|
||||
peers2 := pool.GetPeers(s2)
|
||||
assert.Len(t, peers2, 1)
|
||||
assert.EqualValues(t, "b", peers2[0].ID())
|
||||
|
||||
peers3 := pool.GetPeers(s2)
|
||||
assert.Len(t, peers3, 1)
|
||||
assert.EqualValues(t, "b", peers3[0].ID())
|
||||
|
||||
// it should no longer be possible to add the peer back
|
||||
_, err = pool.Add(peerA, s1)
|
||||
require.NoError(t, err)
|
||||
assert.Empty(t, pool.GetPeers(s1))
|
||||
}
|
||||
|
||||
func TestSnapshotPool_RemovePeer(t *testing.T) {
|
||||
stateProvider := &mocks.StateProvider{}
|
||||
stateProvider.On("AppHash", mock.Anything).Return([]byte("app_hash"), nil)
|
||||
pool := newSnapshotPool(stateProvider)
|
||||
|
||||
peerA := &p2pmocks.Peer{}
|
||||
peerA.On("ID").Return(p2p.ID("a"))
|
||||
peerB := &p2pmocks.Peer{}
|
||||
peerB.On("ID").Return(p2p.ID("b"))
|
||||
|
||||
s1 := &snapshot{Height: 1, Format: 1, Chunks: 1, Hash: []byte{1}}
|
||||
s2 := &snapshot{Height: 2, Format: 1, Chunks: 1, Hash: []byte{2}}
|
||||
|
||||
_, err := pool.Add(peerA, s1)
|
||||
require.NoError(t, err)
|
||||
_, err = pool.Add(peerA, s2)
|
||||
require.NoError(t, err)
|
||||
_, err = pool.Add(peerB, s1)
|
||||
require.NoError(t, err)
|
||||
|
||||
pool.RemovePeer(peerA.ID())
|
||||
|
||||
peers1 := pool.GetPeers(s1)
|
||||
assert.Len(t, peers1, 1)
|
||||
assert.EqualValues(t, "b", peers1[0].ID())
|
||||
|
||||
peers2 := pool.GetPeers(s2)
|
||||
assert.Empty(t, peers2)
|
||||
|
||||
// it should still be possible to add the peer back
|
||||
_, err = pool.Add(peerA, s1)
|
||||
require.NoError(t, err)
|
||||
peers1 = pool.GetPeers(s1)
|
||||
assert.Len(t, peers1, 2)
|
||||
assert.EqualValues(t, "a", peers1[0].ID())
|
||||
assert.EqualValues(t, "b", peers1[1].ID())
|
||||
}
|
||||
179
statesync/stateprovider.go
Normal file
179
statesync/stateprovider.go
Normal file
@@ -0,0 +1,179 @@
|
||||
package statesync
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"strings"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
dbm "github.com/tendermint/tm-db"
|
||||
|
||||
"github.com/tendermint/tendermint/libs/log"
|
||||
lite "github.com/tendermint/tendermint/lite2"
|
||||
liteprovider "github.com/tendermint/tendermint/lite2/provider"
|
||||
litehttp "github.com/tendermint/tendermint/lite2/provider/http"
|
||||
literpc "github.com/tendermint/tendermint/lite2/rpc"
|
||||
litedb "github.com/tendermint/tendermint/lite2/store/db"
|
||||
rpchttp "github.com/tendermint/tendermint/rpc/client/http"
|
||||
sm "github.com/tendermint/tendermint/state"
|
||||
"github.com/tendermint/tendermint/types"
|
||||
)
|
||||
|
||||
//go:generate mockery -case underscore -name StateProvider
|
||||
|
||||
// StateProvider is a provider of trusted state data for bootstrapping a node. This refers
|
||||
// to the state.State object, not the state machine.
|
||||
type StateProvider interface {
|
||||
// AppHash returns the app hash after the given height has been committed.
|
||||
AppHash(height uint64) ([]byte, error)
|
||||
// Commit returns the commit at the given height.
|
||||
Commit(height uint64) (*types.Commit, error)
|
||||
// State returns a state object at the given height.
|
||||
State(height uint64) (sm.State, error)
|
||||
}
|
||||
|
||||
// lightClientStateProvider is a state provider using the light client.
|
||||
type lightClientStateProvider struct {
|
||||
sync.Mutex // lite.Client is not concurrency-safe
|
||||
lc *lite.Client
|
||||
version sm.Version
|
||||
providers map[liteprovider.Provider]string
|
||||
}
|
||||
|
||||
// NewLightClientStateProvider creates a new StateProvider using a light client and RPC clients.
|
||||
func NewLightClientStateProvider(
|
||||
chainID string,
|
||||
version sm.Version,
|
||||
servers []string,
|
||||
trustOptions lite.TrustOptions,
|
||||
logger log.Logger,
|
||||
) (StateProvider, error) {
|
||||
if len(servers) < 2 {
|
||||
return nil, fmt.Errorf("at least 2 RPC servers are required, got %v", len(servers))
|
||||
}
|
||||
|
||||
providers := make([]liteprovider.Provider, 0, len(servers))
|
||||
providerRemotes := make(map[liteprovider.Provider]string)
|
||||
for _, server := range servers {
|
||||
client, err := rpcClient(server)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("failed to set up RPC client: %w", err)
|
||||
}
|
||||
provider := litehttp.NewWithClient(chainID, client)
|
||||
providers = append(providers, provider)
|
||||
// We store the RPC addresses keyed by provider, so we can find the address of the primary
|
||||
// provider used by the light client and use it to fetch consensus parameters.
|
||||
providerRemotes[provider] = server
|
||||
}
|
||||
|
||||
lc, err := lite.NewClient(chainID, trustOptions, providers[0], providers[1:],
|
||||
litedb.New(dbm.NewMemDB(), ""), lite.Logger(logger), lite.MaxRetryAttempts(5))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return &lightClientStateProvider{
|
||||
lc: lc,
|
||||
version: version,
|
||||
providers: providerRemotes,
|
||||
}, nil
|
||||
}
|
||||
|
||||
// AppHash implements StateProvider.
|
||||
func (s *lightClientStateProvider) AppHash(height uint64) ([]byte, error) {
|
||||
s.Lock()
|
||||
defer s.Unlock()
|
||||
|
||||
// We have to fetch the next height, which contains the app hash for the previous height.
|
||||
header, err := s.lc.VerifyHeaderAtHeight(int64(height+1), time.Now())
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return header.AppHash, nil
|
||||
}
|
||||
|
||||
// Commit implements StateProvider.
|
||||
func (s *lightClientStateProvider) Commit(height uint64) (*types.Commit, error) {
|
||||
s.Lock()
|
||||
defer s.Unlock()
|
||||
header, err := s.lc.VerifyHeaderAtHeight(int64(height), time.Now())
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return header.Commit, nil
|
||||
}
|
||||
|
||||
// State implements StateProvider.
|
||||
func (s *lightClientStateProvider) State(height uint64) (sm.State, error) {
|
||||
s.Lock()
|
||||
defer s.Unlock()
|
||||
|
||||
state := sm.State{
|
||||
ChainID: s.lc.ChainID(),
|
||||
Version: s.version,
|
||||
}
|
||||
|
||||
// We need to verify up until h+2, to get the validator set. This also prefetches the headers
|
||||
// for h and h+1 in the typical case where the trusted header is after the snapshot height.
|
||||
_, err := s.lc.VerifyHeaderAtHeight(int64(height+2), time.Now())
|
||||
if err != nil {
|
||||
return sm.State{}, err
|
||||
}
|
||||
header, err := s.lc.VerifyHeaderAtHeight(int64(height), time.Now())
|
||||
if err != nil {
|
||||
return sm.State{}, err
|
||||
}
|
||||
nextHeader, err := s.lc.VerifyHeaderAtHeight(int64(height+1), time.Now())
|
||||
if err != nil {
|
||||
return sm.State{}, err
|
||||
}
|
||||
state.LastBlockHeight = header.Height
|
||||
state.LastBlockTime = header.Time
|
||||
state.LastBlockID = header.Commit.BlockID
|
||||
state.AppHash = nextHeader.AppHash
|
||||
state.LastResultsHash = nextHeader.LastResultsHash
|
||||
|
||||
state.LastValidators, _, err = s.lc.TrustedValidatorSet(int64(height))
|
||||
if err != nil {
|
||||
return sm.State{}, err
|
||||
}
|
||||
state.Validators, _, err = s.lc.TrustedValidatorSet(int64(height + 1))
|
||||
if err != nil {
|
||||
return sm.State{}, err
|
||||
}
|
||||
state.NextValidators, _, err = s.lc.TrustedValidatorSet(int64(height + 2))
|
||||
if err != nil {
|
||||
return sm.State{}, err
|
||||
}
|
||||
state.LastHeightValidatorsChanged = int64(height)
|
||||
|
||||
// We'll also need to fetch consensus params via RPC, using light client verification.
|
||||
primaryURL, ok := s.providers[s.lc.Primary()]
|
||||
if !ok || primaryURL == "" {
|
||||
return sm.State{}, fmt.Errorf("could not find address for primary light client provider")
|
||||
}
|
||||
primaryRPC, err := rpcClient(primaryURL)
|
||||
if err != nil {
|
||||
return sm.State{}, fmt.Errorf("unable to create RPC client: %w", err)
|
||||
}
|
||||
rpcclient := literpc.NewClient(primaryRPC, s.lc)
|
||||
result, err := rpcclient.ConsensusParams(&nextHeader.Height)
|
||||
if err != nil {
|
||||
return sm.State{}, fmt.Errorf("unable to fetch consensus parameters for height %v: %w",
|
||||
nextHeader.Height, err)
|
||||
}
|
||||
state.ConsensusParams = result.ConsensusParams
|
||||
|
||||
return state, nil
|
||||
}
|
||||
|
||||
// rpcClient sets up a new RPC client
|
||||
func rpcClient(server string) (*rpchttp.HTTP, error) {
|
||||
if !strings.Contains(server, "://") {
|
||||
server = "http://" + server
|
||||
}
|
||||
c, err := rpchttp.New(server, "/websocket")
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return c, nil
|
||||
}
|
||||
442
statesync/syncer.go
Normal file
442
statesync/syncer.go
Normal file
@@ -0,0 +1,442 @@
|
||||
package statesync
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"errors"
|
||||
"fmt"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
abci "github.com/tendermint/tendermint/abci/types"
|
||||
"github.com/tendermint/tendermint/libs/log"
|
||||
"github.com/tendermint/tendermint/p2p"
|
||||
"github.com/tendermint/tendermint/proxy"
|
||||
sm "github.com/tendermint/tendermint/state"
|
||||
"github.com/tendermint/tendermint/types"
|
||||
"github.com/tendermint/tendermint/version"
|
||||
)
|
||||
|
||||
const (
|
||||
// defaultDiscoveryTime is the time to spend discovering snapshots.
|
||||
defaultDiscoveryTime = 20 * time.Second
|
||||
// chunkFetchers is the number of concurrent chunk fetchers to run.
|
||||
chunkFetchers = 4
|
||||
// chunkTimeout is the timeout while waiting for the next chunk from the chunk queue.
|
||||
chunkTimeout = 2 * time.Minute
|
||||
// requestTimeout is the timeout before rerequesting a chunk, possibly from a different peer.
|
||||
chunkRequestTimeout = 10 * time.Second
|
||||
)
|
||||
|
||||
var (
|
||||
// errAbort is returned by Sync() when snapshot restoration is aborted.
|
||||
errAbort = errors.New("state sync aborted")
|
||||
// errRetrySnapshot is returned by Sync() when the snapshot should be retried.
|
||||
errRetrySnapshot = errors.New("retry snapshot")
|
||||
// errRejectSnapshot is returned by Sync() when the snapshot is rejected.
|
||||
errRejectSnapshot = errors.New("snapshot was rejected")
|
||||
// errRejectFormat is returned by Sync() when the snapshot format is rejected.
|
||||
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 = 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")
|
||||
)
|
||||
|
||||
// syncer runs a state sync against an ABCI app. Use either SyncAny() to automatically attempt to
|
||||
// sync all snapshots in the pool (pausing to discover new ones), or Sync() to sync a specific
|
||||
// snapshot. Snapshots and chunks are fed via AddSnapshot() and AddChunk() as appropriate.
|
||||
type syncer struct {
|
||||
logger log.Logger
|
||||
stateProvider StateProvider
|
||||
conn proxy.AppConnSnapshot
|
||||
connQuery proxy.AppConnQuery
|
||||
snapshots *snapshotPool
|
||||
tempDir string
|
||||
|
||||
mtx sync.RWMutex
|
||||
chunks *chunkQueue
|
||||
}
|
||||
|
||||
// newSyncer creates a new syncer.
|
||||
func newSyncer(logger log.Logger, conn proxy.AppConnSnapshot, connQuery proxy.AppConnQuery,
|
||||
stateProvider StateProvider, tempDir string) *syncer {
|
||||
return &syncer{
|
||||
logger: logger,
|
||||
stateProvider: stateProvider,
|
||||
conn: conn,
|
||||
connQuery: connQuery,
|
||||
snapshots: newSnapshotPool(stateProvider),
|
||||
tempDir: tempDir,
|
||||
}
|
||||
}
|
||||
|
||||
// AddChunk adds a chunk to the chunk queue, if any. It returns false if the chunk has already
|
||||
// been added to the queue, or an error if there's no sync in progress.
|
||||
func (s *syncer) AddChunk(chunk *chunk) (bool, error) {
|
||||
s.mtx.RLock()
|
||||
defer s.mtx.RUnlock()
|
||||
if s.chunks == nil {
|
||||
return false, errors.New("no state sync in progress")
|
||||
}
|
||||
added, err := s.chunks.Add(chunk)
|
||||
if err != nil {
|
||||
return false, err
|
||||
}
|
||||
if added {
|
||||
s.logger.Debug("Added chunk to queue", "height", chunk.Height, "format", chunk.Format,
|
||||
"chunk", chunk.Index)
|
||||
} else {
|
||||
s.logger.Debug("Ignoring duplicate chunk in queue", "height", chunk.Height, "format", chunk.Format,
|
||||
"chunk", chunk.Index)
|
||||
}
|
||||
return added, nil
|
||||
}
|
||||
|
||||
// AddSnapshot adds a snapshot to the snapshot pool. It returns true if a new, previously unseen
|
||||
// snapshot was accepted and added.
|
||||
func (s *syncer) AddSnapshot(peer p2p.Peer, snapshot *snapshot) (bool, error) {
|
||||
added, err := s.snapshots.Add(peer, snapshot)
|
||||
if err != nil {
|
||||
return false, err
|
||||
}
|
||||
if added {
|
||||
s.logger.Info("Discovered new snapshot", "height", snapshot.Height, "format", snapshot.Format,
|
||||
"hash", fmt.Sprintf("%X", snapshot.Hash))
|
||||
}
|
||||
return added, nil
|
||||
}
|
||||
|
||||
// AddPeer adds a peer to the pool. For now we just keep it simple and send a single request
|
||||
// to discover snapshots, later we may want to do retries and stuff.
|
||||
func (s *syncer) AddPeer(peer p2p.Peer) {
|
||||
s.logger.Debug("Requesting snapshots from peer", "peer", peer.ID())
|
||||
peer.Send(SnapshotChannel, cdc.MustMarshalBinaryBare(&snapshotsRequestMessage{}))
|
||||
}
|
||||
|
||||
// RemovePeer removes a peer from the pool.
|
||||
func (s *syncer) RemovePeer(peer p2p.Peer) {
|
||||
s.logger.Debug("Removing peer from sync", "peer", peer.ID())
|
||||
s.snapshots.RemovePeer(peer.ID())
|
||||
}
|
||||
|
||||
// SyncAny tries to sync any of the snapshots in the snapshot pool, waiting to discover further
|
||||
// snapshots if none were found and discoveryTime > 0. It returns the latest state and block commit
|
||||
// which the caller must use to bootstrap the node.
|
||||
func (s *syncer) SyncAny(discoveryTime time.Duration) (sm.State, *types.Commit, error) {
|
||||
if discoveryTime > 0 {
|
||||
s.logger.Info(fmt.Sprintf("Discovering snapshots for %v", discoveryTime))
|
||||
time.Sleep(discoveryTime)
|
||||
}
|
||||
|
||||
// The app may ask us to retry a snapshot restoration, in which case we need to reuse
|
||||
// the snapshot and chunk queue from the previous loop iteration.
|
||||
var (
|
||||
snapshot *snapshot
|
||||
chunks *chunkQueue
|
||||
err error
|
||||
)
|
||||
for {
|
||||
// If not nil, we're going to retry restoration of the same snapshot.
|
||||
if snapshot == nil {
|
||||
snapshot = s.snapshots.Best()
|
||||
chunks = nil
|
||||
}
|
||||
if snapshot == nil {
|
||||
if discoveryTime == 0 {
|
||||
return sm.State{}, nil, errNoSnapshots
|
||||
}
|
||||
s.logger.Info(fmt.Sprintf("Discovering snapshots for %v", discoveryTime))
|
||||
time.Sleep(discoveryTime)
|
||||
continue
|
||||
}
|
||||
if chunks == nil {
|
||||
chunks, err = newChunkQueue(snapshot, s.tempDir)
|
||||
if err != nil {
|
||||
return sm.State{}, nil, fmt.Errorf("failed to create chunk queue: %w", err)
|
||||
}
|
||||
defer chunks.Close() // in case we forget to close it elsewhere
|
||||
}
|
||||
|
||||
newState, commit, err := s.Sync(snapshot, chunks)
|
||||
switch {
|
||||
case err == nil:
|
||||
return newState, commit, nil
|
||||
|
||||
case errors.Is(err, errAbort):
|
||||
return sm.State{}, nil, err
|
||||
|
||||
case errors.Is(err, errRetrySnapshot):
|
||||
chunks.RetryAll()
|
||||
s.logger.Info("Retrying snapshot", "height", snapshot.Height, "format", snapshot.Format,
|
||||
"hash", fmt.Sprintf("%X", snapshot.Hash))
|
||||
continue
|
||||
|
||||
case errors.Is(err, errTimeout):
|
||||
s.snapshots.Reject(snapshot)
|
||||
s.logger.Error("Timed out waiting for snapshot chunks, rejected snapshot",
|
||||
"height", snapshot.Height, "format", snapshot.Format, "hash", fmt.Sprintf("%X", snapshot.Hash))
|
||||
|
||||
case errors.Is(err, errRejectSnapshot):
|
||||
s.snapshots.Reject(snapshot)
|
||||
s.logger.Info("Snapshot rejected", "height", snapshot.Height, "format", snapshot.Format,
|
||||
"hash", fmt.Sprintf("%X", snapshot.Hash))
|
||||
|
||||
case errors.Is(err, errRejectFormat):
|
||||
s.snapshots.RejectFormat(snapshot.Format)
|
||||
s.logger.Info("Snapshot format rejected", "format", snapshot.Format)
|
||||
|
||||
case errors.Is(err, errRejectSender):
|
||||
s.logger.Info("Snapshot senders rejected", "height", snapshot.Height, "format", snapshot.Format,
|
||||
"hash", fmt.Sprintf("%X", snapshot.Hash))
|
||||
for _, peer := range s.snapshots.GetPeers(snapshot) {
|
||||
s.snapshots.RejectPeer(peer.ID())
|
||||
s.logger.Info("Snapshot sender rejected", "peer", peer.ID())
|
||||
}
|
||||
|
||||
default:
|
||||
return sm.State{}, nil, fmt.Errorf("snapshot restoration failed: %w", err)
|
||||
}
|
||||
|
||||
// Discard snapshot and chunks for next iteration
|
||||
err = chunks.Close()
|
||||
if err != nil {
|
||||
s.logger.Error("Failed to clean up chunk queue", "err", err)
|
||||
}
|
||||
snapshot = nil
|
||||
chunks = nil
|
||||
}
|
||||
}
|
||||
|
||||
// Sync executes a sync for a specific snapshot, returning the latest state and block commit which
|
||||
// the caller must use to bootstrap the node.
|
||||
func (s *syncer) Sync(snapshot *snapshot, chunks *chunkQueue) (sm.State, *types.Commit, error) {
|
||||
s.mtx.Lock()
|
||||
if s.chunks != nil {
|
||||
s.mtx.Unlock()
|
||||
return sm.State{}, nil, errors.New("a state sync is already in progress")
|
||||
}
|
||||
s.chunks = chunks
|
||||
s.mtx.Unlock()
|
||||
defer func() {
|
||||
s.mtx.Lock()
|
||||
s.chunks = nil
|
||||
s.mtx.Unlock()
|
||||
}()
|
||||
|
||||
// Offer snapshot to ABCI app.
|
||||
err := s.offerSnapshot(snapshot)
|
||||
if err != nil {
|
||||
return sm.State{}, nil, err
|
||||
}
|
||||
|
||||
// Spawn chunk fetchers. They will terminate when the chunk queue is closed or context cancelled.
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
for i := int32(0); i < chunkFetchers; i++ {
|
||||
go s.fetchChunks(ctx, snapshot, chunks)
|
||||
}
|
||||
|
||||
// Optimistically build new state, so we don't discover any light client failures at the end.
|
||||
state, err := s.stateProvider.State(snapshot.Height)
|
||||
if err != nil {
|
||||
return sm.State{}, nil, fmt.Errorf("failed to build new state: %w", err)
|
||||
}
|
||||
commit, err := s.stateProvider.Commit(snapshot.Height)
|
||||
if err != nil {
|
||||
return sm.State{}, nil, fmt.Errorf("failed to fetch commit: %w", err)
|
||||
}
|
||||
|
||||
// Restore snapshot
|
||||
err = s.applyChunks(chunks)
|
||||
if err != nil {
|
||||
return sm.State{}, nil, err
|
||||
}
|
||||
|
||||
// Verify app and update app version
|
||||
appVersion, err := s.verifyApp(snapshot)
|
||||
if err != nil {
|
||||
return sm.State{}, nil, err
|
||||
}
|
||||
state.Version.Consensus.App = version.Protocol(appVersion)
|
||||
|
||||
// Done! 🎉
|
||||
s.logger.Info("Snapshot restored", "height", snapshot.Height, "format", snapshot.Format,
|
||||
"hash", fmt.Sprintf("%X", snapshot.Hash))
|
||||
|
||||
return state, commit, nil
|
||||
}
|
||||
|
||||
// offerSnapshot offers a snapshot to the app. It returns various errors depending on the app's
|
||||
// response, or nil if the snapshot was accepted.
|
||||
func (s *syncer) offerSnapshot(snapshot *snapshot) error {
|
||||
s.logger.Info("Offering snapshot to ABCI app", "height", snapshot.Height,
|
||||
"format", snapshot.Format, "hash", fmt.Sprintf("%X", snapshot.Hash))
|
||||
resp, err := s.conn.OfferSnapshotSync(abci.RequestOfferSnapshot{
|
||||
Snapshot: &abci.Snapshot{
|
||||
Height: snapshot.Height,
|
||||
Format: snapshot.Format,
|
||||
Chunks: snapshot.Chunks,
|
||||
Hash: snapshot.Hash,
|
||||
Metadata: snapshot.Metadata,
|
||||
},
|
||||
AppHash: snapshot.trustedAppHash,
|
||||
})
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to offer snapshot: %w", err)
|
||||
}
|
||||
switch resp.Result {
|
||||
case abci.ResponseOfferSnapshot_accept:
|
||||
s.logger.Info("Snapshot accepted, restoring", "height", snapshot.Height,
|
||||
"format", snapshot.Format, "hash", fmt.Sprintf("%X", snapshot.Hash))
|
||||
return nil
|
||||
case abci.ResponseOfferSnapshot_abort:
|
||||
return errAbort
|
||||
case abci.ResponseOfferSnapshot_reject:
|
||||
return errRejectSnapshot
|
||||
case abci.ResponseOfferSnapshot_reject_format:
|
||||
return errRejectFormat
|
||||
case abci.ResponseOfferSnapshot_reject_sender:
|
||||
return errRejectSender
|
||||
default:
|
||||
return fmt.Errorf("invalid ResponseOfferSnapshot result %v", resp.Result)
|
||||
}
|
||||
}
|
||||
|
||||
// applyChunks applies chunks to the app. It returns various errors depending on the app's
|
||||
// response, or nil once the snapshot is fully restored.
|
||||
func (s *syncer) applyChunks(chunks *chunkQueue) error {
|
||||
for {
|
||||
chunk, err := chunks.Next()
|
||||
if err == errDone {
|
||||
return nil
|
||||
} else if err != nil {
|
||||
return fmt.Errorf("failed to fetch chunk: %w", err)
|
||||
}
|
||||
|
||||
resp, err := s.conn.ApplySnapshotChunkSync(abci.RequestApplySnapshotChunk{
|
||||
Index: chunk.Index,
|
||||
Chunk: chunk.Chunk,
|
||||
Sender: string(chunk.Sender),
|
||||
})
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to apply chunk %v: %w", chunk.Index, err)
|
||||
}
|
||||
s.logger.Info("Applied snapshot chunk to ABCI app", "height", chunk.Height,
|
||||
"format", chunk.Format, "chunk", chunk.Index, "total", chunks.Size())
|
||||
|
||||
// Discard and refetch any chunks as requested by the app
|
||||
for _, index := range resp.RefetchChunks {
|
||||
err := chunks.Discard(index)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to discard chunk %v: %w", index, err)
|
||||
}
|
||||
}
|
||||
|
||||
// Reject any senders as requested by the app
|
||||
for _, sender := range resp.RejectSenders {
|
||||
if sender != "" {
|
||||
s.snapshots.RejectPeer(p2p.ID(sender))
|
||||
err := chunks.DiscardSender(p2p.ID(sender))
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to reject sender: %w", err)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
switch resp.Result {
|
||||
case abci.ResponseApplySnapshotChunk_accept:
|
||||
case abci.ResponseApplySnapshotChunk_abort:
|
||||
return errAbort
|
||||
case abci.ResponseApplySnapshotChunk_retry:
|
||||
chunks.Retry(chunk.Index)
|
||||
case abci.ResponseApplySnapshotChunk_retry_snapshot:
|
||||
return errRetrySnapshot
|
||||
case abci.ResponseApplySnapshotChunk_reject_snapshot:
|
||||
return errRejectSnapshot
|
||||
default:
|
||||
return fmt.Errorf("unknown ResponseApplySnapshotChunk result %v", resp.Result)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// fetchChunks requests chunks from peers, receiving allocations from the chunk queue. Chunks
|
||||
// will be received from the reactor via syncer.AddChunks() to chunkQueue.Add().
|
||||
func (s *syncer) fetchChunks(ctx context.Context, snapshot *snapshot, chunks *chunkQueue) {
|
||||
for {
|
||||
index, err := chunks.Allocate()
|
||||
if err == errDone {
|
||||
// Keep checking until the context is cancelled (restore is done), in case any
|
||||
// chunks need to be refetched.
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return
|
||||
default:
|
||||
}
|
||||
time.Sleep(2 * time.Second)
|
||||
continue
|
||||
}
|
||||
if err != nil {
|
||||
s.logger.Error("Failed to allocate chunk from queue", "err", err)
|
||||
return
|
||||
}
|
||||
s.logger.Info("Fetching snapshot chunk", "height", snapshot.Height,
|
||||
"format", snapshot.Format, "chunk", index, "total", chunks.Size())
|
||||
|
||||
ticker := time.NewTicker(chunkRequestTimeout)
|
||||
defer ticker.Stop()
|
||||
s.requestChunk(snapshot, index)
|
||||
select {
|
||||
case <-chunks.WaitFor(index):
|
||||
case <-ticker.C:
|
||||
s.requestChunk(snapshot, index)
|
||||
case <-ctx.Done():
|
||||
return
|
||||
}
|
||||
ticker.Stop()
|
||||
}
|
||||
}
|
||||
|
||||
// requestChunk requests a chunk from a peer.
|
||||
func (s *syncer) requestChunk(snapshot *snapshot, chunk uint32) {
|
||||
peer := s.snapshots.GetPeer(snapshot)
|
||||
if peer == nil {
|
||||
s.logger.Error("No valid peers found for snapshot", "height", snapshot.Height,
|
||||
"format", snapshot.Format, "hash", snapshot.Hash)
|
||||
return
|
||||
}
|
||||
s.logger.Debug("Requesting snapshot chunk", "height", snapshot.Height,
|
||||
"format", snapshot.Format, "chunk", chunk, "peer", peer.ID())
|
||||
peer.Send(ChunkChannel, cdc.MustMarshalBinaryBare(&chunkRequestMessage{
|
||||
Height: snapshot.Height,
|
||||
Format: snapshot.Format,
|
||||
Index: chunk,
|
||||
}))
|
||||
}
|
||||
|
||||
// verifyApp verifies the sync, checking the app hash and last block height. It returns the
|
||||
// app version, which should be returned as part of the initial state.
|
||||
func (s *syncer) verifyApp(snapshot *snapshot) (uint64, error) {
|
||||
resp, err := s.connQuery.InfoSync(proxy.RequestInfo)
|
||||
if err != nil {
|
||||
return 0, fmt.Errorf("failed to query ABCI app for appHash: %w", err)
|
||||
}
|
||||
if !bytes.Equal(snapshot.trustedAppHash, resp.LastBlockAppHash) {
|
||||
s.logger.Error("appHash verification failed",
|
||||
"expected", fmt.Sprintf("%X", snapshot.trustedAppHash),
|
||||
"actual", fmt.Sprintf("%X", resp.LastBlockAppHash))
|
||||
return 0, errVerifyFailed
|
||||
}
|
||||
if uint64(resp.LastBlockHeight) != snapshot.Height {
|
||||
s.logger.Error("ABCI app reported unexpected last block height",
|
||||
"expected", snapshot.Height, "actual", resp.LastBlockHeight)
|
||||
return 0, errVerifyFailed
|
||||
}
|
||||
s.logger.Info("Verified ABCI app", "height", snapshot.Height,
|
||||
"appHash", fmt.Sprintf("%X", snapshot.trustedAppHash))
|
||||
return resp.AppVersion, nil
|
||||
}
|
||||
639
statesync/syncer_test.go
Normal file
639
statesync/syncer_test.go
Normal file
@@ -0,0 +1,639 @@
|
||||
package statesync
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"sync"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/mock"
|
||||
"github.com/stretchr/testify/require"
|
||||
|
||||
abci "github.com/tendermint/tendermint/abci/types"
|
||||
"github.com/tendermint/tendermint/libs/log"
|
||||
"github.com/tendermint/tendermint/p2p"
|
||||
p2pmocks "github.com/tendermint/tendermint/p2p/mocks"
|
||||
"github.com/tendermint/tendermint/proxy"
|
||||
proxymocks "github.com/tendermint/tendermint/proxy/mocks"
|
||||
sm "github.com/tendermint/tendermint/state"
|
||||
"github.com/tendermint/tendermint/statesync/mocks"
|
||||
"github.com/tendermint/tendermint/types"
|
||||
"github.com/tendermint/tendermint/version"
|
||||
)
|
||||
|
||||
// Sets up a basic syncer that can be used to test OfferSnapshot requests
|
||||
func setupOfferSyncer(t *testing.T) (*syncer, *proxymocks.AppConnSnapshot) {
|
||||
connQuery := &proxymocks.AppConnQuery{}
|
||||
connSnapshot := &proxymocks.AppConnSnapshot{}
|
||||
stateProvider := &mocks.StateProvider{}
|
||||
stateProvider.On("AppHash", mock.Anything).Return([]byte("app_hash"), nil)
|
||||
syncer := newSyncer(log.NewNopLogger(), connSnapshot, connQuery, stateProvider, "")
|
||||
return syncer, connSnapshot
|
||||
}
|
||||
|
||||
// Sets up a simple peer mock with an ID
|
||||
func simplePeer(id string) *p2pmocks.Peer {
|
||||
peer := &p2pmocks.Peer{}
|
||||
peer.On("ID").Return(p2p.ID(id))
|
||||
return peer
|
||||
}
|
||||
|
||||
func TestSyncer_SyncAny(t *testing.T) {
|
||||
state := sm.State{
|
||||
ChainID: "chain",
|
||||
Version: sm.Version{
|
||||
Consensus: version.Consensus{
|
||||
Block: version.BlockProtocol,
|
||||
App: 0,
|
||||
},
|
||||
|
||||
Software: version.TMCoreSemVer,
|
||||
},
|
||||
|
||||
LastBlockHeight: 1,
|
||||
LastBlockID: types.BlockID{Hash: []byte("blockhash")},
|
||||
LastBlockTime: time.Now(),
|
||||
LastResultsHash: []byte("last_results_hash"),
|
||||
AppHash: []byte("app_hash"),
|
||||
|
||||
LastValidators: &types.ValidatorSet{Proposer: &types.Validator{Address: []byte("val1")}},
|
||||
Validators: &types.ValidatorSet{Proposer: &types.Validator{Address: []byte("val2")}},
|
||||
NextValidators: &types.ValidatorSet{Proposer: &types.Validator{Address: []byte("val3")}},
|
||||
|
||||
ConsensusParams: *types.DefaultConsensusParams(),
|
||||
LastHeightConsensusParamsChanged: 1,
|
||||
}
|
||||
commit := &types.Commit{BlockID: types.BlockID{Hash: []byte("blockhash")}}
|
||||
|
||||
chunks := []*chunk{
|
||||
{Height: 1, Format: 1, Index: 0, Chunk: []byte{1, 1, 0}},
|
||||
{Height: 1, Format: 1, Index: 1, Chunk: []byte{1, 1, 1}},
|
||||
{Height: 1, Format: 1, Index: 2, Chunk: []byte{1, 1, 2}},
|
||||
}
|
||||
s := &snapshot{Height: 1, Format: 1, Chunks: 3, Hash: []byte{1, 2, 3}}
|
||||
|
||||
stateProvider := &mocks.StateProvider{}
|
||||
stateProvider.On("AppHash", uint64(1)).Return(state.AppHash, nil)
|
||||
stateProvider.On("AppHash", uint64(2)).Return([]byte("app_hash_2"), nil)
|
||||
stateProvider.On("Commit", uint64(1)).Return(commit, nil)
|
||||
stateProvider.On("State", uint64(1)).Return(state, nil)
|
||||
connSnapshot := &proxymocks.AppConnSnapshot{}
|
||||
connQuery := &proxymocks.AppConnQuery{}
|
||||
|
||||
syncer := newSyncer(log.NewNopLogger(), connSnapshot, connQuery, stateProvider, "")
|
||||
|
||||
// Adding a chunk should error when no sync is in progress
|
||||
_, err := syncer.AddChunk(&chunk{Height: 1, Format: 1, Index: 0, Chunk: []byte{1}})
|
||||
require.Error(t, err)
|
||||
|
||||
// Adding a couple of peers should trigger snapshot discovery messages
|
||||
peerA := &p2pmocks.Peer{}
|
||||
peerA.On("ID").Return(p2p.ID("a"))
|
||||
peerA.On("Send", SnapshotChannel, cdc.MustMarshalBinaryBare(&snapshotsRequestMessage{})).Return(true)
|
||||
syncer.AddPeer(peerA)
|
||||
peerA.AssertExpectations(t)
|
||||
|
||||
peerB := &p2pmocks.Peer{}
|
||||
peerB.On("ID").Return(p2p.ID("b"))
|
||||
peerB.On("Send", SnapshotChannel, cdc.MustMarshalBinaryBare(&snapshotsRequestMessage{})).Return(true)
|
||||
syncer.AddPeer(peerB)
|
||||
peerB.AssertExpectations(t)
|
||||
|
||||
// Both peers report back with snapshots. One of them also returns a snapshot we don't want, in
|
||||
// format 2, which will be rejected by the ABCI application.
|
||||
new, err := syncer.AddSnapshot(peerA, s)
|
||||
require.NoError(t, err)
|
||||
assert.True(t, new)
|
||||
|
||||
new, err = syncer.AddSnapshot(peerB, s)
|
||||
require.NoError(t, err)
|
||||
assert.False(t, new)
|
||||
|
||||
new, err = syncer.AddSnapshot(peerB, &snapshot{Height: 2, Format: 2, Chunks: 3, Hash: []byte{1}})
|
||||
require.NoError(t, err)
|
||||
assert.True(t, new)
|
||||
|
||||
// We start a sync, with peers sending back chunks when requested. We first reject the snapshot
|
||||
// with height 2 format 2, and accept the snapshot at height 1.
|
||||
connSnapshot.On("OfferSnapshotSync", abci.RequestOfferSnapshot{
|
||||
Snapshot: &abci.Snapshot{
|
||||
Height: 2,
|
||||
Format: 2,
|
||||
Chunks: 3,
|
||||
Hash: []byte{1},
|
||||
},
|
||||
AppHash: []byte("app_hash_2"),
|
||||
}).Return(&abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_reject_format}, nil)
|
||||
connSnapshot.On("OfferSnapshotSync", abci.RequestOfferSnapshot{
|
||||
Snapshot: &abci.Snapshot{
|
||||
Height: s.Height,
|
||||
Format: s.Format,
|
||||
Chunks: s.Chunks,
|
||||
Hash: s.Hash,
|
||||
Metadata: s.Metadata,
|
||||
},
|
||||
AppHash: []byte("app_hash"),
|
||||
}).Times(2).Return(&abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_accept}, nil)
|
||||
|
||||
chunkRequests := make(map[uint32]int)
|
||||
chunkRequestsMtx := sync.Mutex{}
|
||||
onChunkRequest := func(args mock.Arguments) {
|
||||
msg := &chunkRequestMessage{}
|
||||
err := cdc.UnmarshalBinaryBare(args[1].([]byte), &msg)
|
||||
require.NoError(t, err)
|
||||
require.EqualValues(t, 1, msg.Height)
|
||||
require.EqualValues(t, 1, msg.Format)
|
||||
require.LessOrEqual(t, msg.Index, uint32(len(chunks)))
|
||||
|
||||
added, err := syncer.AddChunk(chunks[msg.Index])
|
||||
require.NoError(t, err)
|
||||
assert.True(t, added)
|
||||
|
||||
chunkRequestsMtx.Lock()
|
||||
chunkRequests[msg.Index]++
|
||||
chunkRequestsMtx.Unlock()
|
||||
}
|
||||
peerA.On("Send", ChunkChannel, mock.Anything).Maybe().Run(onChunkRequest).Return(true)
|
||||
peerB.On("Send", ChunkChannel, mock.Anything).Maybe().Run(onChunkRequest).Return(true)
|
||||
|
||||
// The first time we're applying chunk 2 we tell it to retry the snapshot and discard chunk 1,
|
||||
// which should cause it to keep the existing chunk 0 and 2, and restart restoration from
|
||||
// beginning. We also wait for a little while, to exercise the retry logic in fetchChunks().
|
||||
connSnapshot.On("ApplySnapshotChunkSync", abci.RequestApplySnapshotChunk{
|
||||
Index: 2, Chunk: []byte{1, 1, 2},
|
||||
}).Once().Run(func(args mock.Arguments) { time.Sleep(2 * time.Second) }).Return(
|
||||
&abci.ResponseApplySnapshotChunk{
|
||||
Result: abci.ResponseApplySnapshotChunk_retry_snapshot,
|
||||
RefetchChunks: []uint32{1},
|
||||
}, nil)
|
||||
|
||||
connSnapshot.On("ApplySnapshotChunkSync", abci.RequestApplySnapshotChunk{
|
||||
Index: 0, Chunk: []byte{1, 1, 0},
|
||||
}).Times(2).Return(&abci.ResponseApplySnapshotChunk{Result: abci.ResponseApplySnapshotChunk_accept}, nil)
|
||||
connSnapshot.On("ApplySnapshotChunkSync", abci.RequestApplySnapshotChunk{
|
||||
Index: 1, Chunk: []byte{1, 1, 1},
|
||||
}).Times(2).Return(&abci.ResponseApplySnapshotChunk{Result: abci.ResponseApplySnapshotChunk_accept}, nil)
|
||||
connSnapshot.On("ApplySnapshotChunkSync", abci.RequestApplySnapshotChunk{
|
||||
Index: 2, Chunk: []byte{1, 1, 2},
|
||||
}).Once().Return(&abci.ResponseApplySnapshotChunk{Result: abci.ResponseApplySnapshotChunk_accept}, nil)
|
||||
connQuery.On("InfoSync", proxy.RequestInfo).Return(&abci.ResponseInfo{
|
||||
AppVersion: 9,
|
||||
LastBlockHeight: 1,
|
||||
LastBlockAppHash: []byte("app_hash"),
|
||||
}, nil)
|
||||
|
||||
newState, lastCommit, err := syncer.SyncAny(0)
|
||||
require.NoError(t, err)
|
||||
|
||||
chunkRequestsMtx.Lock()
|
||||
assert.Equal(t, map[uint32]int{0: 1, 1: 2, 2: 1}, chunkRequests)
|
||||
chunkRequestsMtx.Unlock()
|
||||
|
||||
// The syncer should have updated the state app version from the ABCI info response.
|
||||
expectState := state
|
||||
expectState.Version.Consensus.App = 9
|
||||
|
||||
assert.Equal(t, expectState, newState)
|
||||
assert.Equal(t, commit, lastCommit)
|
||||
|
||||
connSnapshot.AssertExpectations(t)
|
||||
connQuery.AssertExpectations(t)
|
||||
peerA.AssertExpectations(t)
|
||||
peerB.AssertExpectations(t)
|
||||
}
|
||||
|
||||
func TestSyncer_SyncAny_noSnapshots(t *testing.T) {
|
||||
syncer, _ := setupOfferSyncer(t)
|
||||
_, _, err := syncer.SyncAny(0)
|
||||
assert.Equal(t, errNoSnapshots, err)
|
||||
}
|
||||
|
||||
func TestSyncer_SyncAny_abort(t *testing.T) {
|
||||
syncer, connSnapshot := setupOfferSyncer(t)
|
||||
|
||||
s := &snapshot{Height: 1, Format: 1, Chunks: 3, Hash: []byte{1, 2, 3}}
|
||||
syncer.AddSnapshot(simplePeer("id"), s)
|
||||
connSnapshot.On("OfferSnapshotSync", abci.RequestOfferSnapshot{
|
||||
Snapshot: toABCI(s), AppHash: []byte("app_hash"),
|
||||
}).Once().Return(&abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_abort}, nil)
|
||||
|
||||
_, _, err := syncer.SyncAny(0)
|
||||
assert.Equal(t, errAbort, err)
|
||||
connSnapshot.AssertExpectations(t)
|
||||
}
|
||||
|
||||
func TestSyncer_SyncAny_reject(t *testing.T) {
|
||||
syncer, connSnapshot := setupOfferSyncer(t)
|
||||
|
||||
// s22 is tried first, then s12, then s11, then errNoSnapshots
|
||||
s22 := &snapshot{Height: 2, Format: 2, Chunks: 3, Hash: []byte{1, 2, 3}}
|
||||
s12 := &snapshot{Height: 1, Format: 2, Chunks: 3, Hash: []byte{1, 2, 3}}
|
||||
s11 := &snapshot{Height: 1, Format: 1, Chunks: 3, Hash: []byte{1, 2, 3}}
|
||||
syncer.AddSnapshot(simplePeer("id"), s22)
|
||||
syncer.AddSnapshot(simplePeer("id"), s12)
|
||||
syncer.AddSnapshot(simplePeer("id"), s11)
|
||||
|
||||
connSnapshot.On("OfferSnapshotSync", abci.RequestOfferSnapshot{
|
||||
Snapshot: toABCI(s22), AppHash: []byte("app_hash"),
|
||||
}).Once().Return(&abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_reject}, nil)
|
||||
|
||||
connSnapshot.On("OfferSnapshotSync", abci.RequestOfferSnapshot{
|
||||
Snapshot: toABCI(s12), AppHash: []byte("app_hash"),
|
||||
}).Once().Return(&abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_reject}, nil)
|
||||
|
||||
connSnapshot.On("OfferSnapshotSync", abci.RequestOfferSnapshot{
|
||||
Snapshot: toABCI(s11), AppHash: []byte("app_hash"),
|
||||
}).Once().Return(&abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_reject}, nil)
|
||||
|
||||
_, _, err := syncer.SyncAny(0)
|
||||
assert.Equal(t, errNoSnapshots, err)
|
||||
connSnapshot.AssertExpectations(t)
|
||||
}
|
||||
|
||||
func TestSyncer_SyncAny_reject_format(t *testing.T) {
|
||||
syncer, connSnapshot := setupOfferSyncer(t)
|
||||
|
||||
// s22 is tried first, which reject s22 and s12, then s11 will abort.
|
||||
s22 := &snapshot{Height: 2, Format: 2, Chunks: 3, Hash: []byte{1, 2, 3}}
|
||||
s12 := &snapshot{Height: 1, Format: 2, Chunks: 3, Hash: []byte{1, 2, 3}}
|
||||
s11 := &snapshot{Height: 1, Format: 1, Chunks: 3, Hash: []byte{1, 2, 3}}
|
||||
syncer.AddSnapshot(simplePeer("id"), s22)
|
||||
syncer.AddSnapshot(simplePeer("id"), s12)
|
||||
syncer.AddSnapshot(simplePeer("id"), s11)
|
||||
|
||||
connSnapshot.On("OfferSnapshotSync", abci.RequestOfferSnapshot{
|
||||
Snapshot: toABCI(s22), AppHash: []byte("app_hash"),
|
||||
}).Once().Return(&abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_reject_format}, nil)
|
||||
|
||||
connSnapshot.On("OfferSnapshotSync", abci.RequestOfferSnapshot{
|
||||
Snapshot: toABCI(s11), AppHash: []byte("app_hash"),
|
||||
}).Once().Return(&abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_abort}, nil)
|
||||
|
||||
_, _, err := syncer.SyncAny(0)
|
||||
assert.Equal(t, errAbort, err)
|
||||
connSnapshot.AssertExpectations(t)
|
||||
}
|
||||
|
||||
func TestSyncer_SyncAny_reject_sender(t *testing.T) {
|
||||
syncer, connSnapshot := setupOfferSyncer(t)
|
||||
|
||||
peerA := simplePeer("a")
|
||||
peerB := simplePeer("b")
|
||||
peerC := simplePeer("c")
|
||||
|
||||
// sbc will be offered first, which will be rejected with reject_sender, causing all snapshots
|
||||
// submitted by both b and c (i.e. sb, sc, sbc) to be rejected. Finally, sa will reject and
|
||||
// errNoSnapshots is returned.
|
||||
sa := &snapshot{Height: 1, Format: 1, Chunks: 3, Hash: []byte{1, 2, 3}}
|
||||
sb := &snapshot{Height: 2, Format: 1, Chunks: 3, Hash: []byte{1, 2, 3}}
|
||||
sc := &snapshot{Height: 3, Format: 1, Chunks: 3, Hash: []byte{1, 2, 3}}
|
||||
sbc := &snapshot{Height: 4, Format: 1, Chunks: 3, Hash: []byte{1, 2, 3}}
|
||||
_, err := syncer.AddSnapshot(peerA, sa)
|
||||
require.NoError(t, err)
|
||||
_, err = syncer.AddSnapshot(peerB, sb)
|
||||
require.NoError(t, err)
|
||||
_, err = syncer.AddSnapshot(peerC, sc)
|
||||
require.NoError(t, err)
|
||||
_, err = syncer.AddSnapshot(peerB, sbc)
|
||||
require.NoError(t, err)
|
||||
_, err = syncer.AddSnapshot(peerC, sbc)
|
||||
require.NoError(t, err)
|
||||
|
||||
connSnapshot.On("OfferSnapshotSync", abci.RequestOfferSnapshot{
|
||||
Snapshot: toABCI(sbc), AppHash: []byte("app_hash"),
|
||||
}).Once().Return(&abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_reject_sender}, nil)
|
||||
|
||||
connSnapshot.On("OfferSnapshotSync", abci.RequestOfferSnapshot{
|
||||
Snapshot: toABCI(sa), AppHash: []byte("app_hash"),
|
||||
}).Once().Return(&abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_reject}, nil)
|
||||
|
||||
_, _, err = syncer.SyncAny(0)
|
||||
assert.Equal(t, errNoSnapshots, err)
|
||||
connSnapshot.AssertExpectations(t)
|
||||
}
|
||||
|
||||
func TestSyncer_SyncAny_abciError(t *testing.T) {
|
||||
syncer, connSnapshot := setupOfferSyncer(t)
|
||||
|
||||
errBoom := errors.New("boom")
|
||||
s := &snapshot{Height: 1, Format: 1, Chunks: 3, Hash: []byte{1, 2, 3}}
|
||||
syncer.AddSnapshot(simplePeer("id"), s)
|
||||
connSnapshot.On("OfferSnapshotSync", abci.RequestOfferSnapshot{
|
||||
Snapshot: toABCI(s), AppHash: []byte("app_hash"),
|
||||
}).Once().Return(nil, errBoom)
|
||||
|
||||
_, _, err := syncer.SyncAny(0)
|
||||
assert.True(t, errors.Is(err, errBoom))
|
||||
connSnapshot.AssertExpectations(t)
|
||||
}
|
||||
|
||||
func TestSyncer_offerSnapshot(t *testing.T) {
|
||||
unknownErr := errors.New("unknown error")
|
||||
boom := errors.New("boom")
|
||||
|
||||
testcases := map[string]struct {
|
||||
result abci.ResponseOfferSnapshot_Result
|
||||
err error
|
||||
expectErr error
|
||||
}{
|
||||
"accept": {abci.ResponseOfferSnapshot_accept, nil, nil},
|
||||
"abort": {abci.ResponseOfferSnapshot_abort, nil, errAbort},
|
||||
"reject": {abci.ResponseOfferSnapshot_reject, nil, errRejectSnapshot},
|
||||
"reject_format": {abci.ResponseOfferSnapshot_reject_format, nil, errRejectFormat},
|
||||
"reject_sender": {abci.ResponseOfferSnapshot_reject_sender, nil, errRejectSender},
|
||||
"error": {0, boom, boom},
|
||||
"unknown result": {9, nil, unknownErr},
|
||||
}
|
||||
for name, tc := range testcases {
|
||||
tc := tc
|
||||
t.Run(name, func(t *testing.T) {
|
||||
syncer, connSnapshot := setupOfferSyncer(t)
|
||||
s := &snapshot{Height: 1, Format: 1, Chunks: 3, Hash: []byte{1, 2, 3}, trustedAppHash: []byte("app_hash")}
|
||||
connSnapshot.On("OfferSnapshotSync", abci.RequestOfferSnapshot{
|
||||
Snapshot: toABCI(s),
|
||||
AppHash: []byte("app_hash"),
|
||||
}).Return(&abci.ResponseOfferSnapshot{Result: tc.result}, tc.err)
|
||||
err := syncer.offerSnapshot(s)
|
||||
if tc.expectErr == unknownErr {
|
||||
require.Error(t, err)
|
||||
} else {
|
||||
unwrapped := errors.Unwrap(err)
|
||||
if unwrapped != nil {
|
||||
err = unwrapped
|
||||
}
|
||||
assert.Equal(t, tc.expectErr, err)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestSyncer_applyChunks_Results(t *testing.T) {
|
||||
unknownErr := errors.New("unknown error")
|
||||
boom := errors.New("boom")
|
||||
|
||||
testcases := map[string]struct {
|
||||
result abci.ResponseApplySnapshotChunk_Result
|
||||
err error
|
||||
expectErr error
|
||||
}{
|
||||
"accept": {abci.ResponseApplySnapshotChunk_accept, nil, nil},
|
||||
"abort": {abci.ResponseApplySnapshotChunk_abort, nil, errAbort},
|
||||
"retry": {abci.ResponseApplySnapshotChunk_retry, nil, nil},
|
||||
"retry_snapshot": {abci.ResponseApplySnapshotChunk_retry_snapshot, nil, errRetrySnapshot},
|
||||
"reject_snapshot": {abci.ResponseApplySnapshotChunk_reject_snapshot, nil, errRejectSnapshot},
|
||||
"error": {0, boom, boom},
|
||||
"unknown result": {9, nil, unknownErr},
|
||||
}
|
||||
for name, tc := range testcases {
|
||||
tc := tc
|
||||
t.Run(name, func(t *testing.T) {
|
||||
connQuery := &proxymocks.AppConnQuery{}
|
||||
connSnapshot := &proxymocks.AppConnSnapshot{}
|
||||
stateProvider := &mocks.StateProvider{}
|
||||
stateProvider.On("AppHash", mock.Anything).Return([]byte("app_hash"), nil)
|
||||
syncer := newSyncer(log.NewNopLogger(), connSnapshot, connQuery, stateProvider, "")
|
||||
|
||||
body := []byte{1, 2, 3}
|
||||
chunks, err := newChunkQueue(&snapshot{Height: 1, Format: 1, Chunks: 1}, "")
|
||||
chunks.Add(&chunk{Height: 1, Format: 1, Index: 0, Chunk: body})
|
||||
require.NoError(t, err)
|
||||
|
||||
connSnapshot.On("ApplySnapshotChunkSync", abci.RequestApplySnapshotChunk{
|
||||
Index: 0, Chunk: body,
|
||||
}).Once().Return(&abci.ResponseApplySnapshotChunk{Result: tc.result}, tc.err)
|
||||
if tc.result == abci.ResponseApplySnapshotChunk_retry {
|
||||
connSnapshot.On("ApplySnapshotChunkSync", abci.RequestApplySnapshotChunk{
|
||||
Index: 0, Chunk: body,
|
||||
}).Once().Return(&abci.ResponseApplySnapshotChunk{
|
||||
Result: abci.ResponseApplySnapshotChunk_accept}, nil)
|
||||
}
|
||||
|
||||
err = syncer.applyChunks(chunks)
|
||||
if tc.expectErr == unknownErr {
|
||||
require.Error(t, err)
|
||||
} else {
|
||||
unwrapped := errors.Unwrap(err)
|
||||
if unwrapped != nil {
|
||||
err = unwrapped
|
||||
}
|
||||
assert.Equal(t, tc.expectErr, err)
|
||||
}
|
||||
connSnapshot.AssertExpectations(t)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestSyncer_applyChunks_RefetchChunks(t *testing.T) {
|
||||
// Discarding chunks via refetch_chunks should work the same for all results
|
||||
testcases := map[string]struct {
|
||||
result abci.ResponseApplySnapshotChunk_Result
|
||||
}{
|
||||
"accept": {abci.ResponseApplySnapshotChunk_accept},
|
||||
"abort": {abci.ResponseApplySnapshotChunk_abort},
|
||||
"retry": {abci.ResponseApplySnapshotChunk_retry},
|
||||
"retry_snapshot": {abci.ResponseApplySnapshotChunk_retry_snapshot},
|
||||
"reject_snapshot": {abci.ResponseApplySnapshotChunk_reject_snapshot},
|
||||
}
|
||||
for name, tc := range testcases {
|
||||
tc := tc
|
||||
t.Run(name, func(t *testing.T) {
|
||||
connQuery := &proxymocks.AppConnQuery{}
|
||||
connSnapshot := &proxymocks.AppConnSnapshot{}
|
||||
stateProvider := &mocks.StateProvider{}
|
||||
stateProvider.On("AppHash", mock.Anything).Return([]byte("app_hash"), nil)
|
||||
syncer := newSyncer(log.NewNopLogger(), connSnapshot, connQuery, stateProvider, "")
|
||||
|
||||
chunks, err := newChunkQueue(&snapshot{Height: 1, Format: 1, Chunks: 3}, "")
|
||||
require.NoError(t, err)
|
||||
added, err := chunks.Add(&chunk{Height: 1, Format: 1, Index: 0, Chunk: []byte{0}})
|
||||
require.True(t, added)
|
||||
require.NoError(t, err)
|
||||
added, err = chunks.Add(&chunk{Height: 1, Format: 1, Index: 1, Chunk: []byte{1}})
|
||||
require.True(t, added)
|
||||
require.NoError(t, err)
|
||||
added, err = chunks.Add(&chunk{Height: 1, Format: 1, Index: 2, Chunk: []byte{2}})
|
||||
require.True(t, added)
|
||||
require.NoError(t, err)
|
||||
|
||||
// The first two chunks are accepted, before the last one asks for 1 to be refetched
|
||||
connSnapshot.On("ApplySnapshotChunkSync", abci.RequestApplySnapshotChunk{
|
||||
Index: 0, Chunk: []byte{0},
|
||||
}).Once().Return(&abci.ResponseApplySnapshotChunk{Result: abci.ResponseApplySnapshotChunk_accept}, nil)
|
||||
connSnapshot.On("ApplySnapshotChunkSync", abci.RequestApplySnapshotChunk{
|
||||
Index: 1, Chunk: []byte{1},
|
||||
}).Once().Return(&abci.ResponseApplySnapshotChunk{Result: abci.ResponseApplySnapshotChunk_accept}, nil)
|
||||
connSnapshot.On("ApplySnapshotChunkSync", abci.RequestApplySnapshotChunk{
|
||||
Index: 2, Chunk: []byte{2},
|
||||
}).Once().Return(&abci.ResponseApplySnapshotChunk{
|
||||
Result: tc.result,
|
||||
RefetchChunks: []uint32{1},
|
||||
}, nil)
|
||||
|
||||
// Since removing the chunk will cause Next() to block, we spawn a goroutine, then
|
||||
// check the queue contents, and finally close the queue to end the goroutine.
|
||||
// We don't really care about the result of applyChunks, since it has separate test.
|
||||
go func() {
|
||||
syncer.applyChunks(chunks)
|
||||
}()
|
||||
|
||||
time.Sleep(50 * time.Millisecond)
|
||||
assert.True(t, chunks.Has(0))
|
||||
assert.False(t, chunks.Has(1))
|
||||
assert.True(t, chunks.Has(2))
|
||||
err = chunks.Close()
|
||||
require.NoError(t, err)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestSyncer_applyChunks_RejectSenders(t *testing.T) {
|
||||
// Banning chunks senders via ban_chunk_senders should work the same for all results
|
||||
testcases := map[string]struct {
|
||||
result abci.ResponseApplySnapshotChunk_Result
|
||||
}{
|
||||
"accept": {abci.ResponseApplySnapshotChunk_accept},
|
||||
"abort": {abci.ResponseApplySnapshotChunk_abort},
|
||||
"retry": {abci.ResponseApplySnapshotChunk_retry},
|
||||
"retry_snapshot": {abci.ResponseApplySnapshotChunk_retry_snapshot},
|
||||
"reject_snapshot": {abci.ResponseApplySnapshotChunk_reject_snapshot},
|
||||
}
|
||||
for name, tc := range testcases {
|
||||
tc := tc
|
||||
t.Run(name, func(t *testing.T) {
|
||||
connQuery := &proxymocks.AppConnQuery{}
|
||||
connSnapshot := &proxymocks.AppConnSnapshot{}
|
||||
stateProvider := &mocks.StateProvider{}
|
||||
stateProvider.On("AppHash", mock.Anything).Return([]byte("app_hash"), nil)
|
||||
syncer := newSyncer(log.NewNopLogger(), connSnapshot, connQuery, stateProvider, "")
|
||||
|
||||
// Set up three peers across two snapshots, and ask for one of them to be banned.
|
||||
// It should be banned from all snapshots.
|
||||
peerA := simplePeer("a")
|
||||
peerB := simplePeer("b")
|
||||
peerC := simplePeer("c")
|
||||
|
||||
s1 := &snapshot{Height: 1, Format: 1, Chunks: 3}
|
||||
s2 := &snapshot{Height: 2, Format: 1, Chunks: 3}
|
||||
syncer.AddSnapshot(peerA, s1)
|
||||
syncer.AddSnapshot(peerA, s2)
|
||||
syncer.AddSnapshot(peerB, s1)
|
||||
syncer.AddSnapshot(peerB, s2)
|
||||
syncer.AddSnapshot(peerC, s1)
|
||||
syncer.AddSnapshot(peerC, s2)
|
||||
|
||||
chunks, err := newChunkQueue(s1, "")
|
||||
require.NoError(t, err)
|
||||
added, err := chunks.Add(&chunk{Height: 1, Format: 1, Index: 0, Chunk: []byte{0}, Sender: peerA.ID()})
|
||||
require.True(t, added)
|
||||
require.NoError(t, err)
|
||||
added, err = chunks.Add(&chunk{Height: 1, Format: 1, Index: 1, Chunk: []byte{1}, Sender: peerB.ID()})
|
||||
require.True(t, added)
|
||||
require.NoError(t, err)
|
||||
added, err = chunks.Add(&chunk{Height: 1, Format: 1, Index: 2, Chunk: []byte{2}, Sender: peerC.ID()})
|
||||
require.True(t, added)
|
||||
require.NoError(t, err)
|
||||
|
||||
// The first two chunks are accepted, before the last one asks for b sender to be rejected
|
||||
connSnapshot.On("ApplySnapshotChunkSync", abci.RequestApplySnapshotChunk{
|
||||
Index: 0, Chunk: []byte{0}, Sender: "a",
|
||||
}).Once().Return(&abci.ResponseApplySnapshotChunk{Result: abci.ResponseApplySnapshotChunk_accept}, nil)
|
||||
connSnapshot.On("ApplySnapshotChunkSync", abci.RequestApplySnapshotChunk{
|
||||
Index: 1, Chunk: []byte{1}, Sender: "b",
|
||||
}).Once().Return(&abci.ResponseApplySnapshotChunk{Result: abci.ResponseApplySnapshotChunk_accept}, nil)
|
||||
connSnapshot.On("ApplySnapshotChunkSync", abci.RequestApplySnapshotChunk{
|
||||
Index: 2, Chunk: []byte{2}, Sender: "c",
|
||||
}).Once().Return(&abci.ResponseApplySnapshotChunk{
|
||||
Result: tc.result,
|
||||
RejectSenders: []string{string(peerB.ID())},
|
||||
}, nil)
|
||||
|
||||
// On retry, the last chunk will be tried again, so we just accept it then.
|
||||
if tc.result == abci.ResponseApplySnapshotChunk_retry {
|
||||
connSnapshot.On("ApplySnapshotChunkSync", abci.RequestApplySnapshotChunk{
|
||||
Index: 2, Chunk: []byte{2}, Sender: "c",
|
||||
}).Once().Return(&abci.ResponseApplySnapshotChunk{Result: abci.ResponseApplySnapshotChunk_accept}, nil)
|
||||
}
|
||||
|
||||
// We don't really care about the result of applyChunks, since it has separate test.
|
||||
// However, it will block on e.g. retry result, so we spawn a goroutine that will
|
||||
// be shut down when the chunk queue closes.
|
||||
go func() {
|
||||
syncer.applyChunks(chunks)
|
||||
}()
|
||||
|
||||
time.Sleep(50 * time.Millisecond)
|
||||
|
||||
s1peers := syncer.snapshots.GetPeers(s1)
|
||||
assert.Len(t, s1peers, 2)
|
||||
assert.EqualValues(t, "a", s1peers[0].ID())
|
||||
assert.EqualValues(t, "c", s1peers[1].ID())
|
||||
|
||||
syncer.snapshots.GetPeers(s1)
|
||||
assert.Len(t, s1peers, 2)
|
||||
assert.EqualValues(t, "a", s1peers[0].ID())
|
||||
assert.EqualValues(t, "c", s1peers[1].ID())
|
||||
|
||||
err = chunks.Close()
|
||||
require.NoError(t, err)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestSyncer_verifyApp(t *testing.T) {
|
||||
boom := errors.New("boom")
|
||||
s := &snapshot{Height: 3, Format: 1, Chunks: 5, Hash: []byte{1, 2, 3}, trustedAppHash: []byte("app_hash")}
|
||||
|
||||
testcases := map[string]struct {
|
||||
response *abci.ResponseInfo
|
||||
err error
|
||||
expectErr error
|
||||
}{
|
||||
"verified": {&abci.ResponseInfo{
|
||||
LastBlockHeight: 3,
|
||||
LastBlockAppHash: []byte("app_hash"),
|
||||
AppVersion: 9,
|
||||
}, nil, nil},
|
||||
"invalid height": {&abci.ResponseInfo{
|
||||
LastBlockHeight: 5,
|
||||
LastBlockAppHash: []byte("app_hash"),
|
||||
AppVersion: 9,
|
||||
}, nil, errVerifyFailed},
|
||||
"invalid hash": {&abci.ResponseInfo{
|
||||
LastBlockHeight: 3,
|
||||
LastBlockAppHash: []byte("xxx"),
|
||||
AppVersion: 9,
|
||||
}, nil, errVerifyFailed},
|
||||
"error": {nil, boom, boom},
|
||||
}
|
||||
for name, tc := range testcases {
|
||||
tc := tc
|
||||
t.Run(name, func(t *testing.T) {
|
||||
connQuery := &proxymocks.AppConnQuery{}
|
||||
connSnapshot := &proxymocks.AppConnSnapshot{}
|
||||
stateProvider := &mocks.StateProvider{}
|
||||
syncer := newSyncer(log.NewNopLogger(), connSnapshot, connQuery, stateProvider, "")
|
||||
|
||||
connQuery.On("InfoSync", proxy.RequestInfo).Return(tc.response, tc.err)
|
||||
version, err := syncer.verifyApp(s)
|
||||
unwrapped := errors.Unwrap(err)
|
||||
if unwrapped != nil {
|
||||
err = unwrapped
|
||||
}
|
||||
assert.Equal(t, tc.expectErr, err)
|
||||
if err == nil {
|
||||
assert.Equal(t, tc.response.AppVersion, version)
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func toABCI(s *snapshot) *abci.Snapshot {
|
||||
return &abci.Snapshot{
|
||||
Height: s.Height,
|
||||
Format: s.Format,
|
||||
Chunks: s.Chunks,
|
||||
Hash: s.Hash,
|
||||
Metadata: s.Metadata,
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user