From 642a1e1d5a507d6080b2eac74baf9d13fcfd9488 Mon Sep 17 00:00:00 2001 From: Julien Robert Date: Thu, 10 Jul 2025 15:40:02 +0200 Subject: [PATCH 1/4] feat: fetch rollkitmngr state in migration command --- server/migration_cmd.go | 135 ++++++++++++++++++++++++++++++++++++---- 1 file changed, 123 insertions(+), 12 deletions(-) diff --git a/server/migration_cmd.go b/server/migration_cmd.go index ef41cdf..e2bcfae 100644 --- a/server/migration_cmd.go +++ b/server/migration_cmd.go @@ -9,15 +9,26 @@ import ( "path/filepath" "time" + "cosmossdk.io/collections" + "cosmossdk.io/log" + "cosmossdk.io/store" + "cosmossdk.io/store/metrics" + storetypes "cosmossdk.io/store/types" goheaderstore "github.com/celestiaorg/go-header/store" - dbm "github.com/cometbft/cometbft-db" + cmtdbm "github.com/cometbft/cometbft-db" cometbftcmd "github.com/cometbft/cometbft/cmd/cometbft/commands" cfg "github.com/cometbft/cometbft/config" "github.com/cometbft/cometbft/crypto" cmtjson "github.com/cometbft/cometbft/libs/json" + cmtproto "github.com/cometbft/cometbft/proto/tendermint/types" "github.com/cometbft/cometbft/state" - "github.com/cometbft/cometbft/store" - cometbfttypes "github.com/cometbft/cometbft/types" + cmtstore "github.com/cometbft/cometbft/store" + cmttypes "github.com/cometbft/cometbft/types" + dbm "github.com/cosmos/cosmos-db" + addresscodec "github.com/cosmos/cosmos-sdk/codec/address" + "github.com/cosmos/cosmos-sdk/runtime" + sdk "github.com/cosmos/cosmos-sdk/types" + moduletestutil "github.com/cosmos/cosmos-sdk/types/module/testutil" ds "github.com/ipfs/go-datastore" ktds "github.com/ipfs/go-datastore/keytransform" "github.com/spf13/cobra" @@ -27,6 +38,9 @@ import ( rollkitstore "github.com/rollkit/rollkit/pkg/store" rollkittypes "github.com/rollkit/rollkit/types" + "github.com/rollkit/go-execution-abci/modules/rollkitmngr" + rollkitmngrkeeper "github.com/rollkit/go-execution-abci/modules/rollkitmngr/keeper" + rollkitmngrtypes "github.com/rollkit/go-execution-abci/modules/rollkitmngr/types" "github.com/rollkit/go-execution-abci/pkg/adapter" ) @@ -187,7 +201,7 @@ After migration, start the node normally - it will automatically detect and use } // cometBlockToRollkit converts a cometBFT block to a rollkit block -func cometBlockToRollkit(block *cometbfttypes.Block) (*rollkittypes.SignedHeader, *rollkittypes.Data, rollkittypes.Signature) { +func cometBlockToRollkit(block *cmttypes.Block) (*rollkittypes.SignedHeader, *rollkittypes.Data, rollkittypes.Signature) { var ( header *rollkittypes.SignedHeader data *rollkittypes.Data @@ -241,26 +255,26 @@ func cometBlockToRollkit(block *cometbfttypes.Block) (*rollkittypes.SignedHeader return header, data, signature } -func loadStateAndBlockStore(config *cfg.Config) (*store.BlockStore, state.Store, error) { - dbType := dbm.BackendType(config.DBBackend) +func loadStateAndBlockStore(config *cfg.Config) (*cmtstore.BlockStore, state.Store, error) { + dbType := cmtdbm.BackendType(config.DBBackend) if ok, err := fileExists(filepath.Join(config.DBDir(), "blockstore.db")); !ok || err != nil { return nil, nil, fmt.Errorf("no blockstore found in %v: %w", config.DBDir(), err) } // Get BlockStore - blockStoreDB, err := dbm.NewDB("blockstore", dbType, config.DBDir()) + blockStoreDB, err := cmtdbm.NewDB("blockstore", dbType, config.DBDir()) if err != nil { return nil, nil, err } - blockStore := store.NewBlockStore(blockStoreDB) + blockStore := cmtstore.NewBlockStore(blockStoreDB) if ok, err := fileExists(filepath.Join(config.DBDir(), "state.db")); !ok || err != nil { return nil, nil, fmt.Errorf("no statestore found in %v: %w", config.DBDir(), err) } // Get StateStore - stateDB, err := dbm.NewDB("state", dbType, config.DBDir()) + stateDB, err := cmtdbm.NewDB("state", dbType, config.DBDir()) if err != nil { return nil, nil, err } @@ -346,10 +360,16 @@ func createRollkitMigrationGenesis(rootDir string, cometBFTState state.State) er if len(cometBFTState.LastValidators.Validators) == 1 { sequencerAddr = cometBFTState.LastValidators.Validators[0].Address.Bytes() sequencerPubKey = cometBFTState.LastValidators.Validators[0].PubKey + } else if len(cometBFTState.LastValidators.Validators) > 1 { + sequencer, err := getSequencerFromRollkitMngrState(rootDir, cometBFTState) + if err != nil { + return fmt.Errorf("failed to get sequencer from rollkitmngr state: %w", err) + } + + sequencerAddr = sequencer.Address + sequencerPubKey = sequencer.PubKey } else { - // TODO(@julienrbrt): Allow to use rollkitmngr state to get the sequencer address - // ref: https://github.com/rollkit/go-execution-abci/issues/164 - return fmt.Errorf("expected exactly one validator in the last validators, found %d", len(cometBFTState.LastValidators.Validators)) + return fmt.Errorf("no validators found in the last validators, cannot determine sequencer address") } migrationGenesis := rollkitMigrationGenesis{ @@ -374,6 +394,97 @@ func createRollkitMigrationGenesis(rootDir string, cometBFTState state.State) er return nil } +// sequencerInfo holds the sequencer information extracted from rollkitmngr state +type sequencerInfo struct { + Address []byte + PubKey crypto.PubKey +} + +// getSequencerFromRollkitMngrState attempts to load the sequencer information from the rollkitmngr module state +func getSequencerFromRollkitMngrState(rootDir string, cometBFTState state.State) (*sequencerInfo, error) { + config := cfg.DefaultConfig() + config.SetRoot(rootDir) + + dbType := dbm.BackendType(config.DBBackend) + + // Check if application database exists + appDBPath := filepath.Join(config.DBDir(), "application.db") + if ok, err := fileExists(appDBPath); err != nil { + return nil, fmt.Errorf("error checking application database in %v: %w", config.DBDir(), err) + } else if !ok { + return nil, fmt.Errorf("no application database found in %v", config.DBDir()) + } + + // Open application database + appDB, err := dbm.NewDB("application", dbType, config.DBDir()) + if err != nil { + return nil, fmt.Errorf("failed to open application database: %w", err) + } + defer func() { _ = appDB.Close() }() + + storeKey := storetypes.NewKVStoreKey(rollkitmngrtypes.ModuleName) + + cms := store.NewCommitMultiStore(appDB, log.NewNopLogger(), metrics.NewNoOpMetrics()) + cms.MountStoreWithDB(storeKey, storetypes.StoreTypeIAVL, appDB) + if err := cms.LoadLatestVersion(); err != nil { + return nil, fmt.Errorf("failed to load latest version of commit multi store: %w", err) + } + + k := rollkitmngrkeeper.NewKeeper( + moduletestutil.MakeTestEncodingConfig(rollkitmngr.AppModuleBasic{}).Codec, + runtime.NewKVStoreService(storeKey), + addresscodec.NewBech32Codec("cosmos"), + nil, + nil, + "", + ) + + ctx := sdk.NewContext(cms, cmtproto.Header{ + Height: int64(cometBFTState.LastBlockHeight), + ChainID: cometBFTState.ChainID, + Time: cometBFTState.LastBlockTime, + }, false, log.NewNopLogger()) + sequencer, err := k.Sequencer.Get(ctx) + if errors.Is(err, collections.ErrNotFound) { + return nil, fmt.Errorf("sequencer not found in rollkitmngr state, ensure the module is initialized and sequencer is set") + } else if err != nil { + return nil, fmt.Errorf("failed to get sequencer from rollkitmngr state: %w", err) + } + + // Extract the public key from the sequencer + pubKeyAny := sequencer.ConsensusPubkey + if pubKeyAny == nil { + return nil, fmt.Errorf("sequencer consensus public key is nil") + } + + // Get the cached value which should be a crypto.PubKey + pubKey, ok := pubKeyAny.GetCachedValue().(crypto.PubKey) + if !ok { + return nil, fmt.Errorf("failed to extract public key from sequencer") + } + + // Get the address from the public key + addr := pubKey.Address() + + // Validate that this sequencer is actually one of the validators + validatorFound := false + for _, validator := range cometBFTState.LastValidators.Validators { + if bytes.Equal(validator.Address.Bytes(), addr) { + validatorFound = true + break + } + } + + if !validatorFound { + return nil, fmt.Errorf("sequencer from rollkitmngr state (address: %x) is not found in the validator set", addr) + } + + return &sequencerInfo{ + Address: addr, + PubKey: pubKey, + }, nil +} + // fileExists checks if a file/directory exists. func fileExists(filename string) (bool, error) { _, err := os.Stat(filename) From b6ba0c60974a0697b5850aa924cab8fcb538c161 Mon Sep 17 00:00:00 2001 From: Julien Robert Date: Thu, 10 Jul 2025 15:41:55 +0200 Subject: [PATCH 2/4] go mod tidy --- go.mod | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/go.mod b/go.mod index 3ea0a34..cb4b93c 100644 --- a/go.mod +++ b/go.mod @@ -18,6 +18,7 @@ require ( github.com/celestiaorg/go-header v0.6.6 github.com/cometbft/cometbft v0.38.17 github.com/cometbft/cometbft-db v0.14.1 + github.com/cosmos/cosmos-db v1.1.1 github.com/cosmos/cosmos-proto v1.0.0-beta.5 github.com/cosmos/cosmos-sdk v0.50.14 github.com/cosmos/gogoproto v1.7.0 @@ -95,7 +96,6 @@ require ( github.com/containerd/stargz-snapshotter/estargz v0.16.3 // indirect github.com/coreos/go-systemd/v22 v22.5.0 // indirect github.com/cosmos/btcutil v1.0.5 // indirect - github.com/cosmos/cosmos-db v1.1.1 // indirect github.com/cosmos/go-bip39 v1.0.0 // indirect github.com/cosmos/gogogateway v1.2.0 // indirect github.com/cosmos/iavl v1.2.2 // indirect From 795bed248151bda49b5caed4a722f408be3155e1 Mon Sep 17 00:00:00 2001 From: Julien Robert Date: Thu, 10 Jul 2025 15:56:29 +0200 Subject: [PATCH 3/4] updates --- server/migration_cmd.go | 29 ++++++++++++++++------------- 1 file changed, 16 insertions(+), 13 deletions(-) diff --git a/server/migration_cmd.go b/server/migration_cmd.go index e2bcfae..d13fbe4 100644 --- a/server/migration_cmd.go +++ b/server/migration_cmd.go @@ -25,7 +25,7 @@ import ( cmtstore "github.com/cometbft/cometbft/store" cmttypes "github.com/cometbft/cometbft/types" dbm "github.com/cosmos/cosmos-db" - addresscodec "github.com/cosmos/cosmos-sdk/codec/address" + "github.com/cosmos/cosmos-sdk/codec" "github.com/cosmos/cosmos-sdk/runtime" sdk "github.com/cosmos/cosmos-sdk/types" moduletestutil "github.com/cosmos/cosmos-sdk/types/module/testutil" @@ -39,7 +39,6 @@ import ( rollkittypes "github.com/rollkit/rollkit/types" "github.com/rollkit/go-execution-abci/modules/rollkitmngr" - rollkitmngrkeeper "github.com/rollkit/go-execution-abci/modules/rollkitmngr/keeper" rollkitmngrtypes "github.com/rollkit/go-execution-abci/modules/rollkitmngr/types" "github.com/rollkit/go-execution-abci/pkg/adapter" ) @@ -424,33 +423,37 @@ func getSequencerFromRollkitMngrState(rootDir string, cometBFTState state.State) storeKey := storetypes.NewKVStoreKey(rollkitmngrtypes.ModuleName) + encCfg := moduletestutil.MakeTestEncodingConfig(rollkitmngr.AppModuleBasic{}) + sequencerCollection := collections.NewItem( + collections.NewSchemaBuilder(runtime.NewKVStoreService(storeKey)), + rollkitmngrtypes.SequencerKey, + "sequencer", + codec.CollValue[rollkitmngrtypes.Sequencer](encCfg.Codec), + ) + + // create context and commit multi-store cms := store.NewCommitMultiStore(appDB, log.NewNopLogger(), metrics.NewNoOpMetrics()) cms.MountStoreWithDB(storeKey, storetypes.StoreTypeIAVL, appDB) if err := cms.LoadLatestVersion(); err != nil { return nil, fmt.Errorf("failed to load latest version of commit multi store: %w", err) } - - k := rollkitmngrkeeper.NewKeeper( - moduletestutil.MakeTestEncodingConfig(rollkitmngr.AppModuleBasic{}).Codec, - runtime.NewKVStoreService(storeKey), - addresscodec.NewBech32Codec("cosmos"), - nil, - nil, - "", - ) - ctx := sdk.NewContext(cms, cmtproto.Header{ Height: int64(cometBFTState.LastBlockHeight), ChainID: cometBFTState.ChainID, Time: cometBFTState.LastBlockTime, }, false, log.NewNopLogger()) - sequencer, err := k.Sequencer.Get(ctx) + + sequencer, err := sequencerCollection.Get(ctx) if errors.Is(err, collections.ErrNotFound) { return nil, fmt.Errorf("sequencer not found in rollkitmngr state, ensure the module is initialized and sequencer is set") } else if err != nil { return nil, fmt.Errorf("failed to get sequencer from rollkitmngr state: %w", err) } + if err := sequencer.UnpackInterfaces(encCfg.InterfaceRegistry); err != nil { + return nil, fmt.Errorf("failed to unpack sequencer interfaces: %w", err) + } + // Extract the public key from the sequencer pubKeyAny := sequencer.ConsensusPubkey if pubKeyAny == nil { From 32fa0596e4fb242757790220fd121860b826d94b Mon Sep 17 00:00:00 2001 From: Julien Robert Date: Wed, 16 Jul 2025 21:28:29 +0200 Subject: [PATCH 4/4] updates --- server/migration_cmd_test.go | 513 +++++++++++++++++++++++++++++++++++ 1 file changed, 513 insertions(+) create mode 100644 server/migration_cmd_test.go diff --git a/server/migration_cmd_test.go b/server/migration_cmd_test.go new file mode 100644 index 0000000..fa131e2 --- /dev/null +++ b/server/migration_cmd_test.go @@ -0,0 +1,513 @@ +package server + +import ( + "testing" + "time" + + "github.com/cometbft/cometbft/crypto" + "github.com/cometbft/cometbft/crypto/ed25519" + cmtstate "github.com/cometbft/cometbft/proto/tendermint/state" + "github.com/cometbft/cometbft/proto/tendermint/version" + "github.com/cometbft/cometbft/state" + cmttypes "github.com/cometbft/cometbft/types" + "github.com/stretchr/testify/require" + + rollkittypes "github.com/rollkit/rollkit/types" +) + +func TestCometBlockToRollkit(t *testing.T) { + // create mock private key and address + privKey := ed25519.GenPrivKey() + pubKey := privKey.PubKey() + address := pubKey.Address() + + // create mock block + blockHeight := int64(100) + blockTime := time.Now() + chainID := "test-chain" + + // create mock transactions + txs := []cmttypes.Tx{ + []byte("tx1"), + []byte("tx2"), + []byte("transaction3"), + } + + // create mock block + block := &cmttypes.Block{ + Header: cmttypes.Header{ + Version: version.Consensus{ + Block: 11, + App: 1, + }, + ChainID: chainID, + Height: blockHeight, + Time: blockTime, + LastBlockID: cmttypes.BlockID{Hash: []byte("lastblockhash")}, + LastCommitHash: []byte("lastcommithash"), + DataHash: []byte("datahash"), + ValidatorsHash: []byte("validatorshash"), + NextValidatorsHash: []byte("nextvalidatorshash"), + ConsensusHash: []byte("consensushash"), + AppHash: []byte("apphash"), + LastResultsHash: []byte("lastresultshash"), + EvidenceHash: []byte("evidencehash"), + ProposerAddress: address, + }, + Data: cmttypes.Data{ + Txs: txs, + }, + Evidence: cmttypes.EvidenceData{}, + LastCommit: &cmttypes.Commit{ + Height: blockHeight - 1, + Round: 0, + BlockID: cmttypes.BlockID{ + Hash: []byte("previousblockhash"), + }, + Signatures: []cmttypes.CommitSig{ + { + BlockIDFlag: cmttypes.BlockIDFlagCommit, + ValidatorAddress: address, + Timestamp: blockTime, + Signature: []byte("signature"), + }, + }, + }, + } + + // call the function under test + header, data, signature := cometBlockToRollkit(block) + + // verify header conversion + require.NotNil(t, header) + require.Equal(t, uint64(blockHeight), header.Height()) + require.Equal(t, uint64(blockTime.UnixNano()), uint64(header.Time().UnixNano())) + require.Equal(t, chainID, header.ChainID()) + require.Equal(t, uint64(11), header.Version.Block) + require.Equal(t, uint64(1), header.Version.App) + require.Equal(t, []byte("lastblockhash"), []byte(header.LastHeaderHash)) + require.Equal(t, []byte("lastcommithash"), []byte(header.LastCommitHash)) + require.Equal(t, []byte("datahash"), []byte(header.DataHash)) + require.Equal(t, []byte("consensushash"), []byte(header.ConsensusHash)) + require.Equal(t, []byte("apphash"), []byte(header.AppHash)) + require.Equal(t, []byte("lastresultshash"), []byte(header.LastResultsHash)) + require.Equal(t, []byte("validatorshash"), []byte(header.ValidatorHash)) + require.Equal(t, address.Bytes(), header.ProposerAddress) + + // verify signature extraction + require.Equal(t, []byte("signature"), []byte(signature)) + require.Equal(t, []byte("signature"), []byte(header.Signature)) + + // verify data conversion + require.NotNil(t, data) + require.Equal(t, chainID, data.Metadata.ChainID) + require.Equal(t, uint64(blockHeight), data.Metadata.Height) + require.Equal(t, uint64(blockTime.UnixNano()), data.Metadata.Time) + require.Equal(t, []byte("datahash"), []byte(data.LastDataHash)) + + // verify transactions + require.Len(t, data.Txs, len(txs)) + for i, tx := range txs { + require.Equal(t, rollkittypes.Tx(tx), data.Txs[i]) + } +} + +func TestCometBlockToRollkitNoSignature(t *testing.T) { + // create mock private key and address + privKey := ed25519.GenPrivKey() + pubKey := privKey.PubKey() + address := pubKey.Address() + + // create different address for proposer + proposerPrivKey := ed25519.GenPrivKey() + proposerPubKey := proposerPrivKey.PubKey() + proposerAddress := proposerPubKey.Address() + + // create mock block with no matching signature + block := &cmttypes.Block{ + Header: cmttypes.Header{ + Version: version.Consensus{ + Block: 11, + App: 1, + }, + ChainID: "test-chain", + Height: 100, + Time: time.Now(), + ProposerAddress: proposerAddress, + }, + Data: cmttypes.Data{ + Txs: []cmttypes.Tx{}, + }, + Evidence: cmttypes.EvidenceData{}, + LastCommit: &cmttypes.Commit{ + Height: 99, + Round: 0, + Signatures: []cmttypes.CommitSig{ + { + BlockIDFlag: cmttypes.BlockIDFlagCommit, + ValidatorAddress: address, // different from proposer + Timestamp: time.Now(), + Signature: []byte("signature"), + }, + }, + }, + } + + // call the function under test + header, data, signature := cometBlockToRollkit(block) + + // verify signature is empty when proposer signature not found + require.Empty(t, signature) + require.Empty(t, header.Signature) + + // verify other fields are still converted correctly + require.Equal(t, uint64(100), header.Height()) + require.Equal(t, "test-chain", header.ChainID()) + require.NotNil(t, data) +} + +func TestCometbftStateToRollkitState(t *testing.T) { + // create mock cometbft state + chainID := "test-chain" + blockHeight := int64(100) + blockTime := time.Now() + daHeight := uint64(50) + + cometBFTState := state.State{ + Version: cmtstate.Version{ + Consensus: version.Consensus{ + Block: 11, + App: 1, + }, + }, + ChainID: chainID, + InitialHeight: 1, + LastBlockHeight: blockHeight, + LastBlockTime: blockTime, + LastResultsHash: []byte("lastresultshash"), + AppHash: []byte("apphash"), + } + + // call the function under test + rollkitState, err := cometbftStateToRollkitState(cometBFTState, daHeight) + + // verify no error + require.NoError(t, err) + + // verify state conversion + require.Equal(t, uint64(11), rollkitState.Version.Block) + require.Equal(t, uint64(1), rollkitState.Version.App) + require.Equal(t, chainID, rollkitState.ChainID) + require.Equal(t, uint64(blockHeight), rollkitState.InitialHeight) // should be migration height + require.Equal(t, uint64(blockHeight), rollkitState.LastBlockHeight) + require.Equal(t, blockTime, rollkitState.LastBlockTime) + require.Equal(t, daHeight, rollkitState.DAHeight) + require.Equal(t, []byte("lastresultshash"), []byte(rollkitState.LastResultsHash)) + require.Equal(t, []byte("apphash"), []byte(rollkitState.AppHash)) +} + +func TestRollkitMigrationGenesis(t *testing.T) { + // create mock private key and validator + privKey := ed25519.GenPrivKey() + pubKey := privKey.PubKey() + address := pubKey.Address() + + chainID := "test-chain" + initialHeight := int64(1) + blockTime := time.Now() + + // create mock validator set with single validator + validator := &cmttypes.Validator{ + Address: address, + PubKey: pubKey, + VotingPower: 100, + } + + validators := []*cmttypes.Validator{validator} + validatorSet := cmttypes.NewValidatorSet(validators) + + // create migration genesis + genesis := rollkitMigrationGenesis{ + ChainID: chainID, + InitialHeight: uint64(initialHeight), + GenesisTime: blockTime.UnixNano(), + SequencerAddr: address.Bytes(), + SequencerPubKey: pubKey, + } + + // convert to rollkit genesis + rollkitGenesis := genesis.ToRollkitGenesis() + + // verify conversion + require.NotNil(t, rollkitGenesis) + require.Equal(t, chainID, rollkitGenesis.ChainID) + require.Equal(t, uint64(initialHeight), rollkitGenesis.InitialHeight) + require.Equal(t, time.Unix(0, blockTime.UnixNano()), rollkitGenesis.GenesisDAStartTime) + require.Equal(t, address.Bytes(), rollkitGenesis.ProposerAddress) + + // verify validator set contains the expected validator + require.Equal(t, address, validatorSet.Validators[0].Address) + require.Equal(t, pubKey, validatorSet.Validators[0].PubKey) +} + +func TestCometBlockToRollkitWithMultipleSignatures(t *testing.T) { + // create multiple validators + privKey1 := ed25519.GenPrivKey() + pubKey1 := privKey1.PubKey() + address1 := pubKey1.Address() + + privKey2 := ed25519.GenPrivKey() + pubKey2 := privKey2.PubKey() + address2 := pubKey2.Address() + + privKey3 := ed25519.GenPrivKey() + pubKey3 := privKey3.PubKey() + address3 := pubKey3.Address() + + // create mock block with multiple signatures + block := &cmttypes.Block{ + Header: cmttypes.Header{ + Version: version.Consensus{ + Block: 11, + App: 1, + }, + ChainID: "test-chain", + Height: 100, + Time: time.Now(), + ProposerAddress: address2, // middle validator is proposer + }, + Data: cmttypes.Data{ + Txs: []cmttypes.Tx{[]byte("tx1")}, + }, + Evidence: cmttypes.EvidenceData{}, + LastCommit: &cmttypes.Commit{ + Height: 99, + Round: 0, + Signatures: []cmttypes.CommitSig{ + { + BlockIDFlag: cmttypes.BlockIDFlagCommit, + ValidatorAddress: address1, + Timestamp: time.Now(), + Signature: []byte("signature1"), + }, + { + BlockIDFlag: cmttypes.BlockIDFlagCommit, + ValidatorAddress: address2, // proposer signature + Timestamp: time.Now(), + Signature: []byte("proposer_signature"), + }, + { + BlockIDFlag: cmttypes.BlockIDFlagCommit, + ValidatorAddress: address3, + Timestamp: time.Now(), + Signature: []byte("signature3"), + }, + }, + }, + } + + // call the function under test + header, data, signature := cometBlockToRollkit(block) + + // verify correct proposer signature is extracted + require.Equal(t, []byte("proposer_signature"), []byte(signature)) + require.Equal(t, []byte("proposer_signature"), []byte(header.Signature)) + + // verify other fields + require.Equal(t, address2.Bytes(), header.ProposerAddress) + require.NotNil(t, data) + require.Len(t, data.Txs, 1) +} + +func TestCometBlockToRollkitEmptyTransactions(t *testing.T) { + // create mock private key and address + privKey := ed25519.GenPrivKey() + pubKey := privKey.PubKey() + address := pubKey.Address() + + // create mock block with no transactions + block := &cmttypes.Block{ + Header: cmttypes.Header{ + Version: version.Consensus{ + Block: 11, + App: 1, + }, + ChainID: "test-chain", + Height: 100, + Time: time.Now(), + ProposerAddress: address, + }, + Data: cmttypes.Data{ + Txs: []cmttypes.Tx{}, // empty transactions + }, + Evidence: cmttypes.EvidenceData{}, + LastCommit: &cmttypes.Commit{ + Height: 99, + Round: 0, + Signatures: []cmttypes.CommitSig{ + { + BlockIDFlag: cmttypes.BlockIDFlagCommit, + ValidatorAddress: address, + Timestamp: time.Now(), + Signature: []byte("signature"), + }, + }, + }, + } + + // call the function under test + header, data, signature := cometBlockToRollkit(block) + + // verify empty transactions are handled correctly + require.NotNil(t, header) + require.NotNil(t, data) + require.Empty(t, data.Txs) // should be empty slice, not nil + require.Equal(t, []byte("signature"), []byte(signature)) + + // verify metadata is still set correctly + require.Equal(t, "test-chain", data.Metadata.ChainID) + require.Equal(t, uint64(100), data.Metadata.Height) +} + +// helper function to create a mock cometbft state +func createMockCometBFTState(chainID string, height int64, validators []*cmttypes.Validator) state.State { + validatorSet := cmttypes.NewValidatorSet(validators) + + return state.State{ + Version: cmtstate.Version{ + Consensus: version.Consensus{ + Block: 11, + App: 1, + }, + }, + ChainID: chainID, + InitialHeight: 1, + LastBlockHeight: height, + LastBlockTime: time.Now(), + LastValidators: validatorSet, + Validators: validatorSet, + NextValidators: validatorSet, + LastResultsHash: []byte("lastresultshash"), + AppHash: []byte("apphash"), + } +} + +// helper function to create a mock validator +func createMockValidator() (*cmttypes.Validator, crypto.PrivKey) { + privKey := ed25519.GenPrivKey() + pubKey := privKey.PubKey() + address := pubKey.Address() + + validator := &cmttypes.Validator{ + Address: address, + PubKey: pubKey, + VotingPower: 100, + } + + return validator, privKey +} + +func TestMigrationWithSingleValidator(t *testing.T) { + // create single validator + validator, privKey := createMockValidator() + validators := []*cmttypes.Validator{validator} + + // create mock state + cometBFTState := createMockCometBFTState("test-chain", 100, validators) + + // test state conversion + rollkitState, err := cometbftStateToRollkitState(cometBFTState, 50) + require.NoError(t, err) + + // verify state + require.Equal(t, "test-chain", rollkitState.ChainID) + require.Equal(t, uint64(100), rollkitState.LastBlockHeight) + require.Equal(t, uint64(50), rollkitState.DAHeight) + + // create migration genesis + genesis := rollkitMigrationGenesis{ + ChainID: cometBFTState.ChainID, + InitialHeight: uint64(cometBFTState.InitialHeight), + GenesisTime: cometBFTState.LastBlockTime.UnixNano(), + SequencerAddr: validator.Address.Bytes(), + SequencerPubKey: validator.PubKey, + } + + // verify genesis + require.Equal(t, "test-chain", genesis.ChainID) + require.Equal(t, validator.Address.Bytes(), genesis.SequencerAddr) + require.Equal(t, validator.PubKey, genesis.SequencerPubKey) + + // verify rollkit genesis conversion + rollkitGenesis := genesis.ToRollkitGenesis() + require.Equal(t, "test-chain", rollkitGenesis.ChainID) + require.Equal(t, validator.Address.Bytes(), rollkitGenesis.ProposerAddress) + + // use privKey to avoid "declared but not used" error + _ = privKey +} + +func TestMigrationIntegration(t *testing.T) { + // create mock validator and state + validator, _ := createMockValidator() + validators := []*cmttypes.Validator{validator} + cometBFTState := createMockCometBFTState("integration-chain", 200, validators) + + // create mock block + blockTime := time.Now() + txs := []cmttypes.Tx{ + []byte("integration_tx1"), + []byte("integration_tx2"), + } + + block := &cmttypes.Block{ + Header: cmttypes.Header{ + Version: version.Consensus{ + Block: 11, + App: 1, + }, + ChainID: "integration-chain", + Height: 200, + Time: blockTime, + ProposerAddress: validator.Address, + DataHash: []byte("integration_datahash"), + }, + Data: cmttypes.Data{ + Txs: txs, + }, + LastCommit: &cmttypes.Commit{ + Height: 199, + Signatures: []cmttypes.CommitSig{ + { + BlockIDFlag: cmttypes.BlockIDFlagCommit, + ValidatorAddress: validator.Address, + Signature: []byte("integration_signature"), + }, + }, + }, + } + + // test block conversion + header, data, signature := cometBlockToRollkit(block) + + // verify block conversion + require.Equal(t, uint64(200), header.Height()) + require.Equal(t, "integration-chain", header.ChainID()) + require.Equal(t, []byte("integration_signature"), []byte(signature)) + require.Len(t, data.Txs, 2) + require.Equal(t, rollkittypes.Tx("integration_tx1"), data.Txs[0]) + require.Equal(t, rollkittypes.Tx("integration_tx2"), data.Txs[1]) + + // test state conversion + rollkitState, err := cometbftStateToRollkitState(cometBFTState, 100) + require.NoError(t, err) + + // verify state conversion + require.Equal(t, "integration-chain", rollkitState.ChainID) + require.Equal(t, uint64(200), rollkitState.LastBlockHeight) + require.Equal(t, uint64(100), rollkitState.DAHeight) + + // verify consistency between block and state + require.Equal(t, header.ChainID(), rollkitState.ChainID) + require.Equal(t, header.Height(), rollkitState.LastBlockHeight) +}