From 4d371b5fc4be2726f5aa50faf8c86def82704130 Mon Sep 17 00:00:00 2001 From: Ethan Reesor Date: Sat, 16 Sep 2023 17:06:27 -0500 Subject: [PATCH] Use a binary file for genesis [#3397] --- cmd/accumulated/cmd_init.go | 8 +- cmd/accumulated/cmd_init_network.go | 3 + go.mod | 2 +- .../core/execute/v1/simulator/simulator.go | 5 +- internal/database/snapshot.go | 22 +++- internal/node/abci/accumulator.go | 4 +- internal/node/daemon/init.go | 89 ++----------- internal/node/daemon/run.go | 4 +- internal/node/genesis/bootstrap.go | 118 +++++++++++------- internal/node/genesis/provider.go | 95 ++++++++++++++ internal/node/genesis/types.go | 69 ++++++++++ internal/node/genesis/types.yml | 27 ++++ internal/node/genesis/types_gen.go | 96 ++++++++++++++ .../keyvalue/badger/changeset_test.go | 2 +- pkg/database/keyvalue/badger/database.go | 14 +-- pkg/database/snapshot/enums.yml | 3 + pkg/database/snapshot/enums_gen.go | 9 +- pkg/database/snapshot/format.go | 10 ++ test/simulator/options.go | 12 +- vdk/node/node.go | 8 +- 20 files changed, 443 insertions(+), 157 deletions(-) create mode 100644 internal/node/genesis/provider.go create mode 100644 internal/node/genesis/types.go create mode 100644 internal/node/genesis/types.yml create mode 100644 internal/node/genesis/types_gen.go diff --git a/cmd/accumulated/cmd_init.go b/cmd/accumulated/cmd_init.go index 1fd9db865..440904cd6 100644 --- a/cmd/accumulated/cmd_init.go +++ b/cmd/accumulated/cmd_init.go @@ -33,6 +33,7 @@ import ( "gitlab.com/accumulatenetwork/accumulate/internal/logging" cfg "gitlab.com/accumulatenetwork/accumulate/internal/node/config" accumulated "gitlab.com/accumulatenetwork/accumulate/internal/node/daemon" + "gitlab.com/accumulatenetwork/accumulate/internal/node/genesis" client "gitlab.com/accumulatenetwork/accumulate/pkg/client/api/v2" "gitlab.com/accumulatenetwork/accumulate/pkg/proxy" "gitlab.com/accumulatenetwork/accumulate/protocol" @@ -679,7 +680,12 @@ func initNode(cmd *cobra.Command, args []string) (string, error) { return "", fmt.Errorf("load/generate node key files, %v", err) } - err = accumulated.WriteNodeFiles(config, privValKey, nodeKey, genDoc) + config.Genesis = "config/genesis.snap" + genDocBytes, err := genesis.ConvertJsonToSnapshot(genDoc) + if err != nil { + return "", fmt.Errorf("write node files, %v", err) + } + err = accumulated.WriteNodeFiles(config, privValKey, nodeKey, genDocBytes) if err != nil { return "", fmt.Errorf("write node files, %v", err) } diff --git a/cmd/accumulated/cmd_init_network.go b/cmd/accumulated/cmd_init_network.go index 2781a1040..d0d2dc7ba 100644 --- a/cmd/accumulated/cmd_init_network.go +++ b/cmd/accumulated/cmd_init_network.go @@ -141,6 +141,9 @@ func initNetworkLocalFS(cmd *cobra.Command, netInit *accumulated.NetworkInit) { for _, configs := range configs { for _, configs := range configs { for _, config := range configs { + // Use binary genesis files + config.Genesis = "config/genesis.snap" + if flagInit.LogLevels != "" { config.LogLevel = flagInit.LogLevels } diff --git a/go.mod b/go.mod index a241507fa..957d22157 100644 --- a/go.mod +++ b/go.mod @@ -45,6 +45,7 @@ require ( github.com/FactomProject/factomd v1.13.0 github.com/chzyer/readline v1.5.1 github.com/cometbft/cometbft v0.38.0-rc3 + github.com/cosmos/gogoproto v1.4.11 github.com/ethereum/go-ethereum v1.12.2 github.com/fatih/astrewrite v0.0.0-20191207154002-9094e544fcef github.com/gobeam/stringy v0.0.6 @@ -85,7 +86,6 @@ require ( github.com/chigopher/pathlib v0.12.0 // indirect github.com/cometbft/cometbft-db v0.7.0 // indirect github.com/containerd/cgroups v1.1.0 // indirect - github.com/cosmos/gogoproto v1.4.11 // indirect github.com/curioswitch/go-reassign v0.2.0 // indirect github.com/davidlazar/go-crypto v0.0.0-20200604182044-b73af7476f6c // indirect github.com/decred/dcrd/dcrec/secp256k1/v4 v4.2.0 // indirect diff --git a/internal/core/execute/v1/simulator/simulator.go b/internal/core/execute/v1/simulator/simulator.go index a57e5ae0d..a10d24bbf 100644 --- a/internal/core/execute/v1/simulator/simulator.go +++ b/internal/core/execute/v1/simulator/simulator.go @@ -11,7 +11,6 @@ package simulator import ( "bytes" "context" - "encoding/json" "fmt" "io" "sync" @@ -324,9 +323,7 @@ func (s *Simulator) InitFromGenesisWith(values *core.GlobalValues) { // Execute bootstrap after the entire network is known for _, x := range s.Executors { - var snap []byte - require.NoError(s, json.Unmarshal(genDocs[x.Partition.Id].AppState, &snap)) - require.NoError(s, snapshot.FullRestore(x.Database, ioutil2.NewBuffer(snap), x.Executor.Logger, x.Executor.Describe.PartitionUrl())) + require.NoError(s, snapshot.FullRestore(x.Database, ioutil2.NewBuffer(genDocs[x.Partition.Id]), x.Executor.Logger, x.Executor.Describe.PartitionUrl())) require.NoError(s, x.Executor.Init(x.Database)) } } diff --git a/internal/database/snapshot.go b/internal/database/snapshot.go index 1065f9596..6575262f9 100644 --- a/internal/database/snapshot.go +++ b/internal/database/snapshot.go @@ -16,6 +16,7 @@ import ( "gitlab.com/accumulatenetwork/accumulate/internal/database/record" "gitlab.com/accumulatenetwork/accumulate/internal/database/smt/storage" "gitlab.com/accumulatenetwork/accumulate/pkg/database" + kvb "gitlab.com/accumulatenetwork/accumulate/pkg/database/keyvalue/badger" "gitlab.com/accumulatenetwork/accumulate/pkg/database/snapshot" "gitlab.com/accumulatenetwork/accumulate/pkg/database/values" "gitlab.com/accumulatenetwork/accumulate/pkg/errors" @@ -29,8 +30,9 @@ const collectIndexTxnPrefix = "txn." const collectIndexRecordPrefix = "rec." type CollectOptions struct { - BuildIndex bool - Predicate func(database.Record) (bool, error) + BuildIndex bool + Predicate func(database.Record) (bool, error) + DidWriteHeader func(*snapshot.Writer) error Metrics *CollectMetrics } @@ -59,6 +61,13 @@ func (db *Database) Collect(file io.WriteSeeker, partition *url.URL, opts *Colle return errors.UnknownError.Wrap(err) } + if opts.DidWriteHeader != nil { + err = opts.DidWriteHeader(w) + if err != nil { + return errors.UnknownError.Wrap(err) + } + } + // Collect the BPT err = db.collectBPT(w, opts) if err != nil { @@ -74,7 +83,7 @@ func (db *Database) Collect(file io.WriteSeeker, partition *url.URL, opts *Colle _ = os.RemoveAll(dir) }(dir) - index, err := badger.Open(badger.DefaultOptions(dir)) + index, err := badger.Open(badger.DefaultOptions(dir).WithLogger(kvb.Slogger{})) if err != nil { return errors.UnknownError.Wrap(err) } @@ -226,9 +235,10 @@ func (db *Database) collectMessages(w *snapshot.Writer, index *badger.DB, opts * return errors.UnknownError.WithFormat("collect %x: %w", hash, err) } - // Collect the transaction status (which is the only part of the - // transaction entity that is still used by exec v2) - err = records.Collect(batch.newTransaction(transactionKey{Hash: hash}).newStatus(), copts) + // Collect the transaction's records. Executor v2 only uses the + // transaction status, but transactions and signatures from v1 are still + // stored here, so they should be collected. + err = records.Collect(batch.newTransaction(transactionKey{Hash: hash}), copts) if err != nil { return errors.UnknownError.WithFormat("collect %x status: %w", hash, err) } diff --git a/internal/node/abci/accumulator.go b/internal/node/abci/accumulator.go index 6db47247c..163c78c10 100644 --- a/internal/node/abci/accumulator.go +++ b/internal/node/abci/accumulator.go @@ -21,7 +21,6 @@ import ( "github.com/cometbft/cometbft/crypto" "github.com/cometbft/cometbft/libs/log" protocrypto "github.com/cometbft/cometbft/proto/tendermint/crypto" - "github.com/cometbft/cometbft/types" "github.com/cometbft/cometbft/version" "gitlab.com/accumulatenetwork/accumulate" "gitlab.com/accumulatenetwork/accumulate/exp/ioutil" @@ -32,6 +31,7 @@ import ( "gitlab.com/accumulatenetwork/accumulate/internal/database/snapshot" "gitlab.com/accumulatenetwork/accumulate/internal/logging" "gitlab.com/accumulatenetwork/accumulate/internal/node/config" + "gitlab.com/accumulatenetwork/accumulate/internal/node/genesis" "gitlab.com/accumulatenetwork/accumulate/pkg/errors" "gitlab.com/accumulatenetwork/accumulate/pkg/types/messaging" "gitlab.com/accumulatenetwork/accumulate/protocol" @@ -242,7 +242,7 @@ func (app *Accumulator) Info(context.Context, *abci.RequestInfo) (*abci.Response } // Check the genesis document - genDoc, err := types.GenesisDocFromFile(app.Config.GenesisFile()) + genDoc, err := genesis.DocProvider(app.Config)() if err != nil { return nil, err } diff --git a/internal/node/daemon/init.go b/internal/node/daemon/init.go index 56ad0011e..c30aff3dc 100644 --- a/internal/node/daemon/init.go +++ b/internal/node/daemon/init.go @@ -8,7 +8,6 @@ package accumulated import ( "bytes" - "encoding/json" "errors" "fmt" "io" @@ -20,7 +19,6 @@ import ( "github.com/cometbft/cometbft/crypto/ed25519" tmed25519 "github.com/cometbft/cometbft/crypto/ed25519" - tmbytes "github.com/cometbft/cometbft/libs/bytes" tmjson "github.com/cometbft/cometbft/libs/json" "github.com/cometbft/cometbft/libs/log" tmos "github.com/cometbft/cometbft/libs/os" @@ -210,83 +208,34 @@ func ConfigureNodePorts(node *NodeInit, cfg *config.Config, part protocol.Partit cfg.Accumulate.API.ListenAddress = node.Listen().Scheme("http").PartitionType(part).AccumulateAPI().String() } -func BuildGenesisDocs(network *NetworkInit, globals *core.GlobalValues, time time.Time, logger log.Logger, factomAddresses func() (io.Reader, error), snapshots []func() (ioutil2.SectionReader, error)) (map[string]*tmtypes.GenesisDoc, error) { - docs := map[string]*tmtypes.GenesisDoc{} +func BuildGenesisDocs(network *NetworkInit, globals *core.GlobalValues, time time.Time, logger log.Logger, factomAddresses func() (io.Reader, error), snapshots []func() (ioutil2.SectionReader, error)) (map[string][]byte, error) { + docs := map[string][]byte{} var operators [][]byte netinfo := new(protocol.NetworkDefinition) netinfo.NetworkName = network.Id netinfo.AddPartition(protocol.Directory, protocol.PartitionTypeDirectory) - var dnTmValidators []tmtypes.GenesisValidator - var i int for _, bvn := range network.Bvns { - var bvnTmValidators []tmtypes.GenesisValidator + netinfo.AddPartition(bvn.Id, protocol.PartitionTypeBlockValidator) - for j, node := range bvn.Nodes { + for _, node := range bvn.Nodes { i++ key := tmed25519.PrivKey(node.PrivValKey) operators = append(operators, key.PubKey().Bytes()) - netinfo.AddValidator(key.PubKey().Bytes(), protocol.Directory, node.DnnType == config.Validator) netinfo.AddValidator(key.PubKey().Bytes(), bvn.Id, node.BvnnType == config.Validator) - - if node.DnnType == config.Validator { - dnTmValidators = append(dnTmValidators, tmtypes.GenesisValidator{ - Name: fmt.Sprintf("Directory.%d", i), - Address: key.PubKey().Address(), - PubKey: key.PubKey(), - Power: 1, - }) - } - - if node.BvnnType == config.Validator { - bvnTmValidators = append(bvnTmValidators, tmtypes.GenesisValidator{ - Name: fmt.Sprintf("%s.%d", bvn.Id, j+1), - Address: key.PubKey().Address(), - PubKey: key.PubKey(), - Power: 1, - }) - } - } - - netinfo.AddPartition(bvn.Id, protocol.PartitionTypeBlockValidator) - docs[bvn.Id] = &tmtypes.GenesisDoc{ - ChainID: bvn.Id, - GenesisTime: time, - InitialHeight: protocol.GenesisBlock + 1, - Validators: bvnTmValidators, - ConsensusParams: tmtypes.DefaultConsensusParams(), } } - var bsnTmValidators []tmtypes.GenesisValidator if network.Bsn != nil { - for j, node := range network.Bsn.Nodes { + for _, node := range network.Bsn.Nodes { key := tmed25519.PrivKey(node.PrivValKey) operators = append(operators, key.PubKey().Bytes()) - netinfo.AddValidator(key.PubKey().Bytes(), network.Bsn.Id, node.BsnnType == config.Validator) - - if node.BsnnType == config.Validator { - bsnTmValidators = append(bsnTmValidators, tmtypes.GenesisValidator{ - Name: fmt.Sprintf("%s.%d", network.Bsn.Id, j+1), - Address: key.PubKey().Address(), - PubKey: key.PubKey(), - Power: 1, - }) - } } } - docs[protocol.Directory] = &tmtypes.GenesisDoc{ - ChainID: protocol.Directory, - GenesisTime: time, - InitialHeight: protocol.GenesisBlock + 1, - Validators: dnTmValidators, - ConsensusParams: tmtypes.DefaultConsensusParams(), - } - globals.Network = netinfo ids := []string{protocol.Directory} @@ -314,7 +263,8 @@ func BuildGenesisDocs(network *NetworkInit, globals *core.GlobalValues, time tim netType = protocol.PartitionTypeDirectory } snapBuf := new(ioutil2.Buffer) - root, err := genesis.Init(snapBuf, genesis.InitOpts{ + err = genesis.Init(snapBuf, genesis.InitOpts{ + NetworkID: network.Id, PartitionId: id, NetworkType: netType, GenesisTime: time, @@ -323,6 +273,7 @@ func BuildGenesisDocs(network *NetworkInit, globals *core.GlobalValues, time tim OperatorKeys: operators, FactomAddresses: factomAddresses, Snapshots: snapshots, + ConsensusParams: tmtypes.DefaultConsensusParams(), }) if err != nil { return nil, err @@ -344,33 +295,17 @@ func BuildGenesisDocs(network *NetworkInit, globals *core.GlobalValues, time tim } } - docs[id].AppHash = root - docs[id].AppState, err = json.Marshal(snapBuf.Bytes()) - if err != nil { - return nil, err - } + docs[id] = snapBuf.Bytes() } if network.Bsn != nil { - b, err := json.Marshal(bsnSnapBuf.Bytes()) - if err != nil { - return nil, err - } - docs[network.Bsn.Id] = &tmtypes.GenesisDoc{ - ChainID: network.Bsn.Id, - GenesisTime: time, - InitialHeight: 1, - Validators: bsnTmValidators, - ConsensusParams: tmtypes.DefaultConsensusParams(), - AppHash: make(tmbytes.HexBytes, 32), - AppState: b, - } + docs[network.Bsn.Id] = bsnSnapBuf.Bytes() } return docs, nil } -func WriteNodeFiles(cfg *config.Config, privValKey, nodeKey []byte, genDoc *tmtypes.GenesisDoc) (err error) { +func WriteNodeFiles(cfg *config.Config, privValKey, nodeKey []byte, genDoc []byte) (err error) { defer func() { if err != nil { _ = os.RemoveAll(cfg.RootDir) @@ -425,7 +360,7 @@ func WriteNodeFiles(cfg *config.Config, privValKey, nodeKey []byte, genDoc *tmty return fmt.Errorf("failed to write node key: %w", err) } - err = genDoc.SaveAs(cfg.GenesisFile()) + err = os.WriteFile(cfg.GenesisFile(), genDoc, 0644) if err != nil { return fmt.Errorf("failed to write genesis file: %w", err) } diff --git a/internal/node/daemon/run.go b/internal/node/daemon/run.go index cb7112c0c..6499c2bca 100644 --- a/internal/node/daemon/run.go +++ b/internal/node/daemon/run.go @@ -51,6 +51,7 @@ import ( "gitlab.com/accumulatenetwork/accumulate/internal/node/config" "gitlab.com/accumulatenetwork/accumulate/internal/node/connections" statuschk "gitlab.com/accumulatenetwork/accumulate/internal/node/connections/status" + "gitlab.com/accumulatenetwork/accumulate/internal/node/genesis" nodeapi "gitlab.com/accumulatenetwork/accumulate/internal/node/http" v3 "gitlab.com/accumulatenetwork/accumulate/pkg/api/v3" "gitlab.com/accumulatenetwork/accumulate/pkg/api/v3/message" @@ -379,7 +380,7 @@ func (d *Daemon) startConsensus(app types.Application) error { d.privVal, d.nodeKey, proxy.NewLocalClientCreator(app), - tmnode.DefaultGenesisDocProviderFunc(&d.Config.Config), + genesis.DocProvider(d.Config), tmcfg.DefaultDBProvider, tmnode.DefaultMetricsProvider(d.Config.Instrumentation), d.Logger, @@ -390,7 +391,6 @@ func (d *Daemon) startConsensus(app types.Application) error { d.node = &node.Node{Node: tmn, Config: d.Config, ABCI: app} // Start node - // TODO Feed Tendermint logger to service logger err = d.node.Start() if err != nil { return errors.UnknownError.WithFormat("start consensus: %v", err) diff --git a/internal/node/genesis/bootstrap.go b/internal/node/genesis/bootstrap.go index 36be85b5f..85e2d73a4 100644 --- a/internal/node/genesis/bootstrap.go +++ b/internal/node/genesis/bootstrap.go @@ -17,20 +17,21 @@ import ( "time" "github.com/cometbft/cometbft/abci/types" + tmed25519 "github.com/cometbft/cometbft/crypto/ed25519" "github.com/cometbft/cometbft/libs/log" + tmtypes "github.com/cometbft/cometbft/types" "gitlab.com/accumulatenetwork/accumulate/internal/api/routing" "gitlab.com/accumulatenetwork/accumulate/internal/core" "gitlab.com/accumulatenetwork/accumulate/internal/core/execute" "gitlab.com/accumulatenetwork/accumulate/internal/core/execute/v1/block" "gitlab.com/accumulatenetwork/accumulate/internal/core/execute/v1/chain" - "gitlab.com/accumulatenetwork/accumulate/internal/database" + coredb "gitlab.com/accumulatenetwork/accumulate/internal/database" "gitlab.com/accumulatenetwork/accumulate/internal/database/record" - "gitlab.com/accumulatenetwork/accumulate/internal/database/snapshot" "gitlab.com/accumulatenetwork/accumulate/internal/node/config" ioutil2 "gitlab.com/accumulatenetwork/accumulate/internal/util/io" "gitlab.com/accumulatenetwork/accumulate/pkg/database/keyvalue" "gitlab.com/accumulatenetwork/accumulate/pkg/database/keyvalue/memory" - snap2 "gitlab.com/accumulatenetwork/accumulate/pkg/database/snapshot" + "gitlab.com/accumulatenetwork/accumulate/pkg/database/snapshot" "gitlab.com/accumulatenetwork/accumulate/pkg/errors" "gitlab.com/accumulatenetwork/accumulate/pkg/types/encoding" "gitlab.com/accumulatenetwork/accumulate/pkg/types/merkle" @@ -40,19 +41,27 @@ import ( ) type InitOpts struct { - PartitionId string - NetworkType protocol.PartitionType - GenesisTime time.Time - Logger log.Logger + Logger log.Logger + + NetworkID string + PartitionId string + NetworkType protocol.PartitionType + GenesisTime time.Time + GenesisGlobals *core.GlobalValues + OperatorKeys [][]byte + + // For Tendermint + ConsensusParams *tmtypes.ConsensusParams + + // Preloaded data FactomAddresses func() (io.Reader, error) Snapshots []func() (ioutil2.SectionReader, error) - GenesisGlobals *core.GlobalValues - OperatorKeys [][]byte + // Flags IncludeHistoryFromSnapshots bool } -func Init(snapshotWriter io.WriteSeeker, opts InitOpts) ([]byte, error) { +func Init(snapshotWriter io.WriteSeeker, opts InitOpts) error { // Initialize globals gg := core.NewGlobals(opts.GenesisGlobals) @@ -81,11 +90,10 @@ func Init(snapshotWriter io.WriteSeeker, opts InitOpts) ([]byte, error) { b := &bootstrap{ InitOpts: opts, kvdb: store, - db: database.New(store, opts.Logger.With("module", "database")), + db: coredb.New(store, opts.Logger.With("module", "database")), dataRecords: make([]DataRecord, 0), records: make([]protocol.Account, 0), acmeIssued: new(big.Int), - omitHistory: map[[32]byte]bool{}, partition: config.NetworkUrl{URL: protocol.PartitionUrl(opts.PartitionId)}, } b.db.SetObserver(execute.NewDatabaseObserver()) @@ -94,13 +102,13 @@ func Init(snapshotWriter io.WriteSeeker, opts InitOpts) ([]byte, error) { var err error b.router, err = routing.NewStaticRouter(gg.Routing, b.Logger) if err != nil { - return nil, errors.UnknownError.Wrap(err) + return errors.UnknownError.Wrap(err) } // Unpack snapshots err = b.unpackSnapshots() if err != nil { - return nil, errors.UnknownError.Wrap(err) + return errors.UnknownError.Wrap(err) } exec, err := block.NewGenesisExecutor(b.db, opts.Logger, &config.Describe{ @@ -108,7 +116,7 @@ func Init(snapshotWriter io.WriteSeeker, opts InitOpts) ([]byte, error) { PartitionId: opts.PartitionId, }, gg, b.router) if err != nil { - return nil, errors.UnknownError.Wrap(err) + return errors.UnknownError.Wrap(err) } // Capture background tasks @@ -124,44 +132,66 @@ func Init(snapshotWriter io.WriteSeeker, opts InitOpts) ([]byte, error) { err = exec.Genesis(b.block, b) if err != nil { - return nil, errors.UnknownError.Wrap(err) + return errors.UnknownError.Wrap(err) } err = b.block.Batch.Commit() if err != nil { - return nil, errors.UnknownError.Wrap(err) + return errors.UnknownError.Wrap(err) } // Wait for background tasks err = errg.Wait() if err != nil { - return nil, errors.UnknownError.Wrap(err) + return errors.UnknownError.Wrap(err) } - // Preserve history in the Genesis snapshot - batch := b.db.Begin(false) - defer batch.Discard() + // Create the snapshot + err = b.db.Collect(snapshotWriter, b.partition.URL, &coredb.CollectOptions{ + DidWriteHeader: func(w *snapshot.Writer) error { + // Convert the consensus parameters + doc := new(consensusDoc) + doc.ChainID = opts.NetworkID + "." + opts.PartitionId + doc.Params = (*consensusParams)(opts.ConsensusParams) + for _, v := range opts.GenesisGlobals.Network.Validators { + if !v.IsActiveOn(opts.PartitionId) { + continue + } - header := new(snapshot.Header) - header.Height = protocol.GenesisBlock + var name string + if v.Operator == nil { + name = fmt.Sprintf("Validator-%x", v.PublicKeyHash[:4]) + } else { + name = v.Operator.ShortString() + } - w, err := snapshot.Collect(batch, header, snapshotWriter, snapshot.CollectOptions{ - Logger: b.Logger, - PreserveAccountHistory: func(account *database.Account) (bool, error) { - return !b.omitHistory[account.Url().AccountID32()], nil + key := tmed25519.PubKey(v.PublicKey) + doc.Validators = append(doc.Validators, &genesisValidator{ + Address: key.Address(), + PubKey: key, + Type: protocol.SignatureTypeED25519, + Power: 1, + Name: name, + }) + } + + // Write it + b, err := doc.MarshalBinary() + if err != nil { + return err + } + sw, err := w.OpenRaw(snapshot.SectionTypeConsensus) + if err != nil { + return err + } + _, err = sw.Write(b) + if err != nil { + return err + } + return sw.Close() }, }) - if err != nil { - return nil, errors.UnknownError.WithFormat("collect snapshot: %w", err) - } - - err = snapshot.CollectAnchors(w, batch, exec.Describe.PartitionUrl()) - if err != nil { - return nil, errors.UnknownError.Wrap(err) - } - - h, err := batch.GetBptRootHash() - return h[:], err + return errors.UnknownError.Wrap(err) } type bootstrap struct { @@ -171,7 +201,7 @@ type bootstrap struct { localAuthority *url.URL kvdb keyvalue.Beginner - db *database.Database + db *coredb.Database block *block.Block urls []*url.URL records []protocol.Account @@ -180,7 +210,6 @@ type bootstrap struct { accountsFromSnapshots []*url.URL acmeIssued *big.Int - omitHistory map[[32]byte]bool } type DataRecord struct { @@ -289,6 +318,7 @@ func (b *bootstrap) createMainLedger() { ledger := new(protocol.SystemLedger) ledger.Url = b.partition.Ledger() ledger.Index = protocol.GenesisBlock + ledger.Timestamp = b.InitOpts.GenesisTime ledger.ExecutorVersion = b.GenesisGlobals.ExecutorVersion b.WriteRecords(ledger) } @@ -442,10 +472,10 @@ func (b *bootstrap) unpackSnapshots() error { defer c.Close() } - err = b.db.Restore(file, &database.RestoreOptions{ + err = b.db.Restore(file, &coredb.RestoreOptions{ BatchRecordLimit: 50_000, SkipHashCheck: true, - Predicate: func(e *snap2.RecordEntry, v record.TerminalRecord) (bool, error) { + Predicate: func(e *snapshot.RecordEntry, v record.TerminalRecord) (bool, error) { switch e.Key.Get(0) { case "Account": // Skip the faucet @@ -537,10 +567,10 @@ func (b *bootstrap) unpackSnapshots() error { defer c.Close() } - err = b.db.Restore(file, &database.RestoreOptions{ + err = b.db.Restore(file, &coredb.RestoreOptions{ BatchRecordLimit: 50_000, SkipHashCheck: true, - Predicate: func(e *snap2.RecordEntry, v record.TerminalRecord) (bool, error) { + Predicate: func(e *snapshot.RecordEntry, v record.TerminalRecord) (bool, error) { switch e.Key.Get(0) { case "Transaction", "Message": h := e.Key.Get(1).([32]byte) diff --git a/internal/node/genesis/provider.go b/internal/node/genesis/provider.go new file mode 100644 index 000000000..5eddc57af --- /dev/null +++ b/internal/node/genesis/provider.go @@ -0,0 +1,95 @@ +// Copyright 2023 The Accumulate Authors +// +// Use of this source code is governed by an MIT-style +// license that can be found in the LICENSE file or at +// https://opensource.org/licenses/MIT. + +package genesis + +import ( + "bytes" + "encoding/json" + "fmt" + "os" + "path/filepath" + + "github.com/cometbft/cometbft/crypto/ed25519" + "github.com/cometbft/cometbft/node" + "github.com/cometbft/cometbft/types" + "gitlab.com/accumulatenetwork/accumulate/internal/node/config" + "gitlab.com/accumulatenetwork/accumulate/pkg/database/snapshot" + "gitlab.com/accumulatenetwork/accumulate/protocol" +) + +func ConvertJsonToSnapshot(doc *types.GenesisDoc) ([]byte, error) { + // The JSON genesis document was generated from a snapshot, which itself + // contains sufficient information to recreate the JSON genesis doc. So all + // we need to do here is extract the snapshot. + var b []byte + err := json.Unmarshal(doc.AppState, &b) + return b, err +} + +// DocProvider reads the genesis document. +func DocProvider(config *config.Config) node.GenesisDocProvider { + // If it's a JSON file, fallback to the default + if filepath.Ext(config.Genesis) == ".json" { + return node.DefaultGenesisDocProviderFunc(&config.Config) + } + + return func() (*types.GenesisDoc, error) { + // Open the snapshot + all, err := os.ReadFile(config.GenesisFile()) + if err != nil { + return nil, err + } + + s, err := snapshot.Open(bytes.NewReader(all)) + if err != nil { + return nil, err + } + + // Read the consensus section + rd, err := s.Open(snapshot.SectionTypeConsensus) + if err != nil { + return nil, err + } + p := new(consensusDoc) + err = p.UnmarshalBinaryFrom(rd) + if err != nil { + return nil, err + } + + // Convert + jsonBytes, err := json.Marshal(all) + if err != nil { + return nil, err + } + + doc := &types.GenesisDoc{ + GenesisTime: s.Header.SystemLedger.Timestamp, + ChainID: p.ChainID, + InitialHeight: int64(s.Header.SystemLedger.Index) + 1, + ConsensusParams: (*types.ConsensusParams)(p.Params), + Validators: make([]types.GenesisValidator, len(p.Validators)), + AppHash: s.Header.RootHash[:], + AppState: jsonBytes, + } + for i, v := range p.Validators { + u := types.GenesisValidator{ + Address: v.Address, + Power: v.Power, + Name: v.Name, + } + switch v.Type { + case protocol.SignatureTypeED25519: + u.PubKey = ed25519.PubKey(v.PubKey) + default: + return nil, fmt.Errorf("unsupported key type %s", v.Type) + } + doc.Validators[i] = u + } + + return doc, nil + } +} diff --git a/internal/node/genesis/types.go b/internal/node/genesis/types.go new file mode 100644 index 000000000..1eca15f61 --- /dev/null +++ b/internal/node/genesis/types.go @@ -0,0 +1,69 @@ +// Copyright 2023 The Accumulate Authors +// +// Use of this source code is governed by an MIT-style +// license that can be found in the LICENSE file or at +// https://opensource.org/licenses/MIT. + +package genesis + +import ( + "io" + + cmtproto "github.com/cometbft/cometbft/proto/tendermint/types" + "github.com/cometbft/cometbft/types" + "github.com/cosmos/gogoproto/proto" +) + +//go:generate go run gitlab.com/accumulatenetwork/accumulate/tools/cmd/gen-types --package genesis --language go-alt types.yml + +type consensusParams types.ConsensusParams + +func (c *consensusParams) CopyAsInterface() any { + d := *c + return &d +} + +func (c *consensusParams) Equal(d *consensusParams) bool { + switch { + case c == d: + return true + case c == nil, + d == nil, + c.Block != d.Block, + c.Evidence != d.Evidence, + len(c.Validator.PubKeyTypes) != len(d.Validator.PubKeyTypes), + c.Version != d.Version, + c.ABCI != d.ABCI: + return false + } + for i := range c.Validator.PubKeyTypes { + c, d := c.Validator.PubKeyTypes[i], d.Validator.PubKeyTypes[i] + if c != d { + return false + } + } + return true +} + +func (c *consensusParams) MarshalBinary() ([]byte, error) { + d := (*types.ConsensusParams)(c).ToProto() + return proto.Marshal(&d) +} + +func (c *consensusParams) UnmarshalBinary(b []byte) error { + d := new(cmtproto.ConsensusParams) + err := proto.Unmarshal(b, d) + if err != nil { + return err + } + *c = consensusParams(types.ConsensusParamsFromProto(*d)) + return nil +} + +func (c *consensusParams) UnmarshalBinaryFrom(rd io.Reader) error { + b, err := io.ReadAll(rd) + if err != nil { + return err + } + return c.UnmarshalBinary(b) +} diff --git a/internal/node/genesis/types.yml b/internal/node/genesis/types.yml new file mode 100644 index 000000000..03ea9f90e --- /dev/null +++ b/internal/node/genesis/types.yml @@ -0,0 +1,27 @@ +consensusDoc: + fields: + - name: ChainID + type: string + - name: Params + type: consensusParams + marshal-as: reference + pointer: true + - name: Validators + type: genesisValidator + marshal-as: reference + repeatable: true + pointer: true + +genesisValidator: + fields: + - name: Address + type: bytes + - name: Type + type: protocol.SignatureType + marshal-as: enum + - name: PubKey + type: bytes + - name: Power + type: int + - name: Name + type: string \ No newline at end of file diff --git a/internal/node/genesis/types_gen.go b/internal/node/genesis/types_gen.go new file mode 100644 index 000000000..45c6bfcb8 --- /dev/null +++ b/internal/node/genesis/types_gen.go @@ -0,0 +1,96 @@ +// Copyright 2022 The Accumulate Authors +// +// Use of this source code is governed by an MIT-style +// license that can be found in the LICENSE file or at +// https://opensource.org/licenses/MIT. + +package genesis + +// GENERATED BY go run ./tools/cmd/gen-types. DO NOT EDIT. + +//lint:file-ignore S1001,S1002,S1008,SA4013 generated code + +import ( + "io" + + "gitlab.com/accumulatenetwork/accumulate/pkg/types/encoding" + "gitlab.com/accumulatenetwork/accumulate/protocol" +) + +type consensusDoc struct { + fieldsSet []bool + ChainID string `json:"chainID,omitempty" form:"chainID" query:"chainID" validate:"required"` + Params *consensusParams `json:"params,omitempty" form:"params" query:"params" validate:"required"` + Validators []*genesisValidator `json:"validators,omitempty" form:"validators" query:"validators" validate:"required"` + extraData []byte +} + +var machine_consensusDoc = &encoding.Machine[*consensusDoc]{ + ExtraData: func(v *consensusDoc) *[]byte { return &v.extraData }, + Seen: func(v *consensusDoc) *[]bool { return &v.fieldsSet }, + Fields: []*encoding.Field[*consensusDoc]{ + {Name: "ChainID", Number: 1, Binary: true, OmitEmpty: true, Required: true, Accessor: encoding.StringField[*consensusDoc](func(v *consensusDoc) *string { return &v.ChainID })}, + {Name: "Params", Number: 2, Binary: true, OmitEmpty: true, Required: true, Accessor: encoding.StructPtrField[*consensusDoc, *consensusParams, consensusParams](func(v *consensusDoc) **consensusParams { return &v.Params })}, + {Name: "Validators", Number: 3, Binary: true, OmitEmpty: true, Required: true, Accessor: encoding.SliceField[*consensusDoc, *genesisValidator, encoding.StructPtrField[encoding.SliceIndex[*genesisValidator], *genesisValidator, genesisValidator]](func(v *consensusDoc) *[]*genesisValidator { return &v.Validators })}, + }, +} + +func (v *consensusDoc) IsValid() error { return machine_consensusDoc.IsValid(v) } +func (v *consensusDoc) Copy() *consensusDoc { return encoding.Copy(machine_consensusDoc, v) } +func (v *consensusDoc) CopyAsInterface() interface{} { return v.Copy() } +func (v *consensusDoc) Equal(u *consensusDoc) bool { return machine_consensusDoc.Equal(v, u) } +func (v *consensusDoc) MarshalBinary() ([]byte, error) { return machine_consensusDoc.MarshalBinary(v) } +func (v *consensusDoc) UnmarshalBinary(data []byte) error { + return machine_consensusDoc.Unmarshal(data, v) +} +func (v *consensusDoc) UnmarshalBinaryFrom(rd io.Reader) error { + return machine_consensusDoc.UnmarshalFrom(rd, v) +} +func (v *consensusDoc) MarshalJSON() ([]byte, error) { return machine_consensusDoc.JSONMarshal(v) } +func (v *consensusDoc) UnmarshalJSON(b []byte) error { return machine_consensusDoc.JSONUnmarshal(b, v) } + +type genesisValidator struct { + fieldsSet []bool + Address []byte `json:"address,omitempty" form:"address" query:"address" validate:"required"` + Type protocol.SignatureType `json:"type,omitempty" form:"type" query:"type" validate:"required"` + PubKey []byte `json:"pubKey,omitempty" form:"pubKey" query:"pubKey" validate:"required"` + Power int64 `json:"power,omitempty" form:"power" query:"power" validate:"required"` + Name string `json:"name,omitempty" form:"name" query:"name" validate:"required"` + extraData []byte +} + +var machine_genesisValidator = &encoding.Machine[*genesisValidator]{ + ExtraData: func(v *genesisValidator) *[]byte { return &v.extraData }, + Seen: func(v *genesisValidator) *[]bool { return &v.fieldsSet }, + Fields: []*encoding.Field[*genesisValidator]{ + {Name: "Address", Number: 1, Binary: true, OmitEmpty: true, Required: true, Accessor: encoding.BytesField[*genesisValidator](func(v *genesisValidator) *[]byte { return &v.Address })}, + {Name: "Type", Number: 2, Binary: true, OmitEmpty: true, Required: true, Accessor: encoding.EnumField[*genesisValidator, *protocol.SignatureType, protocol.SignatureType](func(v *genesisValidator) *protocol.SignatureType { return &v.Type })}, + {Name: "PubKey", Number: 3, Binary: true, OmitEmpty: true, Required: true, Accessor: encoding.BytesField[*genesisValidator](func(v *genesisValidator) *[]byte { return &v.PubKey })}, + {Name: "Power", Number: 4, Binary: true, OmitEmpty: true, Required: true, Accessor: encoding.IntField[*genesisValidator](func(v *genesisValidator) *int64 { return &v.Power })}, + {Name: "Name", Number: 5, Binary: true, OmitEmpty: true, Required: true, Accessor: encoding.StringField[*genesisValidator](func(v *genesisValidator) *string { return &v.Name })}, + }, +} + +func (v *genesisValidator) IsValid() error { return machine_genesisValidator.IsValid(v) } +func (v *genesisValidator) Copy() *genesisValidator { + return encoding.Copy(machine_genesisValidator, v) +} +func (v *genesisValidator) CopyAsInterface() interface{} { return v.Copy() } +func (v *genesisValidator) Equal(u *genesisValidator) bool { + return machine_genesisValidator.Equal(v, u) +} +func (v *genesisValidator) MarshalBinary() ([]byte, error) { + return machine_genesisValidator.MarshalBinary(v) +} +func (v *genesisValidator) UnmarshalBinary(data []byte) error { + return machine_genesisValidator.Unmarshal(data, v) +} +func (v *genesisValidator) UnmarshalBinaryFrom(rd io.Reader) error { + return machine_genesisValidator.UnmarshalFrom(rd, v) +} +func (v *genesisValidator) MarshalJSON() ([]byte, error) { + return machine_genesisValidator.JSONMarshal(v) +} +func (v *genesisValidator) UnmarshalJSON(b []byte) error { + return machine_genesisValidator.JSONUnmarshal(b, v) +} diff --git a/pkg/database/keyvalue/badger/changeset_test.go b/pkg/database/keyvalue/badger/changeset_test.go index 534dce5a8..9efc29f0e 100644 --- a/pkg/database/keyvalue/badger/changeset_test.go +++ b/pkg/database/keyvalue/badger/changeset_test.go @@ -29,7 +29,7 @@ func TestWriteLimit(t *testing.T) { raw, err := badger.Open(badger. DefaultOptions(t.TempDir()). WithMaxTableSize(1 << 20). // 1MB - WithLogger(slogger{})) + WithLogger(Slogger{})) require.NoError(t, err) defer raw.Close() diff --git a/pkg/database/keyvalue/badger/database.go b/pkg/database/keyvalue/badger/database.go index b5e24b8de..bd91d8732 100644 --- a/pkg/database/keyvalue/badger/database.go +++ b/pkg/database/keyvalue/badger/database.go @@ -44,7 +44,7 @@ func New(filepath string) (*Database, error) { } opts := badger.DefaultOptions(filepath) - opts = opts.WithLogger(slogger{}) + opts = opts.WithLogger(Slogger{}) // Truncate corrupted data if TruncateBadger { @@ -184,25 +184,25 @@ func (d *Database) lock(closing bool) (sync.Locker, error) { return l, nil } -type slogger struct{} +type Slogger struct{} -func (l slogger) format(format string, args ...interface{}) string { +func (l Slogger) format(format string, args ...interface{}) string { s := fmt.Sprintf(format, args...) return strings.TrimRight(s, "\n") } -func (l slogger) Errorf(format string, args ...interface{}) { +func (l Slogger) Errorf(format string, args ...interface{}) { slog.Error(l.format(format, args...), "module", "badger") } -func (l slogger) Warningf(format string, args ...interface{}) { +func (l Slogger) Warningf(format string, args ...interface{}) { slog.Warn(l.format(format, args...), "module", "badger") } -func (l slogger) Infof(format string, args ...interface{}) { +func (l Slogger) Infof(format string, args ...interface{}) { slog.Info(l.format(format, args...), "module", "badger") } -func (l slogger) Debugf(format string, args ...interface{}) { +func (l Slogger) Debugf(format string, args ...interface{}) { slog.Debug(l.format(format, args...), "module", "badger") } diff --git a/pkg/database/snapshot/enums.yml b/pkg/database/snapshot/enums.yml index 0543f64e3..b33148b94 100644 --- a/pkg/database/snapshot/enums.yml +++ b/pkg/database/snapshot/enums.yml @@ -26,3 +26,6 @@ SectionType: BPT: description: contains the BPT value: 9 + Consensus: + description: contains consensus parameters + value: 10 diff --git a/pkg/database/snapshot/enums_gen.go b/pkg/database/snapshot/enums_gen.go index c02af0c32..26e3d8a81 100644 --- a/pkg/database/snapshot/enums_gen.go +++ b/pkg/database/snapshot/enums_gen.go @@ -41,6 +41,9 @@ const SectionTypeRecordIndex SectionType = 8 // SectionTypeBPT contains the BPT. const SectionTypeBPT SectionType = 9 +// SectionTypeConsensus contains consensus parameters. +const SectionTypeConsensus SectionType = 10 + // GetEnumValue returns the value of the Section Type func (v SectionType) GetEnumValue() uint64 { return uint64(v) } @@ -48,7 +51,7 @@ func (v SectionType) GetEnumValue() uint64 { return uint64(v) } func (v *SectionType) SetEnumValue(id uint64) bool { u := SectionType(id) switch u { - case SectionTypeHeader, SectionTypeAccountsV1, SectionTypeTransactionsV1, SectionTypeSignaturesV1, SectionTypeGzTransactionsV1, SectionTypeSnapshot, SectionTypeRecords, SectionTypeRecordIndex, SectionTypeBPT: + case SectionTypeHeader, SectionTypeAccountsV1, SectionTypeTransactionsV1, SectionTypeSignaturesV1, SectionTypeGzTransactionsV1, SectionTypeSnapshot, SectionTypeRecords, SectionTypeRecordIndex, SectionTypeBPT, SectionTypeConsensus: *v = u return true } @@ -76,6 +79,8 @@ func (v SectionType) String() string { return "recordIndex" case SectionTypeBPT: return "bpt" + case SectionTypeConsensus: + return "consensus" } return fmt.Sprintf("SectionType:%d", v) } @@ -101,6 +106,8 @@ func SectionTypeByName(name string) (SectionType, bool) { return SectionTypeRecordIndex, true case "bpt": return SectionTypeBPT, true + case "consensus": + return SectionTypeConsensus, true } return 0, false } diff --git a/pkg/database/snapshot/format.go b/pkg/database/snapshot/format.go index d8629dba3..943dbe2ea 100644 --- a/pkg/database/snapshot/format.go +++ b/pkg/database/snapshot/format.go @@ -108,6 +108,16 @@ func (r *Reader) open(i int, typ SectionType) (ioutil.SectionReader, error) { return rd, nil } +// Open opens the first section of the given type +func (r *Reader) Open(typ SectionType) (ioutil.SectionReader, error) { + for _, s := range r.Sections { + if s.Type() == typ { + return s.Open() + } + } + return nil, errors.NotFound.WithFormat("%v section not found", typ) +} + func (r *Reader) OpenIndex(i int) (*IndexReader, error) { rd, err := r.open(i, SectionTypeRecordIndex) if err != nil { diff --git a/test/simulator/options.go b/test/simulator/options.go index 39100ec56..26ead7c52 100644 --- a/test/simulator/options.go +++ b/test/simulator/options.go @@ -7,7 +7,6 @@ package simulator import ( - "encoding/json" "fmt" "io" "net" @@ -16,7 +15,6 @@ import ( "time" "github.com/cometbft/cometbft/libs/log" - tmtypes "github.com/cometbft/cometbft/types" "gitlab.com/accumulatenetwork/accumulate/internal/core" "gitlab.com/accumulatenetwork/accumulate/internal/node/config" accumulated "gitlab.com/accumulatenetwork/accumulate/internal/node/daemon" @@ -201,7 +199,7 @@ func genesis(time time.Time, values *core.GlobalValues) SnapshotFunc { values = new(core.GlobalValues) } - var genDocs map[string]*tmtypes.GenesisDoc + var genDocs map[string][]byte return func(partition string, network *accumulated.NetworkInit, logger log.Logger) (ioutil2.SectionReader, error) { var err error if genDocs == nil { @@ -211,13 +209,7 @@ func genesis(time time.Time, values *core.GlobalValues) SnapshotFunc { } } - var snapshot []byte - err = json.Unmarshal(genDocs[partition].AppState, &snapshot) - if err != nil { - return nil, errors.UnknownError.Wrap(err) - } - - return ioutil2.NewBuffer(snapshot), nil + return ioutil2.NewBuffer(genDocs[partition]), nil } } diff --git a/vdk/node/node.go b/vdk/node/node.go index d7b3d7e26..2ebf74455 100644 --- a/vdk/node/node.go +++ b/vdk/node/node.go @@ -23,6 +23,7 @@ import ( "gitlab.com/accumulatenetwork/accumulate/internal/api/v2" "gitlab.com/accumulatenetwork/accumulate/internal/node/config" accumulated "gitlab.com/accumulatenetwork/accumulate/internal/node/daemon" + "gitlab.com/accumulatenetwork/accumulate/internal/node/genesis" client "gitlab.com/accumulatenetwork/accumulate/pkg/client/api/v2" "gitlab.com/accumulatenetwork/accumulate/pkg/proxy" "gitlab.com/accumulatenetwork/accumulate/pkg/url" @@ -89,7 +90,12 @@ func InitializeFollowerFromSeed(workDir string, expectedPartitionType protocol.P return fmt.Errorf("load/generate node key files, %v", err) } - err = accumulated.WriteNodeFiles(config, privValKey, nodeKey, genDoc) + config.Genesis = "config/genesis.snap" + genDocBytes, err := genesis.ConvertJsonToSnapshot(genDoc) + if err != nil { + return fmt.Errorf("write node files, %v", err) + } + err = accumulated.WriteNodeFiles(config, privValKey, nodeKey, genDocBytes) if err != nil { return fmt.Errorf("write node files, %v", err) }