diff --git a/simplex/block.go b/simplex/block.go index 3d6d9abdc205..476e25a0d482 100644 --- a/simplex/block.go +++ b/simplex/block.go @@ -7,19 +7,26 @@ package simplex import ( "context" + "errors" "fmt" + "sync" "github.com/ava-labs/simplex" "github.com/ava-labs/avalanchego/snow/consensus/snowman" "github.com/ava-labs/avalanchego/snow/engine/snowman/block" "github.com/ava-labs/avalanchego/utils/hashing" + "github.com/ava-labs/avalanchego/utils/tree" ) var ( _ simplex.BlockDeserializer = (*blockDeserializer)(nil) _ simplex.Block = (*Block)(nil) _ simplex.VerifiedBlock = (*Block)(nil) + + errDigestNotFound = errors.New("digest not found in block tracker") + errMismatchedPrevDigest = errors.New("prev digest does not match block parent") + errGenesisVerification = errors.New("genesis block should not be verified") ) type Block struct { @@ -30,6 +37,8 @@ type Block struct { // the parsed block vmBlock snowman.Block + + blockTracker *blockTracker } // CanotoSimplexBlock is the Canoto representation of a block @@ -58,9 +67,41 @@ func (b *Block) Bytes() ([]byte, error) { return cBlock.MarshalCanoto(), nil } +// Verify verifies the block. func (b *Block) Verify(ctx context.Context) (simplex.VerifiedBlock, error) { - // TODO: track blocks that have been verified to ensure they are either rejected or accepted - return b, b.vmBlock.Verify(ctx) + // we should not verify the genesis block + if b.metadata.Seq == 0 { + return nil, errGenesisVerification + } + + if err := b.verifyParentMatchesPrevBlock(); err != nil { + return nil, err + } + + if err := b.blockTracker.verifyAndTrackBlock(ctx, b); err != nil { + return nil, fmt.Errorf("failed to verify block: %w", err) + } + + return b, nil +} + +// verifyParentMatchesPrevBlock verifies that the previous block referenced in the current block's metadata +// matches the parent of the current block's vmBlock. +func (b *Block) verifyParentMatchesPrevBlock() error { + prevBlock, ok := b.blockTracker.getBlockByDigest(b.metadata.Prev) + if !ok { + return fmt.Errorf("%w: %s", errDigestNotFound, b.metadata.Prev) + } + + if b.vmBlock.Parent() != prevBlock.vmBlock.ID() { + return fmt.Errorf("%w: parentID %s, prevID %s", errMismatchedPrevDigest, b.vmBlock.Parent(), prevBlock.vmBlock.ID()) + } + + return nil +} + +func computeDigest(bytes []byte) simplex.Digest { + return hashing.ComputeHash256Array(bytes) } type blockDeserializer struct { @@ -84,11 +125,80 @@ func (d *blockDeserializer) DeserializeBlock(ctx context.Context, bytes []byte) return nil, err } - digest := hashing.ComputeHash256Array(bytes) - return &Block{ metadata: *md, vmBlock: vmblock, - digest: digest, + digest: computeDigest(bytes), }, nil } + +// blockTracker is used to ensure that blocks are properly rejected, if competing blocks are accepted. +type blockTracker struct { + lock sync.Mutex + + // tracks the simplex digests to the blocks that have been verified + simplexDigestsToBlock map[simplex.Digest]*Block + + // handles block acceptance and rejection of inner blocks + tree tree.Tree +} + +func newBlockTracker(latestBlock *Block) *blockTracker { + return &blockTracker{ + tree: tree.New(), + simplexDigestsToBlock: map[simplex.Digest]*Block{ + latestBlock.digest: latestBlock, + }, + } +} + +func (bt *blockTracker) getBlockByDigest(digest simplex.Digest) (*Block, bool) { + bt.lock.Lock() + defer bt.lock.Unlock() + + block, exists := bt.simplexDigestsToBlock[digest] + return block, exists +} + +// verifyAndTrackBlock verifies the block and tracks it in the block tracker. +// If the block is already verified, it does nothing. +func (bt *blockTracker) verifyAndTrackBlock(ctx context.Context, block *Block) error { + bt.lock.Lock() + defer bt.lock.Unlock() + + // check if the block is already verified + if _, exists := bt.tree.Get(block.vmBlock); exists { + bt.simplexDigestsToBlock[block.digest] = block + return nil + } + + if err := block.vmBlock.Verify(ctx); err != nil { + return fmt.Errorf("failed to verify block: %w", err) + } + + // track the block + bt.simplexDigestsToBlock[block.digest] = block + bt.tree.Add(block.vmBlock) + return nil +} + +// indexBlock calls accept on the block with the given digest, and reject on competing blocks. +func (bt *blockTracker) indexBlock(ctx context.Context, digest simplex.Digest) error { + bt.lock.Lock() + defer bt.lock.Unlock() + + bd, exists := bt.simplexDigestsToBlock[digest] + if !exists { + return fmt.Errorf("%w: %s", errDigestNotFound, digest) + } + + // removes all digests with a lower seq + for d, block := range bt.simplexDigestsToBlock { + if block.metadata.Seq < bd.metadata.Seq { + delete(bt.simplexDigestsToBlock, d) + } + } + + // notify the VM that we are accepting this block, and reject all competing blocks + return bt.tree.Accept(ctx, bd.vmBlock) +} diff --git a/simplex/block_test.go b/simplex/block_test.go index 6b200d97d8be..c031d1709348 100644 --- a/simplex/block_test.go +++ b/simplex/block_test.go @@ -13,10 +13,12 @@ import ( "github.com/ava-labs/simplex" "github.com/stretchr/testify/require" + "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/avalanchego/snow/consensus/snowman" "github.com/ava-labs/avalanchego/snow/consensus/snowman/snowmantest" "github.com/ava-labs/avalanchego/snow/engine/enginetest" "github.com/ava-labs/avalanchego/snow/engine/snowman/block/blocktest" + "github.com/ava-labs/avalanchego/snow/snowtest" ) func TestBlockSerialization(t *testing.T) { @@ -98,29 +100,150 @@ func TestBlockSerialization(t *testing.T) { } } -// TestBlockVerify tests the verificatin of a block results in the same bytes as the original block. -func TestBlockVerify(t *testing.T) { - testBlock := snowmantest.BuildChild(snowmantest.Genesis) +// TestVerifyPrevNotFound attempts to verify a block with a prev digest that is not valid. +func TestVerifyPrevNotFound(t *testing.T) { + ctx := context.Background() - b := &Block{ - vmBlock: testBlock, - metadata: simplex.ProtocolMetadata{ - Version: 1, - Epoch: 1, - Round: 1, - Seq: 1, - Prev: [32]byte{0x00, 0x01, 0x02, 0x03, 0x04, 0x05, 0x06, 0x07}, - }, - } + genesis := newBlock(t, newBlockConfig{}) + b := newBlock(t, newBlockConfig{ + prev: genesis, + }) + b.metadata.Prev[0]++ // Invalid prev digest + + _, err := b.Verify(ctx) + require.ErrorIs(t, err, errDigestNotFound) +} + +// TestVerifyTwice tests that a block the same vmBlock will only +// have its Verify method called once, even if Verify is called multiple times. +func TestVerifyTwice(t *testing.T) { + ctx := context.Background() + + genesis := newBlock(t, newBlockConfig{}) + b := newBlock(t, newBlockConfig{ + prev: genesis, + }) + + // Verify the block for the first time + _, err := b.Verify(ctx) + require.NoError(t, err) - verifiedBlock, err := b.Verify(context.Background()) - require.NoError(t, err, "block should be verified successfully") + // Attempt to verify the block again + b.vmBlock.(*snowmantest.Block).VerifyV = errors.New("should not be called again") + _, err = b.Verify(ctx) + require.NoError(t, err) +} + +// TestVerifyGenesis tests that a block with a sequence number of 0 cannot be verified. +func TestVerifyGenesis(t *testing.T) { + ctx := context.Background() + + genesis := newBlock(t, newBlockConfig{}) + _, err := genesis.Verify(ctx) + require.ErrorIs(t, err, errGenesisVerification) +} + +func TestVerify(t *testing.T) { + ctx := context.Background() + + genesis := newBlock(t, newBlockConfig{}) + b := newBlock(t, newBlockConfig{ + prev: genesis, + }) + + verifiedBlock, err := b.Verify(ctx) + require.NoError(t, err) + // Ensure the verified block matches the original block vBlockBytes, err := verifiedBlock.Bytes() require.NoError(t, err) blockBytes, err := b.Bytes() require.NoError(t, err) - require.Equal(t, blockBytes, vBlockBytes, "block bytes should match after verification") } + +// TestVerifyParentAccepted tests that a block, whose parent has been verified +// and indexed, can also be verified and indexed successfully. +func TestVerifyParentAccepted(t *testing.T) { + ctx := context.Background() + + genesis := newBlock(t, newBlockConfig{}) + seq1Block := newBlock(t, newBlockConfig{ + prev: genesis, + }) + seq2Block := newBlock(t, newBlockConfig{ + prev: seq1Block, + }) + + _, err := seq1Block.Verify(ctx) + require.NoError(t, err) + require.NoError(t, seq1Block.blockTracker.indexBlock(ctx, seq1Block.digest)) + require.Equal(t, snowtest.Accepted, seq1Block.vmBlock.(*snowmantest.Block).Decidable.Status) + + // Verify the second block with the first block as its parent + _, err = seq2Block.Verify(ctx) + require.NoError(t, err) + require.NoError(t, seq2Block.blockTracker.indexBlock(ctx, seq2Block.digest)) + require.Equal(t, snowtest.Accepted, seq2Block.vmBlock.(*snowmantest.Block).Decidable.Status) + + // ensure tracker cleans up the block + require.NotContains(t, genesis.blockTracker.simplexDigestsToBlock, seq1Block.digest) +} + +func TestVerifyBlockRejectsSiblings(t *testing.T) { + ctx := context.Background() + + genesis := newBlock(t, newBlockConfig{}) + // genesisChild0 and genesisChild1 are siblings, both children of genesis. + // This can happen if we verify a block for round 1, but the network + // notarizes the dummy block. Then we will verify a sibling block for round + // 2 and must reject the round 1 block. + genesisChild0 := newBlock(t, newBlockConfig{ + prev: genesis, + }) + genesisChild1 := newBlock(t, newBlockConfig{ + prev: genesis, + round: genesisChild0.metadata.Round + 1, + }) + + // Verify the second block with the first block as its parent + _, err := genesisChild0.Verify(ctx) + require.NoError(t, err) + _, err = genesisChild1.Verify(ctx) + require.NoError(t, err) + + // When the we index the second block, the first block should be rejected + require.NoError(t, genesis.blockTracker.indexBlock(ctx, genesisChild1.digest)) + require.Equal(t, snowtest.Rejected, genesisChild0.vmBlock.(*snowmantest.Block).Decidable.Status) + require.Equal(t, snowtest.Accepted, genesisChild1.vmBlock.(*snowmantest.Block).Decidable.Status) + + _, exists := genesis.blockTracker.getBlockByDigest(genesis.digest) + require.False(t, exists) +} + +func TestVerifyInnerBlockBreaksHashChain(t *testing.T) { + ctx := context.Background() + + genesis := newBlock(t, newBlockConfig{}) + b := newBlock(t, newBlockConfig{ + prev: genesis, + }) + + // This block does not extend the genesis, however it has a valid previous + // digest. + b.vmBlock.(*snowmantest.Block).ParentV[0]++ + + _, err := b.Verify(ctx) + require.ErrorIs(t, err, errMismatchedPrevDigest) +} + +func TestIndexBlockDigestNotFound(t *testing.T) { + ctx := context.Background() + + genesis := newBlock(t, newBlockConfig{}) + + unknownDigest := ids.GenerateTestID() + err := genesis.blockTracker.indexBlock(ctx, simplex.Digest(unknownDigest)) + require.ErrorIs(t, err, errDigestNotFound) +} diff --git a/simplex/util_test.go b/simplex/util_test.go index 99d0b6d8cd8c..7d56e3904ac3 100644 --- a/simplex/util_test.go +++ b/simplex/util_test.go @@ -6,15 +6,69 @@ package simplex import ( "testing" + "github.com/ava-labs/simplex" "github.com/stretchr/testify/require" "github.com/ava-labs/avalanchego/ids" + "github.com/ava-labs/avalanchego/snow/consensus/snowman/snowmantest" "github.com/ava-labs/avalanchego/snow/validators" "github.com/ava-labs/avalanchego/utils/constants" "github.com/ava-labs/avalanchego/utils/crypto/bls/signer/localsigner" "github.com/ava-labs/avalanchego/utils/logging" ) +type newBlockConfig struct { + // If prev is nil, newBlock will create the genesis block + prev *Block + // If round is 0, it will be set to one higher than the prev's round + round uint64 +} + +func newBlock(t *testing.T, config newBlockConfig) *Block { + if config.prev == nil { + block := &Block{ + vmBlock: snowmantest.Genesis, + metadata: simplex.ProtocolMetadata{ + Version: 1, + Epoch: 1, + Round: 0, + Seq: 0, + }, + } + bytes, err := block.Bytes() + require.NoError(t, err) + + digest := computeDigest(bytes) + block.digest = digest + + block.blockTracker = newBlockTracker(block) + return block + } + if config.round == 0 { + config.round = config.prev.metadata.Round + 1 + } + + vmBlock := snowmantest.BuildChild(config.prev.vmBlock.(*snowmantest.Block)) + block := &Block{ + vmBlock: vmBlock, + blockTracker: config.prev.blockTracker, + metadata: simplex.ProtocolMetadata{ + Version: 1, + Epoch: 1, + Round: config.round, + Seq: vmBlock.Height(), + Prev: config.prev.digest, + }, + } + + bytes, err := block.Bytes() + require.NoError(t, err) + + digest := computeDigest(bytes) + block.digest = digest + return block +} + func newTestValidatorInfo(allNodes []*testNode) map[ids.NodeID]*validators.GetValidatorOutput { vds := make(map[ids.NodeID]*validators.GetValidatorOutput, len(allNodes)) for _, node := range allNodes { diff --git a/vms/proposervm/tree/tree.go b/utils/tree/tree.go similarity index 100% rename from vms/proposervm/tree/tree.go rename to utils/tree/tree.go diff --git a/vms/proposervm/tree/tree_test.go b/utils/tree/tree_test.go similarity index 100% rename from vms/proposervm/tree/tree_test.go rename to utils/tree/tree_test.go diff --git a/vms/proposervm/vm.go b/vms/proposervm/vm.go index f21558fddc68..2a491e40aedd 100644 --- a/vms/proposervm/vm.go +++ b/vms/proposervm/vm.go @@ -26,10 +26,10 @@ import ( "github.com/ava-labs/avalanchego/utils/constants" "github.com/ava-labs/avalanchego/utils/math" "github.com/ava-labs/avalanchego/utils/timer/mockable" + "github.com/ava-labs/avalanchego/utils/tree" "github.com/ava-labs/avalanchego/utils/units" "github.com/ava-labs/avalanchego/vms/proposervm/proposer" "github.com/ava-labs/avalanchego/vms/proposervm/state" - "github.com/ava-labs/avalanchego/vms/proposervm/tree" statelessblock "github.com/ava-labs/avalanchego/vms/proposervm/block" )