From 1551e6611f41c967751e5f4b8e1d53dc245b6dee Mon Sep 17 00:00:00 2001 From: JimboJ <40345116+jimjbrettj@users.noreply.github.com> Date: Mon, 24 Oct 2022 14:08:53 -0600 Subject: [PATCH] fix(core): fix txn pool for latest runtime (#2809) Fix transaction formatting, fix txnPool, and redo core service orchestrations for service tests --- dot/core/errors.go | 9 +- dot/core/helpers_test.go | 125 +++++++++++ dot/core/messages.go | 13 +- dot/core/messages_integration_test.go | 8 +- dot/core/messages_test.go | 38 +++- dot/core/service.go | 89 ++++++-- dot/core/service_integration_test.go | 150 +++---------- dot/core/service_test.go | 209 +++++++++++++----- dot/rpc/modules/author_integration_test.go | 70 ++++-- lib/blocktree/hashtoruntime_test.go | 2 - lib/blocktree/mocks_generate_test.go | 6 + .../{mock_instance_test.go => mocks_test.go} | 0 lib/common/hasher.go | 27 +++ lib/common/hasher_test.go | 6 + lib/runtime/version.go | 15 ++ lib/runtime/wasmer/exports.go | 2 +- 16 files changed, 539 insertions(+), 230 deletions(-) create mode 100644 lib/blocktree/mocks_generate_test.go rename lib/blocktree/{mock_instance_test.go => mocks_test.go} (100%) diff --git a/dot/core/errors.go b/dot/core/errors.go index 468a29032b..5e8ab8e693 100644 --- a/dot/core/errors.go +++ b/dot/core/errors.go @@ -8,17 +8,12 @@ import ( ) var ( - - // ErrServiceStopped is returned when the service has been stopped - ErrServiceStopped = errors.New("service has been stopped") - - // ErrInvalidBlock is returned when a block cannot be verified - ErrInvalidBlock = errors.New("could not verify block") - ErrNilRuntime = errors.New("cannot have nil runtime") ErrNilBlockHandlerParameter = errors.New("unable to handle block due to nil parameter") // ErrEmptyRuntimeCode is returned when the storage :code is empty ErrEmptyRuntimeCode = errors.New("new :code is empty") + + errInvalidTransactionQueueVersion = errors.New("invalid transaction queue version") ) diff --git a/dot/core/helpers_test.go b/dot/core/helpers_test.go index 419e4eb494..16793586b3 100644 --- a/dot/core/helpers_test.go +++ b/dot/core/helpers_test.go @@ -4,9 +4,11 @@ package core import ( + "bytes" "path/filepath" "testing" + "github.com/ChainSafe/gossamer/dot/network" "github.com/ChainSafe/gossamer/dot/state" "github.com/ChainSafe/gossamer/dot/types" "github.com/ChainSafe/gossamer/internal/log" @@ -19,10 +21,133 @@ import ( "github.com/ChainSafe/gossamer/lib/runtime/wasmer" "github.com/ChainSafe/gossamer/lib/trie" "github.com/ChainSafe/gossamer/lib/utils" + "github.com/ChainSafe/gossamer/pkg/scale" "github.com/golang/mock/gomock" "github.com/stretchr/testify/require" ) +func balanceKey(t *testing.T, pub []byte) (bKey []byte) { + t.Helper() + + h0, err := common.Twox128Hash([]byte("System")) + require.NoError(t, err) + h1, err := common.Twox128Hash([]byte("Account")) + require.NoError(t, err) + h2, err := common.Blake2b128(pub) + require.NoError(t, err) + return bytes.Join([][]byte{h0, h1, h2, pub}, nil) +} + +// Creates test service, used now for testing txnPool but can be used elsewhere when needed +func createTestService(t *testing.T, genesisFilePath string, + pubKey []byte, accountInfo types.AccountInfo, ctrl *gomock.Controller) (service *Service, encodedExtrinsic []byte) { + t.Helper() + + gen, err := genesis.NewGenesisFromJSONRaw(genesisFilePath) + require.NoError(t, err) + + genesisTrie, err := wasmer.NewTrieFromGenesis(*gen) + require.NoError(t, err) + + // Extrinsic and context related stuff + aliceBalanceKey := balanceKey(t, pubKey) + encodedAccountInfo, err := scale.Marshal(accountInfo) + require.NoError(t, err) + + genesisHeader := &types.Header{ + StateRoot: genesisTrie.MustHash(), + Number: 0, + } + + cfgKeystore := keystore.NewGlobalKeystore() + kp, err := sr25519.GenerateKeypair() + require.NoError(t, err) + err = cfgKeystore.Acco.Insert(kp) + require.NoError(t, err) + + // Create state service + var stateSrvc *state.Service + testDatadirPath := t.TempDir() + + // Set up block and storage state + telemetryMock := NewMockClient(ctrl) + telemetryMock.EXPECT().SendMessage(gomock.Any()).AnyTimes() + + stateConfig := state.Config{ + Path: testDatadirPath, + LogLevel: log.Critical, + Telemetry: telemetryMock, + } + + stateSrvc = state.NewService(stateConfig) + stateSrvc.UseMemDB() + + err = stateSrvc.Initialise(gen, genesisHeader, &genesisTrie) + require.NoError(t, err) + + // Start state service + err = stateSrvc.Start() + require.NoError(t, err) + + cfgBlockState := stateSrvc.Block + cfgStorageState := stateSrvc.Storage + cfgCodeSubstitutedState := stateSrvc.Base + + var rtCfg wasmer.Config + rtCfg.Storage = rtstorage.NewTrieState(&genesisTrie) + + rtCfg.CodeHash, err = cfgStorageState.LoadCodeHash(nil) + require.NoError(t, err) + + nodeStorage := runtime.NodeStorage{} + nodeStorage.BaseDB = stateSrvc.Base + + rtCfg.NodeStorage = nodeStorage + + cfgRuntime, err := wasmer.NewRuntimeFromGenesis(rtCfg) + require.NoError(t, err) + + cfgRuntime.(*wasmer.Instance).GetContext().Storage.Set(aliceBalanceKey, encodedAccountInfo) + // this key is System.UpgradedToDualRefCount -> set to true since all accounts have been upgraded to v0.9 format + cfgRuntime.(*wasmer.Instance).GetContext().Storage.Set(common.UpgradedToDualRefKey, []byte{1}) + + cfgBlockState.StoreRuntime(cfgBlockState.BestBlockHash(), cfgRuntime) + + // Hash of encrypted centrifuge extrinsic + testCallArguments := []byte{0xab, 0xcd} + extHex := runtime.NewTestExtrinsic(t, cfgRuntime, genesisHeader.Hash(), cfgBlockState.BestBlockHash(), + 0, "System.remark", testCallArguments) + encodedExtrinsic = common.MustHexToBytes(extHex) + + cfgCodeSubstitutes := make(map[common.Hash]string) + + genesisData, err := cfgCodeSubstitutedState.LoadGenesisData() + require.NoError(t, err) + + for k, v := range genesisData.CodeSubstitutes { + cfgCodeSubstitutes[common.MustHexToHash(k)] = v + } + + cfgCodeSubstitutedState = stateSrvc.Base + + cfg := &Config{ + Keystore: cfgKeystore, + LogLvl: log.Critical, + BlockState: cfgBlockState, + StorageState: cfgStorageState, + TransactionState: stateSrvc.Transaction, + EpochState: stateSrvc.Epoch, + CodeSubstitutedState: cfgCodeSubstitutedState, + Runtime: cfgRuntime, + Network: new(network.Service), + CodeSubstitutes: cfgCodeSubstitutes, + } + service, err = NewService(cfg) + require.NoError(t, err) + + return service, encodedExtrinsic +} + // NewTestService creates a new test core service func NewTestService(t *testing.T, cfg *Config) *Service { t.Helper() diff --git a/dot/core/messages.go b/dot/core/messages.go index 464ae5640f..155656d734 100644 --- a/dot/core/messages.go +++ b/dot/core/messages.go @@ -28,7 +28,11 @@ func (s *Service) validateTransaction(head *types.Header, rt RuntimeInstance, rt.SetContextStorage(ts) // validate each transaction - externalExt := types.Extrinsic(append([]byte{byte(types.TxnExternal)}, tx...)) + externalExt, err := s.buildExternalTransaction(rt, tx) + if err != nil { + return nil, fmt.Errorf("building external transaction: %w", err) + } + validity, err = rt.ValidateTransaction(externalExt) if err != nil { logger.Debugf("failed to validate transaction: %s", err) @@ -72,12 +76,10 @@ func (s *Service) HandleTransactionMessage(peerID peer.ID, msg *network.Transact allTxnsAreValid := true for _, tx := range txs { - txnIsValid := true validity, err := s.validateTransaction(head, rt, tx) if err != nil { - txnIsValid = false allTxnsAreValid = false - switch err := err.(type) { + switch err.(type) { case runtime.InvalidTransaction: s.net.ReportPeer(peerset.ReputationChange{ Value: peerset.BadTransactionValue, @@ -87,9 +89,10 @@ func (s *Service) HandleTransactionMessage(peerID peer.ID, msg *network.Transact default: return false, fmt.Errorf("validating transaction from peerID %s: %w", peerID, err) } + continue } - if txnIsValid && validity.Propagate { + if validity.Propagate { // find tx(s) that should propagate toPropagate = append(toPropagate, tx) } diff --git a/dot/core/messages_integration_test.go b/dot/core/messages_integration_test.go index c5569d9166..fbbedeb29d 100644 --- a/dot/core/messages_integration_test.go +++ b/dot/core/messages_integration_test.go @@ -88,11 +88,17 @@ func TestService_HandleBlockProduced(t *testing.T) { err = digest.Add(*prd) require.NoError(t, err) + // Used to define the state root of new block for testing + parentHash := s.blockState.GenesisHash() + genesisBlock, err := s.blockState.GetBlockByHash(parentHash) + require.NoError(t, err) + newBlock := types.Block{ Header: types.Header{ Number: 1, - ParentHash: s.blockState.BestBlockHash(), + ParentHash: parentHash, Digest: digest, + StateRoot: genesisBlock.Header.StateRoot, }, Body: *types.NewBody([]types.Extrinsic{}), } diff --git a/dot/core/messages_test.go b/dot/core/messages_test.go index 44476ffe03..75679c9483 100644 --- a/dot/core/messages_test.go +++ b/dot/core/messages_test.go @@ -4,6 +4,7 @@ package core import ( + "bytes" "errors" "testing" @@ -44,8 +45,9 @@ type mockGetRuntime struct { } type mockBlockState struct { - bestHeader *mockBestHeader - getRuntime *mockGetRuntime + bestHeader *mockBestHeader + getRuntime *mockGetRuntime + callsBestBlockHash bool } type mockStorageState struct { @@ -252,6 +254,7 @@ func TestServiceHandleTransactionMessage(t *testing.T) { getRuntime: &mockGetRuntime{ runtime: runtimeMock2, }, + callsBestBlockHash: true, }, mockStorageState: &mockStorageState{ input: &common.Hash{}, @@ -261,8 +264,12 @@ func TestServiceHandleTransactionMessage(t *testing.T) { runtime: runtimeMock2, setContextStorage: &mockSetContextStorage{trieState: &storage.TrieState{}}, validateTxn: &mockValidateTxn{ - input: types.Extrinsic(append([]byte{byte(types.TxnExternal)}, testExtrinsic[0]...)), - err: invalidTransaction, + input: types.Extrinsic(bytes.Join([][]byte{ + {byte(types.TxnExternal)}, + testExtrinsic[0], + testEmptyHeader.StateRoot.ToBytes(), + }, nil)), + err: invalidTransaction, }, }, args: args{ @@ -291,6 +298,7 @@ func TestServiceHandleTransactionMessage(t *testing.T) { getRuntime: &mockGetRuntime{ runtime: runtimeMock3, }, + callsBestBlockHash: true, }, mockStorageState: &mockStorageState{ input: &common.Hash{}, @@ -308,7 +316,11 @@ func TestServiceHandleTransactionMessage(t *testing.T) { runtime: runtimeMock3, setContextStorage: &mockSetContextStorage{trieState: &storage.TrieState{}}, validateTxn: &mockValidateTxn{ - input: types.Extrinsic(append([]byte{byte(types.TxnExternal)}, testExtrinsic[0]...)), + input: types.Extrinsic(bytes.Join([][]byte{ + {byte(types.TxnExternal)}, + testExtrinsic[0], + testEmptyHeader.StateRoot.ToBytes(), + }, nil)), validity: &transaction.Validity{Propagate: true}, }, }, @@ -344,6 +356,9 @@ func TestServiceHandleTransactionMessage(t *testing.T) { tt.mockBlockState.getRuntime.runtime, tt.mockBlockState.getRuntime.err) } + if tt.mockBlockState.callsBestBlockHash { + blockState.EXPECT().BestBlockHash().Return(common.Hash{}) + } s.blockState = blockState } if tt.mockStorageState != nil { @@ -365,6 +380,19 @@ func TestServiceHandleTransactionMessage(t *testing.T) { rt.EXPECT().SetContextStorage(tt.mockRuntime.setContextStorage.trieState) rt.EXPECT().ValidateTransaction(tt.mockRuntime.validateTxn.input). Return(tt.mockRuntime.validateTxn.validity, tt.mockRuntime.validateTxn.err) + rt.EXPECT().Version().Return(runtime.Version{ + SpecName: []byte("polkadot"), + ImplName: []byte("parity-polkadot"), + AuthoringVersion: authoringVersion, + SpecVersion: specVersion, + ImplVersion: implVersion, + APIItems: []runtime.APIItem{{ + Name: common.MustBlake2b8([]byte("TaggedTransactionQueue")), + Ver: 3, + }}, + TransactionVersion: transactionVersion, + StateVersion: stateVersion, + }) } res, err := s.HandleTransactionMessage(tt.args.peerID, tt.args.msg) diff --git a/dot/core/service.go b/dot/core/service.go index 833f63e984..528bda8c4f 100644 --- a/dot/core/service.go +++ b/dot/core/service.go @@ -22,6 +22,7 @@ import ( "github.com/ChainSafe/gossamer/lib/runtime/wasmer" "github.com/ChainSafe/gossamer/lib/services" "github.com/ChainSafe/gossamer/lib/transaction" + cscale "github.com/centrifuge/go-substrate-rpc-client/v4/scale" ctypes "github.com/centrifuge/go-substrate-rpc-client/v4/types" ) @@ -293,8 +294,6 @@ func (s *Service) handleCodeSubstitution(hash common.Hash, // does not need to be completed before the next block can be imported. func (s *Service) handleBlocksAsync() { for { - prev := s.blockState.BestBlockHash() - select { case block, ok := <-s.blockAddCh: if !ok { @@ -305,11 +304,16 @@ func (s *Service) handleBlocksAsync() { continue } - if err := s.handleChainReorg(prev, block.Header.Hash()); err != nil { - logger.Warnf("failed to re-add transactions to chain upon re-org: %s", err) + bestBlockHash := s.blockState.BestBlockHash() + if err := s.handleChainReorg(bestBlockHash, block.Header.Hash()); err != nil { + // TODO remove once gossamer is in stable state + panic(fmt.Errorf("failed to re-add transactions to chain upon re-org: %s", err)) } - s.maintainTransactionPool(block) + if err := s.maintainTransactionPool(block, bestBlockHash); err != nil { + // TODO remove once gossamer is in stable state + panic(fmt.Errorf("failed to maintain txn pool after re-org: %s", err)) + } case <-s.ctx.Done(): return } @@ -365,7 +369,7 @@ func (s *Service) handleChainReorg(prev, curr common.Hash) error { decExt := &ctypes.Extrinsic{} decoder := cscale.NewDecoder(bytes.NewReader(ext)) if err = decoder.Decode(&decExt); err != nil { - continue + return fmt.Errorf("decoding extrinsic: %s", err) } // Inherent are not signed. @@ -373,15 +377,18 @@ func (s *Service) handleChainReorg(prev, curr common.Hash) error { continue } - externalExt := make(types.Extrinsic, 0, 1+len(ext)) - externalExt = append(externalExt, byte(types.TxnExternal)) - externalExt = append(externalExt, ext...) - txv, err := rt.ValidateTransaction(externalExt) + externalExt, err := s.buildExternalTransaction(rt, ext) if err != nil { - logger.Debugf("failed to validate transaction for extrinsic %s: %s", ext, err) + return fmt.Errorf("building external transaction: %s", err) + } + + transactionValidity, err := rt.ValidateTransaction(externalExt) + if err != nil { + logger.Debugf("failed to validate transaction for extrinsic %s: %s skipping in chain reorg", ext, err) + s.transactionState.RemoveExtrinsic(ext) continue } - vtx := transaction.NewValidTransaction(ext, txv) + vtx := transaction.NewValidTransaction(ext, transactionValidity) s.transactionState.AddToPool(vtx) } } @@ -393,24 +400,41 @@ func (s *Service) handleChainReorg(prev, curr common.Hash) error { // the new block, revalidates the transactions in the pool, and moves // them to the queue if valid. // See https://github.com/paritytech/substrate/blob/74804b5649eccfb83c90aec87bdca58e5d5c8789/client/transaction-pool/src/lib.rs#L545 -func (s *Service) maintainTransactionPool(block *types.Block) { +func (s *Service) maintainTransactionPool(block *types.Block, bestBlockHash common.Hash) error { // remove extrinsics included in a block for _, ext := range block.Body { s.transactionState.RemoveExtrinsic(ext) } + stateRoot, err := s.storageState.GetStateRootFromBlock(&bestBlockHash) + if err != nil { + logger.Errorf("could not get state root from block %s: %w", bestBlockHash, err) + return err + } + + ts, err := s.storageState.TrieState(stateRoot) + if err != nil { + logger.Errorf(err.Error()) + return err + } + // re-validate transactions in the pool and move them to the queue txs := s.transactionState.PendingInPool() for _, tx := range txs { - // get the best block corresponding runtime - rt, err := s.blockState.GetRuntime(nil) + rt, err := s.blockState.GetRuntime(&bestBlockHash) if err != nil { - logger.Warnf("failed to get runtime to re-validate transactions in pool: %s", err) - continue + return fmt.Errorf("failed to get runtime to re-validate transactions in pool: %s", err) + } + + rt.SetContextStorage(ts) + externalExt, err := s.buildExternalTransaction(rt, tx.Extrinsic) + if err != nil { + return fmt.Errorf("building external transaction: %s", err) } - txnValidity, err := rt.ValidateTransaction(tx.Extrinsic) + txnValidity, err := rt.ValidateTransaction(externalExt) if err != nil { + logger.Debugf("failed to validate transaction for extrinsic %s: %s", tx.Extrinsic, err) s.transactionState.RemoveExtrinsic(tx.Extrinsic) continue } @@ -423,6 +447,7 @@ func (s *Service) maintainTransactionPool(block *types.Block) { s.transactionState.RemoveExtrinsicFromPool(tx.Extrinsic) logger.Tracef("moved transaction %s to queue", h) } + return nil } // InsertKey inserts keypair into the account keystore @@ -513,8 +538,12 @@ func (s *Service) HandleSubmittedExtrinsic(ext types.Extrinsic) error { } rt.SetContextStorage(ts) - // the transaction source is External - externalExt := types.Extrinsic(append([]byte{byte(types.TxnExternal)}, ext...)) + + externalExt, err := s.buildExternalTransaction(rt, ext) + if err != nil { + return fmt.Errorf("building external transaction: %w", err) + } + transactionValidity, err := rt.ValidateTransaction(externalExt) if err != nil { return err @@ -578,3 +607,23 @@ func (s *Service) GetReadProofAt(block common.Hash, keys [][]byte) ( return block, proofForKeys, nil } + +// buildExternalTransaction builds an external transaction based on the current transaction queue API version +// See https://github.com/paritytech/substrate/blob/polkadot-v0.9.25/primitives/transaction-pool/src/runtime_api.rs#L25-L55 +func (s *Service) buildExternalTransaction(rt runtime.Instance, ext types.Extrinsic) (types.Extrinsic, error) { + runtimeVersion := rt.Version() + txQueueVersion, err := runtimeVersion.TaggedTransactionQueueVersion() + if err != nil { + return nil, err + } + var extrinsicParts [][]byte + switch txQueueVersion { + case 3: + extrinsicParts = [][]byte{{byte(types.TxnExternal)}, ext, s.blockState.BestBlockHash().ToBytes()} + case 2: + extrinsicParts = [][]byte{{byte(types.TxnExternal)}, ext} + default: + return nil, fmt.Errorf("%w: %d", errInvalidTransactionQueueVersion, txQueueVersion) + } + return types.Extrinsic(bytes.Join(extrinsicParts, nil)), nil +} diff --git a/dot/core/service_integration_test.go b/dot/core/service_integration_test.go index e9638ed4aa..51b984eddf 100644 --- a/dot/core/service_integration_test.go +++ b/dot/core/service_integration_test.go @@ -17,87 +17,19 @@ import ( "github.com/ChainSafe/gossamer/dot/state" "github.com/ChainSafe/gossamer/dot/sync" "github.com/ChainSafe/gossamer/dot/types" - "github.com/ChainSafe/gossamer/internal/log" "github.com/ChainSafe/gossamer/lib/common" - "github.com/ChainSafe/gossamer/lib/genesis" "github.com/ChainSafe/gossamer/lib/keystore" "github.com/ChainSafe/gossamer/lib/runtime" rtstorage "github.com/ChainSafe/gossamer/lib/runtime/storage" "github.com/ChainSafe/gossamer/lib/runtime/wasmer" "github.com/ChainSafe/gossamer/lib/transaction" + "github.com/ChainSafe/gossamer/lib/trie" "github.com/ChainSafe/gossamer/lib/utils" "github.com/ChainSafe/gossamer/pkg/scale" "github.com/golang/mock/gomock" "github.com/stretchr/testify/require" ) -//go:generate mockgen -destination=mock_telemetry_test.go -package $GOPACKAGE github.com/ChainSafe/gossamer/dot/telemetry Client - -func balanceKey(t *testing.T, pub []byte) (bKey []byte) { - t.Helper() - - h0, err := common.Twox128Hash([]byte("System")) - require.NoError(t, err) - bKey = append(bKey, h0...) - h1, err := common.Twox128Hash([]byte("Account")) - require.NoError(t, err) - bKey = append(bKey, h1...) - h2, err := common.Blake2b128(pub) - require.NoError(t, err) - bKey = append(bKey, h2...) - bKey = append(bKey, pub...) - return -} - -func generateTestValidRemarkTxns(t *testing.T, pubKey []byte, accInfo types.AccountInfo) ([]byte, RuntimeInstance) { - t.Helper() - projectRootPath := utils.GetGssmrV3SubstrateGenesisRawPathTest(t) - gen, err := genesis.NewGenesisFromJSONRaw(projectRootPath) - require.NoError(t, err) - - genesisTrie, err := wasmer.NewTrieFromGenesis(*gen) - require.NoError(t, err) - - genState := rtstorage.NewTrieState(&genesisTrie) - - nodeStorage := runtime.NodeStorage{ - BaseDB: runtime.NewInMemoryDB(t), - } - cfg := wasmer.Config{ - Storage: genState, - LogLvl: log.Error, - NodeStorage: nodeStorage, - } - - rt, err := wasmer.NewRuntimeFromGenesis(cfg) - require.NoError(t, err) - - aliceBalanceKey := balanceKey(t, pubKey) - encBal, err := scale.Marshal(accInfo) - require.NoError(t, err) - - rt.(*wasmer.Instance).GetContext().Storage.Set(aliceBalanceKey, encBal) - // this key is System.UpgradedToDualRefCount -> set to true since all accounts have been upgraded to v0.9 format - rt.(*wasmer.Instance).GetContext().Storage.Set(common.UpgradedToDualRefKey, []byte{1}) - - genesisHeader := &types.Header{ - Number: 0, - StateRoot: genesisTrie.MustHash(), - } - - // Hash of encrypted centrifuge extrinsic - testCallArguments := []byte{0xab, 0xcd} - extHex := runtime.NewTestExtrinsic(t, rt, genesisHeader.Hash(), genesisHeader.Hash(), - 0, "System.remark", testCallArguments) - - extBytes := common.MustHexToBytes(extHex) - const txnType = byte(types.TxnExternal) - extBytes = append([]byte{txnType}, extBytes...) - - runtime.InitializeRuntimeToTest(t, rt, genesisHeader.Hash()) - return extBytes, rt -} - func TestStartService(t *testing.T) { s := NewTestService(t, nil) @@ -128,11 +60,16 @@ func TestAnnounceBlock(t *testing.T) { err = digest.Add(*prd) require.NoError(t, err) + // Used to define the state root of new block for testing + genesisBlock, err := s.blockState.GetBlockByHash(s.blockState.GenesisHash()) + require.NoError(t, err) + newBlock := types.Block{ Header: types.Header{ Number: 1, ParentHash: s.blockState.BestBlockHash(), Digest: digest, + StateRoot: genesisBlock.Header.StateRoot, }, Body: *types.NewBody([]types.Extrinsic{}), } @@ -408,9 +345,7 @@ func TestHandleChainReorg_WithReorg_Transactions(t *testing.T) { require.Equal(t, transaction.NewValidTransaction(tx, validity), pending[0]) } -// TODO: add test against latest gssmr runtime -// See https://github.com/ChainSafe/gossamer/issues/2702 -func TestMaintainTransactionPool_EmptyBlock(t *testing.T) { +func TestMaintainTransactionPoolLatestTxnQueue_EmptyBlock(t *testing.T) { accountInfo := types.AccountInfo{ Nonce: 0, Data: types.AccountData{ @@ -423,30 +358,23 @@ func TestMaintainTransactionPool_EmptyBlock(t *testing.T) { keyring, err := keystore.NewSr25519Keyring() require.NoError(t, err) alicePub := common.MustHexToBytes(keyring.Alice().Public().Hex()) - encExt, runtimeInstance := generateTestValidRemarkTxns(t, alicePub, accountInfo) - cfg := &Config{ - Runtime: runtimeInstance, - } + genesisFilePath, err := utils.GetGssmrGenesisRawPath() + require.NoError(t, err) ctrl := gomock.NewController(t) - telemetryMock := NewMockClient(ctrl) - telemetryMock.EXPECT().SendMessage(gomock.Any()).AnyTimes() + service, encExt := createTestService(t, genesisFilePath, alicePub, accountInfo, ctrl) - transactionState := state.NewTransactionState(telemetryMock) tx := &transaction.ValidTransaction{ Extrinsic: types.Extrinsic(encExt), Validity: &transaction.Validity{Priority: 1}, } - _ = transactionState.AddToPool(tx) - - service := NewTestService(t, cfg) - service.transactionState = transactionState + _ = service.transactionState.AddToPool(tx) // provides is a list of transaction hashes that depend on this tx, see: // https://github.com/paritytech/substrate/blob/5420de3face1349a97eb954ae71c5b0b940c31de/core/sr-primitives/src/transaction_validity.rs#L195 provides := common.MustHexToBytes("0xd43593c715fdd31c61141abd04a99fd6822c8558854ccde39a5684e7a56da27d00000000") txnValidity := &transaction.Validity{ - Priority: 39325240425794630, + Priority: 36074, Provides: [][]byte{provides}, Longevity: 18446744073709551614, Propagate: true, @@ -454,21 +382,21 @@ func TestMaintainTransactionPool_EmptyBlock(t *testing.T) { expectedTx := transaction.NewValidTransaction(tx.Extrinsic, txnValidity) - service.maintainTransactionPool(&types.Block{ + bestBlockHash := service.blockState.BestBlockHash() + err = service.maintainTransactionPool(&types.Block{ Body: *types.NewBody([]types.Extrinsic{}), - }) + }, bestBlockHash) + require.NoError(t, err) - resultTx := transactionState.Pop() + resultTx := service.transactionState.(*state.TransactionState).Pop() require.Equal(t, expectedTx, resultTx) - transactionState.RemoveExtrinsic(tx.Extrinsic) - head := transactionState.Pop() + service.transactionState.RemoveExtrinsic(tx.Extrinsic) + head := service.transactionState.(*state.TransactionState).Pop() require.Nil(t, head) } -// TODO: add test against latest gssmr runtime -// See https://github.com/ChainSafe/gossamer/issues/2702 -func TestMaintainTransactionPool_BlockWithExtrinsics(t *testing.T) { +func TestMaintainTransactionPoolLatestTxnQueue_BlockWithExtrinsics(t *testing.T) { accountInfo := types.AccountInfo{ Nonce: 0, Data: types.AccountData{ @@ -481,39 +409,32 @@ func TestMaintainTransactionPool_BlockWithExtrinsics(t *testing.T) { keyring, err := keystore.NewSr25519Keyring() require.NoError(t, err) alicePub := common.MustHexToBytes(keyring.Alice().Public().Hex()) - extrinsicBytes, _ := generateTestValidRemarkTxns(t, alicePub, accountInfo) + genesisFilePath, err := utils.GetGssmrGenesisRawPath() + require.NoError(t, err) ctrl := gomock.NewController(t) - telemetryMock := NewMockClient(ctrl) - telemetryMock.EXPECT().SendMessage(gomock.Any()).AnyTimes() + service, encodedExtrinsic := createTestService(t, genesisFilePath, alicePub, accountInfo, ctrl) - ts := state.NewTransactionState(telemetryMock) - - // Maybe replace validity tx := &transaction.ValidTransaction{ - Extrinsic: types.Extrinsic(extrinsicBytes), + Extrinsic: types.Extrinsic(encodedExtrinsic), Validity: &transaction.Validity{Priority: 1}, } + _ = service.transactionState.AddToPool(tx) - ts.AddToPool(tx) - - s := &Service{ - transactionState: ts, - } - - s.maintainTransactionPool(&types.Block{ - Body: types.Body([]types.Extrinsic{extrinsicBytes}), - }) + bestBlockHash := service.blockState.BestBlockHash() + err = service.maintainTransactionPool(&types.Block{ + Body: types.Body([]types.Extrinsic{encodedExtrinsic}), + }, bestBlockHash) + require.NoError(t, err) res := []*transaction.ValidTransaction{} for { - tx := ts.Pop() + tx := service.transactionState.(*state.TransactionState).Pop() if tx == nil { break } res = append(res, tx) } - // Extrinsic is removed. so empty res require.Empty(t, res) } @@ -523,7 +444,6 @@ func TestService_GetRuntimeVersion(t *testing.T) { require.NoError(t, err) rtExpected := rt.Version() - rtv, err := s.GetRuntimeVersion(nil) require.NoError(t, err) require.Equal(t, rtExpected, rtv) @@ -554,7 +474,6 @@ func TestService_HandleSubmittedExtrinsic(t *testing.T) { require.NoError(t, err) extBytes := createExtrinsic(t, rt, genHeader.Hash(), 0) - err = s.HandleSubmittedExtrinsic(extBytes) require.NoError(t, err) } @@ -577,7 +496,6 @@ func TestService_HandleRuntimeChanges(t *testing.T) { require.NoError(t, err) v := rt.Version() - currSpecVersion := v.SpecVersion // genesis runtime version. hash := s.blockState.BestBlockHash() // genesisHash @@ -661,8 +579,7 @@ func TestService_HandleCodeSubstitutes(t *testing.T) { s.blockState.StoreRuntime(blockHash, rt) - ts := rtstorage.NewTrieState(nil) - + ts := rtstorage.NewTrieState(trie.NewEmptyTrie()) err = s.handleCodeSubstitution(blockHash, ts) require.NoError(t, err) codSub := s.codeSubstitutedState.LoadCodeSubstitutedBlockHash() @@ -689,8 +606,7 @@ func TestService_HandleRuntimeChangesAfterCodeSubstitutes(t *testing.T) { Body: *body, } - ts := rtstorage.NewTrieState(nil) - + ts := rtstorage.NewTrieState(trie.NewEmptyTrie()) err = s.handleCodeSubstitution(blockHash, ts) require.NoError(t, err) require.Equal(t, codeHashBefore, parentRt.GetCodeHash()) // codeHash should remain unchanged after code substitute diff --git a/dot/core/service_test.go b/dot/core/service_test.go index bb98124291..93ece3ba05 100644 --- a/dot/core/service_test.go +++ b/dot/core/service_test.go @@ -8,7 +8,6 @@ import ( "context" "encoding/hex" "errors" - "fmt" "testing" "github.com/ChainSafe/gossamer/dot/network" @@ -42,6 +41,7 @@ const ( specVersion = 25 implVersion = 0 transactionVersion = 0 + stateVersion = 0 ) func generateTestCentrifugeMetadata(t *testing.T) *ctypes.Metadata { @@ -112,7 +112,12 @@ func generateExtrinsic(t *testing.T) (extrinsic, externalExtrinsic types.Extrins require.NoError(t, err) encExt := []types.Extrinsic{extEnc.Bytes()} - testExternalExt := types.Extrinsic(append([]byte{byte(types.TxnExternal)}, encExt[0]...)) + testHeader := types.NewEmptyHeader() + testExternalExt := types.Extrinsic(bytes.Join([][]byte{ + {byte(types.TxnExternal)}, + encExt[0], + testHeader.StateRoot.ToBytes(), + }, nil)) testUnencryptedBody := types.NewBody(encExt) return encExt[0], testExternalExt, testUnencryptedBody } @@ -556,21 +561,48 @@ func Test_Service_maintainTransactionPool(t *testing.T) { } extrinsic := types.Extrinsic{21} + externalExt := types.Extrinsic(bytes.Join([][]byte{ + {byte(types.TxnExternal)}, + extrinsic, + testHeader.StateRoot.ToBytes(), + }, nil)) vt := transaction.NewValidTransaction(extrinsic, validity) ctrl := gomock.NewController(t) runtimeMock := NewMockRuntimeInstance(ctrl) - runtimeMock.EXPECT().ValidateTransaction(types.Extrinsic{21}).Return(nil, errTestDummyError) + runtimeMock.EXPECT().ValidateTransaction(externalExt).Return(nil, errTestDummyError) + runtimeMock.EXPECT().Version().Return(runtime.Version{ + SpecName: []byte("polkadot"), + ImplName: []byte("parity-polkadot"), + AuthoringVersion: authoringVersion, + SpecVersion: specVersion, + ImplVersion: implVersion, + APIItems: []runtime.APIItem{{ + Name: common.MustBlake2b8([]byte("TaggedTransactionQueue")), + Ver: 3, + }}, + TransactionVersion: transactionVersion, + StateVersion: stateVersion, + }) + runtimeMock.EXPECT().SetContextStorage(&rtstorage.TrieState{}) + mockTxnState := NewMockTransactionState(ctrl) mockTxnState.EXPECT().RemoveExtrinsic(types.Extrinsic{21}).Times(2) mockTxnState.EXPECT().PendingInPool().Return([]*transaction.ValidTransaction{vt}) mockBlockState := NewMockBlockState(ctrl) - mockBlockState.EXPECT().GetRuntime(nil).Return(runtimeMock, nil) + mockBlockState.EXPECT().GetRuntime(&common.Hash{1}).Return(runtimeMock, nil) + mockBlockState.EXPECT().BestBlockHash().Return(common.Hash{}) + + mockStorageState := NewMockStorageState(ctrl) + mockStorageState.EXPECT().TrieState(&common.Hash{1}).Return(&rtstorage.TrieState{}, nil) + mockStorageState.EXPECT().GetStateRootFromBlock(&common.Hash{1}).Return(&common.Hash{1}, nil) service := &Service{ transactionState: mockTxnState, blockState: mockBlockState, + storageState: mockStorageState, } - service.maintainTransactionPool(&block) + err := service.maintainTransactionPool(&block, common.Hash{1}) + require.NoError(t, err) }) t.Run("Validate Transaction ok", func(t *testing.T) { @@ -589,26 +621,52 @@ func Test_Service_maintainTransactionPool(t *testing.T) { Propagate: true, } - extrinsic := types.Extrinsic{21} - vt := transaction.NewValidTransaction(extrinsic, validity) - tx := transaction.NewValidTransaction(types.Extrinsic{21}, &transaction.Validity{Propagate: true}) + ext := types.Extrinsic{21} + externalExt := types.Extrinsic(bytes.Join([][]byte{ + {byte(types.TxnExternal)}, + ext, + testHeader.StateRoot.ToBytes(), + }, nil)) + vt := transaction.NewValidTransaction(ext, validity) + tx := transaction.NewValidTransaction(ext, &transaction.Validity{Propagate: true}) ctrl := gomock.NewController(t) runtimeMock := NewMockRuntimeInstance(ctrl) - runtimeMock.EXPECT().ValidateTransaction(types.Extrinsic{21}). - Return(&transaction.Validity{Propagate: true}, nil) + runtimeMock.EXPECT().ValidateTransaction(externalExt).Return(&transaction.Validity{Propagate: true}, nil) + runtimeMock.EXPECT().Version().Return(runtime.Version{ + SpecName: []byte("polkadot"), + ImplName: []byte("parity-polkadot"), + AuthoringVersion: authoringVersion, + SpecVersion: specVersion, + ImplVersion: implVersion, + APIItems: []runtime.APIItem{{ + Name: common.MustBlake2b8([]byte("TaggedTransactionQueue")), + Ver: 3, + }}, + TransactionVersion: transactionVersion, + StateVersion: stateVersion, + }) + runtimeMock.EXPECT().SetContextStorage(&rtstorage.TrieState{}) mockTxnState := NewMockTransactionState(ctrl) mockTxnState.EXPECT().RemoveExtrinsic(types.Extrinsic{21}) mockTxnState.EXPECT().PendingInPool().Return([]*transaction.ValidTransaction{vt}) mockTxnState.EXPECT().Push(tx).Return(common.Hash{}, nil) mockTxnState.EXPECT().RemoveExtrinsicFromPool(types.Extrinsic{21}) + mockBlockStateOk := NewMockBlockState(ctrl) - mockBlockStateOk.EXPECT().GetRuntime(nil).Return(runtimeMock, nil) + mockBlockStateOk.EXPECT().GetRuntime(&common.Hash{1}).Return(runtimeMock, nil) + mockBlockStateOk.EXPECT().BestBlockHash().Return(common.Hash{}) + + mockStorageState := NewMockStorageState(ctrl) + mockStorageState.EXPECT().TrieState(&common.Hash{1}).Return(&rtstorage.TrieState{}, nil) + mockStorageState.EXPECT().GetStateRootFromBlock(&common.Hash{1}).Return(&common.Hash{1}, nil) service := &Service{ transactionState: mockTxnState, blockState: mockBlockStateOk, + storageState: mockStorageState, } - service.maintainTransactionPool(&block) + err := service.maintainTransactionPool(&block, common.Hash{1}) + require.NoError(t, err) }) } @@ -616,14 +674,10 @@ func Test_Service_handleBlocksAsync(t *testing.T) { t.Parallel() t.Run("cancelled context", func(t *testing.T) { t.Parallel() - ctrl := gomock.NewController(t) - mockBlockState := NewMockBlockState(ctrl) - mockBlockState.EXPECT().BestBlockHash().Return(common.Hash{}) blockAddChan := make(chan *types.Block) ctx, cancel := context.WithCancel(context.Background()) cancel() service := &Service{ - blockState: mockBlockState, blockAddCh: blockAddChan, ctx: ctx, } @@ -632,13 +686,9 @@ func Test_Service_handleBlocksAsync(t *testing.T) { t.Run("channel not ok", func(t *testing.T) { t.Parallel() - ctrl := gomock.NewController(t) - mockBlockState := NewMockBlockState(ctrl) - mockBlockState.EXPECT().BestBlockHash().Return(common.Hash{}) blockAddChan := make(chan *types.Block) close(blockAddChan) service := &Service{ - blockState: mockBlockState, blockAddCh: blockAddChan, ctx: context.Background(), } @@ -647,16 +697,12 @@ func Test_Service_handleBlocksAsync(t *testing.T) { t.Run("nil block", func(t *testing.T) { t.Parallel() - ctrl := gomock.NewController(t) - mockBlockState := NewMockBlockState(ctrl) - mockBlockState.EXPECT().BestBlockHash().Return(common.Hash{}).Times(2) blockAddChan := make(chan *types.Block) go func() { blockAddChan <- nil close(blockAddChan) }() service := &Service{ - blockState: mockBlockState, blockAddCh: blockAddChan, ctx: context.Background(), } @@ -665,46 +711,30 @@ func Test_Service_handleBlocksAsync(t *testing.T) { t.Run("handleChainReorg error", func(t *testing.T) { t.Parallel() - validity := &transaction.Validity{ - Priority: 0x3e8, - Requires: [][]byte{{0xb5, 0x47, 0xb1, 0x90, 0x37, 0x10, 0x7e, 0x1f, 0x79, 0x4c, - 0xa8, 0x69, 0x0, 0xa1, 0xb5, 0x98}}, - Provides: [][]byte{{0xe4, 0x80, 0x7d, 0x1b, 0x67, 0x49, 0x37, 0xbf, 0xc7, 0x89, - 0xbb, 0xdd, 0x88, 0x6a, 0xdd, 0xd6}}, - Longevity: 0x40, - Propagate: true, - } - - extrinsic := types.Extrinsic{21} - vt := transaction.NewValidTransaction(extrinsic, validity) testHeader := types.NewEmptyHeader() block := types.NewBlock(*testHeader, *types.NewBody([]types.Extrinsic{[]byte{21}})) block.Header.Number = 21 ctrl := gomock.NewController(t) - runtimeMock := NewMockRuntimeInstance(ctrl) - runtimeMock.EXPECT().ValidateTransaction(types.Extrinsic{21}).Return(nil, errTestDummyError) mockBlockState := NewMockBlockState(ctrl) - mockBlockState.EXPECT().BestBlockHash().Return(common.Hash{}).Times(2) + mockBlockState.EXPECT().BestBlockHash().Return(common.Hash{}) mockBlockState.EXPECT().HighestCommonAncestor(common.Hash{}, block.Header.Hash()). Return(common.Hash{}, errTestDummyError) - mockBlockState.EXPECT().GetRuntime(nil).Return(runtimeMock, nil) - mockTxnStateErr := NewMockTransactionState(ctrl) - mockTxnStateErr.EXPECT().RemoveExtrinsic(types.Extrinsic{21}).Times(2) - mockTxnStateErr.EXPECT().PendingInPool().Return([]*transaction.ValidTransaction{vt}) + blockAddChan := make(chan *types.Block) go func() { blockAddChan <- &block close(blockAddChan) }() service := &Service{ - blockState: mockBlockState, - transactionState: mockTxnStateErr, - blockAddCh: blockAddChan, - ctx: context.Background(), + blockState: mockBlockState, + blockAddCh: blockAddChan, + ctx: context.Background(), } - service.handleBlocksAsync() + + assert.PanicsWithError(t, "failed to re-add transactions to chain upon re-org: test dummy error", + service.handleBlocksAsync) }) } @@ -712,7 +742,6 @@ func TestService_handleChainReorg(t *testing.T) { t.Parallel() execTest := func(t *testing.T, s *Service, prevHash common.Hash, currHash common.Hash, expErr error) { err := s.handleChainReorg(prevHash, currHash) - assert.ErrorIs(t, err, expErr) if expErr != nil { assert.EqualError(t, err, expErr.Error()) } @@ -813,7 +842,23 @@ func TestService_handleChainReorg(t *testing.T) { t.Run("invalid transaction", func(t *testing.T) { t.Parallel() ctrl := gomock.NewController(t) + runtimeMockErr := NewMockRuntimeInstance(ctrl) + runtimeMockErr.EXPECT().ValidateTransaction(externExt).Return(nil, errTestDummyError) + runtimeMockErr.EXPECT().Version().Return(runtime.Version{ + SpecName: []byte("polkadot"), + ImplName: []byte("parity-polkadot"), + AuthoringVersion: authoringVersion, + SpecVersion: specVersion, + ImplVersion: implVersion, + APIItems: []runtime.APIItem{{ + Name: common.MustBlake2b8([]byte("TaggedTransactionQueue")), + Ver: 3, + }}, + TransactionVersion: transactionVersion, + StateVersion: stateVersion, + }) + mockBlockState := NewMockBlockState(ctrl) mockBlockState.EXPECT().HighestCommonAncestor(testPrevHash, testCurrentHash). Return(testAncestorHash, nil) @@ -821,11 +866,15 @@ func TestService_handleChainReorg(t *testing.T) { mockBlockState.EXPECT().GetRuntime(nil).Return(runtimeMockErr, nil) mockBlockState.EXPECT().GetBlockBody(testCurrentHash).Return(nil, errDummyErr) mockBlockState.EXPECT().GetBlockBody(testAncestorHash).Return(body, nil) - runtimeMockErr.EXPECT().ValidateTransaction(externExt).Return(nil, errTestDummyError) + mockBlockState.EXPECT().BestBlockHash().Return(common.Hash{}) + mockTxnState := NewMockTransactionState(ctrl) + mockTxnState.EXPECT().RemoveExtrinsic(ext) service := &Service{ - blockState: mockBlockState, + blockState: mockBlockState, + transactionState: mockTxnState, } + execTest(t, service, testPrevHash, testCurrentHash, nil) }) @@ -833,6 +882,21 @@ func TestService_handleChainReorg(t *testing.T) { t.Parallel() ctrl := gomock.NewController(t) runtimeMockOk := NewMockRuntimeInstance(ctrl) + runtimeMockOk.EXPECT().ValidateTransaction(externExt).Return(testValidity, nil) + runtimeMockOk.EXPECT().Version().Return(runtime.Version{ + SpecName: []byte("polkadot"), + ImplName: []byte("parity-polkadot"), + AuthoringVersion: authoringVersion, + SpecVersion: specVersion, + ImplVersion: implVersion, + APIItems: []runtime.APIItem{{ + Name: common.MustBlake2b8([]byte("TaggedTransactionQueue")), + Ver: 3, + }}, + TransactionVersion: transactionVersion, + StateVersion: stateVersion, + }) + mockBlockState := NewMockBlockState(ctrl) mockBlockState.EXPECT().HighestCommonAncestor(testPrevHash, testCurrentHash). Return(testAncestorHash, nil) @@ -840,7 +904,7 @@ func TestService_handleChainReorg(t *testing.T) { mockBlockState.EXPECT().GetRuntime(nil).Return(runtimeMockOk, nil) mockBlockState.EXPECT().GetBlockBody(testCurrentHash).Return(nil, errDummyErr) mockBlockState.EXPECT().GetBlockBody(testAncestorHash).Return(body, nil) - runtimeMockOk.EXPECT().ValidateTransaction(externExt).Return(testValidity, nil) + mockBlockState.EXPECT().BestBlockHash().Return(common.Hash{}) mockTxnStateOk := NewMockTransactionState(ctrl) mockTxnStateOk.EXPECT().AddToPool(vtx).Return(common.Hash{}) @@ -1091,8 +1155,12 @@ func TestServiceGetRuntimeVersion(t *testing.T) { func TestServiceHandleSubmittedExtrinsic(t *testing.T) { t.Parallel() ext := types.Extrinsic{} - externalExt := types.Extrinsic(append([]byte{byte(types.TxnExternal)}, ext...)) - fmt.Println(ext) + testHeader := types.NewEmptyHeader() + externalExt := types.Extrinsic(bytes.Join([][]byte{ + {byte(types.TxnExternal)}, + ext, + testHeader.StateRoot.ToBytes(), + }, nil)) execTest := func(t *testing.T, s *Service, ext types.Extrinsic, expErr error) { err := s.HandleSubmittedExtrinsic(ext) assert.ErrorIs(t, err, expErr) @@ -1157,6 +1225,7 @@ func TestServiceHandleSubmittedExtrinsic(t *testing.T) { mockBlockState.EXPECT().BestBlockHash().Return(common.Hash{}) runtimeMockErr := NewMockRuntimeInstance(ctrl) mockBlockState.EXPECT().GetRuntime(&common.Hash{}).Return(runtimeMockErr, nil).MaxTimes(2) + mockBlockState.EXPECT().BestBlockHash().Return(common.Hash{}) mockStorageState := NewMockStorageState(ctrl) mockStorageState.EXPECT().TrieState(&common.Hash{}).Return(&rtstorage.TrieState{}, nil) @@ -1165,8 +1234,21 @@ func TestServiceHandleSubmittedExtrinsic(t *testing.T) { mockTxnState := NewMockTransactionState(ctrl) mockTxnState.EXPECT().Exists(types.Extrinsic{}) - runtimeMockErr.EXPECT().SetContextStorage(&rtstorage.TrieState{}) runtimeMockErr.EXPECT().ValidateTransaction(externalExt).Return(nil, errDummyErr) + runtimeMockErr.EXPECT().Version().Return(runtime.Version{ + SpecName: []byte("polkadot"), + ImplName: []byte("parity-polkadot"), + AuthoringVersion: authoringVersion, + SpecVersion: specVersion, + ImplVersion: implVersion, + APIItems: []runtime.APIItem{{ + Name: common.MustBlake2b8([]byte("TaggedTransactionQueue")), + Ver: 3, + }}, + TransactionVersion: transactionVersion, + StateVersion: stateVersion, + }) + runtimeMockErr.EXPECT().SetContextStorage(&rtstorage.TrieState{}) service := &Service{ storageState: mockStorageState, transactionState: mockTxnState, @@ -1184,8 +1266,23 @@ func TestServiceHandleSubmittedExtrinsic(t *testing.T) { mockBlockState := NewMockBlockState(ctrl) mockBlockState.EXPECT().BestBlockHash().Return(common.Hash{}) mockBlockState.EXPECT().GetRuntime(&common.Hash{}).Return(runtimeMock, nil).MaxTimes(2) - runtimeMock.EXPECT().SetContextStorage(&rtstorage.TrieState{}) + mockBlockState.EXPECT().BestBlockHash().Return(common.Hash{}) + runtimeMock.EXPECT().ValidateTransaction(externalExt).Return(&transaction.Validity{Propagate: true}, nil) + runtimeMock.EXPECT().Version().Return(runtime.Version{ + SpecName: []byte("polkadot"), + ImplName: []byte("parity-polkadot"), + AuthoringVersion: authoringVersion, + SpecVersion: specVersion, + ImplVersion: implVersion, + APIItems: []runtime.APIItem{{ + Name: common.MustBlake2b8([]byte("TaggedTransactionQueue")), + Ver: 3, + }}, + TransactionVersion: transactionVersion, + StateVersion: stateVersion, + }) + runtimeMock.EXPECT().SetContextStorage(&rtstorage.TrieState{}) mockStorageState := NewMockStorageState(ctrl) mockStorageState.EXPECT().TrieState(&common.Hash{}).Return(&rtstorage.TrieState{}, nil) diff --git a/dot/rpc/modules/author_integration_test.go b/dot/rpc/modules/author_integration_test.go index 7aac7a3b26..709ec1add8 100644 --- a/dot/rpc/modules/author_integration_test.go +++ b/dot/rpc/modules/author_integration_test.go @@ -6,6 +6,7 @@ package modules import ( + "bytes" "context" "errors" "fmt" @@ -29,6 +30,12 @@ import ( "github.com/ChainSafe/gossamer/lib/runtime/wasmer" "github.com/ChainSafe/gossamer/lib/transaction" "github.com/ChainSafe/gossamer/lib/trie" + "github.com/ChainSafe/gossamer/pkg/scale" + cscale "github.com/centrifuge/go-substrate-rpc-client/v4/scale" + "github.com/centrifuge/go-substrate-rpc-client/v4/signature" + ctypes "github.com/centrifuge/go-substrate-rpc-client/v4/types" + "github.com/centrifuge/go-substrate-rpc-client/v4/types/codec" + "github.com/golang/mock/gomock" "github.com/stretchr/testify/require" ) @@ -79,6 +86,48 @@ func useInstanceFromRuntimeV0910(t *testing.T, rtStorage *storage.TrieState) (in return runtimeInstance } +func createExtrinsic(t *testing.T, rt runtime.Instance, genHash common.Hash, nonce uint64) types.Extrinsic { + t.Helper() + rawMeta, err := rt.Metadata() + require.NoError(t, err) + + var decoded []byte + err = scale.Unmarshal(rawMeta, &decoded) + require.NoError(t, err) + + meta := &ctypes.Metadata{} + + err = codec.Decode(decoded, meta) + require.NoError(t, err) + + runtimeVersion := rt.Version() + + metaCall, err := ctypes.NewCall(meta, "System.remark", []byte{0xab, 0xcd}) + require.NoError(t, err) + + extrinsic := ctypes.NewExtrinsic(metaCall) + options := ctypes.SignatureOptions{ + BlockHash: ctypes.Hash(genHash), + Era: ctypes.ExtrinsicEra{IsImmortalEra: false}, + GenesisHash: ctypes.Hash(genHash), + Nonce: ctypes.NewUCompactFromUInt(nonce), + SpecVersion: ctypes.U32(runtimeVersion.SpecVersion), + Tip: ctypes.NewUCompactFromUInt(0), + TransactionVersion: ctypes.U32(runtimeVersion.TransactionVersion), + } + + // Sign the transaction using Alice's key + err = extrinsic.Sign(signature.TestKeyringPairAlice, options) + require.NoError(t, err) + + extEnc := bytes.NewBuffer(nil) + encoder := cscale.NewEncoder(extEnc) + err = extrinsic.Encode(*encoder) + require.NoError(t, err) + + return extEnc.Bytes() +} + func TestAuthorModule_Pending_Integration(t *testing.T) { t.Parallel() integrationTestController := setupStateAndRuntime(t, t.TempDir(), nil) @@ -510,19 +559,11 @@ func TestAuthorModule_SubmitExtrinsic_WithVersion_V0910(t *testing.T) { integrationTestController.stateSrv.Transaction = state.NewTransactionState(telemetryMock) genesisHash := integrationTestController.genesisHeader.Hash() - - extHex := runtime.NewTestExtrinsic(t, - integrationTestController.runtime, genesisHash, genesisHash, 1, "System.remark", []byte{0xab, 0xcd}) - - // to extrinsic works with a runtime version 0910 we need to - // append the block hash bytes at the end of the extrinsics - hashBytes := genesisHash.ToBytes() - extBytes := append(common.MustHexToBytes(extHex), hashBytes...) - - extHex = common.BytesToHex(extBytes) + extrinsic := createExtrinsic(t, integrationTestController.runtime, genesisHash, 0) + extHex := common.BytesToHex(extrinsic) net2test := NewMockNetwork(ctrl) - net2test.EXPECT().GossipMessage(&network.TransactionMessage{Extrinsics: []types.Extrinsic{extBytes}}) + net2test.EXPECT().GossipMessage(&network.TransactionMessage{Extrinsics: []types.Extrinsic{extrinsic}}) integrationTestController.network = net2test // setup auth module @@ -532,16 +573,13 @@ func TestAuthorModule_SubmitExtrinsic_WithVersion_V0910(t *testing.T) { err := auth.SubmitExtrinsic(nil, &Extrinsic{extHex}, res) require.NoError(t, err) - expectedExtrinsic := types.NewExtrinsic(extBytes) + expectedExtrinsic := types.NewExtrinsic(extrinsic) expected := &transaction.ValidTransaction{ Extrinsic: expectedExtrinsic, Validity: &transaction.Validity{ Priority: 4295664014726, - Requires: [][]byte{ - common.MustHexToBytes("0xd43593c715fdd31c61141abd04a99fd6822c8558854ccde39a5684e7a56da27d00000000"), - }, Provides: [][]byte{ - common.MustHexToBytes("0xd43593c715fdd31c61141abd04a99fd6822c8558854ccde39a5684e7a56da27d01000000"), + common.MustHexToBytes("0xd43593c715fdd31c61141abd04a99fd6822c8558854ccde39a5684e7a56da27d00000000"), }, Longevity: 18446744073709551613, Propagate: true, diff --git a/lib/blocktree/hashtoruntime_test.go b/lib/blocktree/hashtoruntime_test.go index d5af93fa56..74f0f1c7ac 100644 --- a/lib/blocktree/hashtoruntime_test.go +++ b/lib/blocktree/hashtoruntime_test.go @@ -25,8 +25,6 @@ func Test_newHashToRuntime(t *testing.T) { assert.Equal(t, expected, hti) } -//go:generate mockgen -destination=mock_instance_test.go -package $GOPACKAGE github.com/ChainSafe/gossamer/lib/runtime Instance - func Test_hashToRuntime_get(t *testing.T) { t.Parallel() diff --git a/lib/blocktree/mocks_generate_test.go b/lib/blocktree/mocks_generate_test.go new file mode 100644 index 0000000000..b28c8d756b --- /dev/null +++ b/lib/blocktree/mocks_generate_test.go @@ -0,0 +1,6 @@ +// Copyright 2022 ChainSafe Systems (ON) +// SPDX-License-Identifier: LGPL-3.0-only + +package blocktree + +//go:generate mockgen -destination=mocks_test.go -package $GOPACKAGE github.com/ChainSafe/gossamer/lib/runtime Instance diff --git a/lib/blocktree/mock_instance_test.go b/lib/blocktree/mocks_test.go similarity index 100% rename from lib/blocktree/mock_instance_test.go rename to lib/blocktree/mocks_test.go diff --git a/lib/common/hasher.go b/lib/common/hasher.go index e9bc0732f2..003754736a 100644 --- a/lib/common/hasher.go +++ b/lib/common/hasher.go @@ -27,6 +27,33 @@ func Blake2b128(in []byte) ([]byte, error) { return h.Sum(nil), nil } +// Blake2b8 returns the first 8 bytes of the Blake2b hash of the input data +func Blake2b8(data []byte) (digest [8]byte, err error) { + const bytes = 8 + hasher, err := blake2b.New(bytes, nil) + if err != nil { + return [8]byte{}, err + } + + _, err = hasher.Write(data) + if err != nil { + return [8]byte{}, err + } + + digestBytes := hasher.Sum(nil) + copy(digest[:], digestBytes) + return digest, nil +} + +// MustBlake2b8 returns the first 8 bytes of the Blake2b hash of the input data +func MustBlake2b8(data []byte) (digest [8]byte) { + digest, err := Blake2b8(data) + if err != nil { + panic(err) + } + return digest +} + // Blake2bHash returns the 256-bit blake2b hash of the input data func Blake2bHash(in []byte) (Hash, error) { h, err := blake2b.New256(nil) diff --git a/lib/common/hasher_test.go b/lib/common/hasher_test.go index c347838030..b1ae212327 100644 --- a/lib/common/hasher_test.go +++ b/lib/common/hasher_test.go @@ -33,6 +33,12 @@ func TestBlake128(t *testing.T) { require.Equal(t, expected, h) } +func Test_MustBlake2b8(t *testing.T) { + expectedDigest := [8]byte{0xd2, 0xbc, 0x98, 0x97, 0xee, 0xd0, 0x8f, 0x15} + digest := common.MustBlake2b8([]byte("TaggedTransactionQueue")) + require.Equal(t, expectedDigest, digest) +} + func TestBlake2bHash_EmptyHash(t *testing.T) { // test case from https://github.com/noot/blake2b_test which uses the blake2-rfp rust crate // also see https://github.com/paritytech/substrate/blob/master/core/primitives/src/hashing.rs diff --git a/lib/runtime/version.go b/lib/runtime/version.go index 349b10003d..277b3b71b0 100644 --- a/lib/runtime/version.go +++ b/lib/runtime/version.go @@ -9,6 +9,7 @@ import ( "fmt" "io" + "github.com/ChainSafe/gossamer/lib/common" "github.com/ChainSafe/gossamer/pkg/scale" ) @@ -34,6 +35,20 @@ var ( ErrDecodingVersionField = errors.New("decoding version field") ) +// TaggedTransactionQueueVersion returns the TaggedTransactionQueue API version +func (v Version) TaggedTransactionQueueVersion() (txQueueVersion uint32, err error) { + encodedTaggedTransactionQueue, err := common.Blake2b8([]byte("TaggedTransactionQueue")) + if err != nil { + return 0, fmt.Errorf("getting blake2b8: %s", err) + } + for _, apiItem := range v.APIItems { + if apiItem.Name == encodedTaggedTransactionQueue { + return apiItem.Ver, nil + } + } + return 0, errors.New("taggedTransactionQueueAPI not found") +} + // DecodeVersion scale decodes the encoded version data. // For older version data with missing fields (such as `transaction_version`) // the missing field is set to its zero value (such as `0`). diff --git a/lib/runtime/wasmer/exports.go b/lib/runtime/wasmer/exports.go index 4ad2038ae0..e723c22f5f 100644 --- a/lib/runtime/wasmer/exports.go +++ b/lib/runtime/wasmer/exports.go @@ -13,7 +13,7 @@ import ( ) // ValidateTransaction runs the extrinsic through the runtime function -// TaggedTransactionQueue_validate_transaction and returns **transaction.Validity. The error can +// TaggedTransactionQueue_validate_transaction and returns *transaction.Validity. The error can // be a VDT of either transaction.InvalidTransaction or transaction.UnknownTransaction, or can represent // a normal error i.e. unmarshalling error func (in *Instance) ValidateTransaction(e types.Extrinsic) (