diff --git a/astria/execution/v1alpha2.go b/astria/execution/v1alpha2.go index 307ac0d35..e4d03dce4 100644 --- a/astria/execution/v1alpha2.go +++ b/astria/execution/v1alpha2.go @@ -3,10 +3,10 @@ package execution import ( "context" "sync" + "time" astriaGrpc "buf.build/gen/go/astria/execution-apis/grpc/go/astria/execution/v1alpha2/executionv1alpha2grpc" astriaPb "buf.build/gen/go/astria/execution-apis/protocolbuffers/go/astria/execution/v1alpha2" - "github.com/astriaorg/rollkit/astria/state/commitment" "github.com/astriaorg/rollkit/block" "github.com/astriaorg/rollkit/store" "github.com/astriaorg/rollkit/types" @@ -23,12 +23,12 @@ type ExecutionServiceServerV1Alpha2 struct { // UnimplementedExecutionServiceServer for forward compatibility astriaGrpc.UnimplementedExecutionServiceServer - commitmentStore *commitment.CommitmentState - store store.Store - blockManager *block.SSManager - genesis GenesisInfo - logger log.Logger - blockExecutionLock sync.Mutex + store store.Store + blockManager *block.SSManager + genesis GenesisInfo + logger log.Logger + blockExecutionLock sync.Mutex + commitementUpdateLock sync.Mutex } type GenesisInfo struct { @@ -36,15 +36,15 @@ type GenesisInfo struct { SequencerGenesisBlockHeight uint64 CelestiaBaseBlockHeight uint64 CelestiaBlockVariance uint64 + GenesisTime time.Time } -func NewExecutionServiceServerV1Alpha2(blockManager *block.SSManager, genesis GenesisInfo, commitmentStore *commitment.CommitmentState, store store.Store, logger log.Logger) *ExecutionServiceServerV1Alpha2 { +func NewExecutionServiceServerV1Alpha2(blockManager *block.SSManager, genesis GenesisInfo, store store.Store, logger log.Logger) *ExecutionServiceServerV1Alpha2 { return &ExecutionServiceServerV1Alpha2{ - blockManager: blockManager, - genesis: genesis, - commitmentStore: commitmentStore, - store: store, - logger: logger, + blockManager: blockManager, + genesis: genesis, + store: store, + logger: logger, } } @@ -119,9 +119,9 @@ func (s *ExecutionServiceServerV1Alpha2) ExecuteBlock(ctx context.Context, req * txs[i] = types.Tx(req.Transactions[i]) } - block, err := s.blockManager.PublishBlock(ctx, types.Hash(req.PrevBlockHash), req.Timestamp.AsTime(), txs) + block, err := s.blockManager.ExecuteBlock(ctx, types.Hash(req.PrevBlockHash), req.Timestamp.AsTime(), txs) if err != nil { - s.logger.Error("Failed to publish block to chain", "hash", block.Hash(), "prevHash", types.Hash(req.PrevBlockHash), "err", err) + s.logger.Error("Failed to publish new block to chain", "prevHash", types.Hash(req.PrevBlockHash), "err", err) return nil, status.Error(codes.Internal, "failed to insert block to chain") } @@ -150,14 +150,14 @@ func (s *ExecutionServiceServerV1Alpha2) GetCommitmentState(ctx context.Context, reqJson, _ := protojson.Marshal(req) s.logger.Info("GetCommitmentState called", "request", reqJson) - res, err := s.commitmentStore.GetCommitmentState() - if err != nil { - s.logger.Error("GetCommitmentState failed", "err", err) - return &astriaPb.CommitmentState{}, err + res := &astriaPb.CommitmentState{ + Soft: s.getPbBlock(ctx, s.blockManager.GetStoreHeight()), + Firm: s.getPbBlock(ctx, s.blockManager.GetDAHeight()), } resJson, _ := protojson.Marshal(res) s.logger.Info("GetCommitmentState completed", "response", resJson) + return res, nil } @@ -166,11 +166,26 @@ func (s *ExecutionServiceServerV1Alpha2) UpdateCommitmentState(ctx context.Conte reqJson, _ := protojson.Marshal(req) s.logger.Info("UpdateCommitmentState called", "request", reqJson) - if err := s.commitmentStore.UpdateCommitmentState(req.CommitmentState); err != nil { + s.commitementUpdateLock.Lock() + defer s.commitementUpdateLock.Unlock() + + currHeight := s.blockManager.GetStoreHeight() + if uint64(req.CommitmentState.Soft.Number) > currHeight { + err := s.blockManager.Commit(ctx, uint64(req.CommitmentState.Soft.Number), false) + if err != nil { + s.logger.Error("UpdateCommitmentState failed", "err", err) + } + } + + err := s.blockManager.SetDAHeight(ctx, uint64(req.CommitmentState.Firm.Number)) + if err != nil { s.logger.Error("UpdateCommitmentState failed", "err", err) } - return req.CommitmentState, nil + return &astriaPb.CommitmentState{ + Soft: s.getPbBlock(ctx, s.blockManager.GetStoreHeight()), + Firm: req.CommitmentState.Firm, + }, nil } func (s *ExecutionServiceServerV1Alpha2) getBlockFromIdentifier(ctx context.Context, identifier *astriaPb.BlockIdentifier) (*astriaPb.Block, error) { @@ -203,3 +218,35 @@ func (s *ExecutionServiceServerV1Alpha2) getBlockFromIdentifier(ctx context.Cont Timestamp: timestamppb.New(block.Time()), }, nil } + +func (s *ExecutionServiceServerV1Alpha2) getPbBlock(ctx context.Context, height uint64) *astriaPb.Block { + genHash := [32]byte{0x0} + if height == 0 { + return &astriaPb.Block{ + Number: uint32(0), + Hash: genHash[:], + ParentBlockHash: genHash[:], + Timestamp: timestamppb.New(s.genesis.GenesisTime), + } + } else { + block, err := s.store.GetBlock(ctx, height) + if err != nil { + s.logger.Error("failed finding block with height", "height", height, "error", err) + return nil + } + + var parentBlockHash []byte + if height == 1 { + parentBlockHash = genHash[:] + } else { + parentBlockHash = cmbytes.HexBytes(block.LastHeader()) + } + + return &astriaPb.Block{ + Number: uint32(block.Height()), + Hash: cmbytes.HexBytes(block.Hash()), + ParentBlockHash: parentBlockHash, + Timestamp: timestamppb.New(block.Time()), + } + } +} diff --git a/astria/state/commitment/kv.go b/astria/state/commitment/kv.go deleted file mode 100644 index 1d1fa0e89..000000000 --- a/astria/state/commitment/kv.go +++ /dev/null @@ -1,74 +0,0 @@ -package commitment - -import ( - "context" - "fmt" - "time" - - astriaPb "buf.build/gen/go/astria/execution-apis/protocolbuffers/go/astria/execution/v1alpha2" - ds "github.com/ipfs/go-datastore" - "google.golang.org/protobuf/proto" - "google.golang.org/protobuf/types/known/timestamppb" -) - -const CommitmentKey = "commitment" - -type CommitmentState struct { - store ds.TxnDatastore - ctx context.Context -} - -func NewCommitmentState(ctx context.Context, store ds.TxnDatastore) *CommitmentState { - return &CommitmentState{ - store: store, - ctx: ctx, - } -} - -func (cs *CommitmentState) UpdateCommitmentState(commitment *astriaPb.CommitmentState) error { - txn, err := cs.store.NewTransaction(cs.ctx, false) - if err != nil { - return fmt.Errorf("failed to create a new batch for transaction: %w", err) - } - defer txn.Discard(cs.ctx) - - key := ds.NewKey(CommitmentKey) - val, err := proto.Marshal(commitment) - if err != nil { - return fmt.Errorf("failed to marshal the commitment: %w", err) - } - - if err := txn.Put(cs.ctx, key, val); err != nil { - return err - } - - return txn.Commit(cs.ctx) -} - -func (cs *CommitmentState) GetCommitmentState() (*astriaPb.CommitmentState, error) { - val, err := cs.store.Get(cs.ctx, ds.NewKey(CommitmentKey)) - if err != nil { - if err == ds.ErrNotFound { - genHash := [32]byte{0x0} - pbGenBlock := &astriaPb.Block{ - Number: uint32(0), - Hash: genHash[:], - ParentBlockHash: genHash[:], - Timestamp: timestamppb.New(time.Now()), - } - return &astriaPb.CommitmentState{ - Soft: pbGenBlock, - Firm: pbGenBlock, - }, nil - } else { - return nil, fmt.Errorf("failed to get the commitment: %w", err) - } - } - - commitment := &astriaPb.CommitmentState{} - if err := proto.Unmarshal(val, commitment); err != nil { - return nil, fmt.Errorf("failed to unmarshal the commitment: %w", err) - } - - return commitment, nil -} diff --git a/block/ss_manager.go b/block/ss_manager.go index f282e1950..19a975153 100644 --- a/block/ss_manager.go +++ b/block/ss_manager.go @@ -3,7 +3,6 @@ package block import ( "bytes" "context" - "encoding/hex" "errors" "fmt" "sync" @@ -35,6 +34,7 @@ type SSManager struct { proposerKey crypto.PrivKey executor *state.BlockExecutor logger log.Logger + proxyApp proxy.AppConns // Rollkit doesn't have "validators", but // we store the sequencer in this struct for compatibility. @@ -51,7 +51,7 @@ func NewSSManager( genesis *cmtypes.GenesisDoc, store store.Store, mempool mempool.Mempool, - proxyApp proxy.AppConnConsensus, + proxyApp proxy.AppConns, eventBus *cmtypes.EventBus, logger log.Logger, seqMetrics *Metrics, @@ -70,7 +70,7 @@ func NewSSManager( return nil, err } - exec := state.NewBlockExecutor(proposerAddress, genesis.ChainID, mempool, proxyApp, eventBus, logger, execMetrics) + exec := state.NewBlockExecutor(proposerAddress, genesis.ChainID, mempool, proxyApp.Consensus(), eventBus, logger, execMetrics) if s.LastBlockHeight+1 == uint64(genesis.InitialHeight) { logger.Info("Initializing chain") res, err := exec.InitChain(genesis) @@ -95,10 +95,31 @@ func NewSSManager( logger: logger, validatorSet: &valSet, metrics: seqMetrics, + proxyApp: proxyApp, } return agg, nil } +func (m *SSManager) CheckCrashRecovery(ctx context.Context) error { + m.logger.Info("checking for crash recovery") + res, err := m.proxyApp.Query().Info(ctx, proxy.RequestInfo) + if err != nil { + return fmt.Errorf("error calling proxyApp.Query().Info: %v", err) + } + m.logger.Info("app handshake", "LastBlockHeight", res.LastBlockHeight, "LastBlockAppHash", res.LastBlockAppHash) + storeHeight := m.GetStoreHeight() + if storeHeight != uint64(res.LastBlockHeight) { + m.logger.Info("store height and app height mismatch", "store_height", storeHeight, "app_height", res.LastBlockHeight) + if res.LastBlockHeight-int64(storeHeight) == 1 { + m.logger.Info("committing block", "height", res.LastBlockHeight) + m.Commit(ctx, uint64(res.LastBlockHeight), true) + } else { + panic("what do") + } + } + return nil +} + // SetLastState is used to set lastState used by Manager. func (m *SSManager) SetLastState(state types.State) { m.lastStateMtx.Lock() @@ -111,6 +132,25 @@ func (m *SSManager) GetStoreHeight() uint64 { return m.store.Height() } +func (m *SSManager) GetLastState() types.State { + return m.lastState +} + +func (m *SSManager) GetDAHeight() uint64 { + return m.lastState.DAHeight +} + +func (m *SSManager) SetDAHeight(ctx context.Context, height uint64) error { + m.lastStateMtx.Lock() + defer m.lastStateMtx.Unlock() + m.lastState.DAHeight = height + err := m.store.UpdateState(ctx, m.lastState) + if err != nil { + return err + } + return nil +} + func (m *SSManager) getCommit(header types.Header) (*types.Commit, error) { headerBytes, err := header.MarshalBinary() if err != nil { @@ -125,7 +165,7 @@ func (m *SSManager) getCommit(header types.Header) (*types.Commit, error) { }, nil } -func (m *SSManager) PublishBlock(ctx context.Context, prevBlockHash types.Hash, timestamp time.Time, txs types.Txs) (*types.Block, error) { +func (m *SSManager) ExecuteBlock(ctx context.Context, prevBlockHash types.Hash, timestamp time.Time, txs types.Txs) (*types.Block, error) { select { case <-ctx.Done(): return nil, ctx.Err() @@ -153,7 +193,7 @@ func (m *SSManager) PublishBlock(ctx context.Context, prevBlockHash types.Hash, // Validate block being created has valid previous hash lastHeaderHash = lastBlock.Hash() if !bytes.Equal(lastHeaderHash, prevBlockHash) { - return nil, fmt.Errorf("block can only be created on top of soft block.") + return nil, fmt.Errorf("block can only be created on top of soft block. Last recorded block height: %d, hash: %s", height, lastBlock.Hash()) } } @@ -199,7 +239,23 @@ func (m *SSManager) PublishBlock(ctx context.Context, prevBlockHash types.Hash, block.SignedHeader.Validators = m.validatorSet - newState, responses, err := m.applyBlock(ctx, block) + if pendingBlock == nil { + isAppValid, err := m.executor.ProcessProposal(block, m.lastState) + if err != nil { + return nil, err + } + if !isAppValid { + return nil, fmt.Errorf("error while processing the proposal: %v", err) + } + + // SaveBlock commits the DB tx + err = m.store.SaveBlock(ctx, block, commit) + if err != nil { + return nil, fmt.Errorf("error while saving block: %w", err) + } + } + + responses, err := m.applyBlock(ctx, block) if err != nil { if ctx.Err() != nil { return nil, fmt.Errorf("error while applying block: %w", err) @@ -228,8 +284,6 @@ func (m *SSManager) PublishBlock(ctx context.Context, prevBlockHash types.Hash, } blockHeight := block.Height() - // Update the stored height before submitting to the DA layer and committing to the DB - m.store.SetHeight(ctx, blockHeight) // SaveBlock commits the DB tx err = m.store.SaveBlock(ctx, block, commit) @@ -237,30 +291,58 @@ func (m *SSManager) PublishBlock(ctx context.Context, prevBlockHash types.Hash, return nil, fmt.Errorf("error while saving block: %w", err) } - // Commit the new state and block which writes to disk on the proxy app - _, _, err = m.executor.Commit(ctx, newState, block, responses) - if err != nil { - return nil, fmt.Errorf("error while committing block: %w", err) - } - // SaveBlockResponses commits the DB tx err = m.store.SaveBlockResponses(ctx, blockHeight, responses) if err != nil { return nil, fmt.Errorf("error while saving block responses: %w", err) } + return block, nil +} + +func (m *SSManager) Commit(ctx context.Context, height uint64, skipExec bool) error { + currHeight := m.store.Height() + if height != currHeight+1 { + m.logger.Error("Trying to commit invalid height", "currHeight", currHeight, "newHeight", height) + return fmt.Errorf("cannot commit an invalid height: current height %d, new height %d", currHeight, height) + } + + pendingBlock, err := m.store.GetBlock(ctx, height) + if err != nil { + return fmt.Errorf("error while loading block: %w", err) + } + + blockResponses, err := m.store.GetBlockResponses(ctx, height) + if err != nil { + return err + } + + newState, err := m.executor.UpdateState(m.lastState, pendingBlock, blockResponses) + if err != nil { + return err + } + + // Update the stored height + m.store.SetHeight(ctx, height) + + // Commit the new state and block which writes to disk on the proxy app + _, err = m.executor.Commit(ctx, newState, pendingBlock, blockResponses, skipExec) + if err != nil { + return fmt.Errorf("error while committing block: %w", err) + } + // After this call m.lastState is the NEW state returned from ApplyBlock // updateState also commits the DB tx err = m.updateState(ctx, newState) if err != nil { - return nil, fmt.Errorf("error while updating state: %w", err) + return fmt.Errorf("error while updating state: %w", err) } - m.recordMetrics(block) + m.recordMetrics(pendingBlock) - m.logger.Debug("successfully proposed block", "proposer", hex.EncodeToString(block.SignedHeader.ProposerAddress), "height", blockHeight) + m.logger.Info("Successfully commited height", "height", height) - return block, nil + return nil } func (m *SSManager) recordMetrics(block *types.Block) { @@ -295,7 +377,7 @@ func (m *SSManager) createBlock(height uint64, timestamp time.Time, txs types.Tx return m.executor.CreateBlockFromSeqencer(height, timestamp, txs, lastCommit, lastHeaderHash, m.lastState) } -func (m *SSManager) applyBlock(ctx context.Context, block *types.Block) (types.State, *abci.ResponseFinalizeBlock, error) { +func (m *SSManager) applyBlock(ctx context.Context, block *types.Block) (*abci.ResponseFinalizeBlock, error) { m.lastStateMtx.RLock() defer m.lastStateMtx.RUnlock() return m.executor.ApplyBlock(ctx, m.lastState, block) diff --git a/node/full.go b/node/full.go index e20aebedf..4eb7886b5 100644 --- a/node/full.go +++ b/node/full.go @@ -27,7 +27,6 @@ import ( "github.com/astriaorg/rollkit/astria/execution" astriamempool "github.com/astriaorg/rollkit/astria/mempool" "github.com/astriaorg/rollkit/astria/sequencer" - "github.com/astriaorg/rollkit/astria/state/commitment" "github.com/astriaorg/rollkit/block" "github.com/astriaorg/rollkit/config" "github.com/astriaorg/rollkit/mempool" @@ -167,12 +166,15 @@ func newFullNode( return nil, err } + blockManager.CheckCrashRecovery(ctx) + // genesis info for exec api & sequencer client execGenesisInfo := execution.GenesisInfo{ RollupId: sha256.Sum256([]byte(genesis.ChainID)), SequencerGenesisBlockHeight: nodeConfig.Astria.SeqInitialHeight, CelestiaBaseBlockHeight: nodeConfig.DAStartHeight, CelestiaBlockVariance: nodeConfig.DAVariance, + GenesisTime: genesis.GenesisTime, } // init mempool reaper @@ -185,8 +187,7 @@ func newFullNode( reaper := astriamempool.NewMempoolReaper(seqClient, mempool, logger.With("module", "reaper")) // init grpc execution api - commitmentStore := commitment.NewCommitmentState(ctx, newPrefixKV(baseKV, commitmentPrefix)) - serviceV1a2 := execution.NewExecutionServiceServerV1Alpha2(blockManager, execGenesisInfo, commitmentStore, store, logger.With("module", "execution")) + serviceV1a2 := execution.NewExecutionServiceServerV1Alpha2(blockManager, execGenesisInfo, store, logger.With("module", "execution")) grpcServerHandler := execution.NewGRPCServerHandler(serviceV1a2, nodeConfig.Astria.GrpcListen, logger.With("module", "execution")) node := &FullNode{ @@ -272,7 +273,7 @@ func initMempool(logger log.Logger, proxyApp proxy.AppConns, memplMetrics *mempo // } func initBlockManager(signingKey crypto.PrivKey, nodeConfig config.NodeConfig, genesis *cmtypes.GenesisDoc, store store.Store, mempool mempool.Mempool, proxyApp proxy.AppConns, eventBus *cmtypes.EventBus, logger log.Logger, seqMetrics *block.Metrics, execMetrics *state.Metrics) (*block.SSManager, error) { - blockManager, err := block.NewSSManager(signingKey, nodeConfig.BlockManagerConfig, genesis, store, mempool, proxyApp.Consensus(), eventBus, logger.With("module", "BlockManager"), seqMetrics, execMetrics) + blockManager, err := block.NewSSManager(signingKey, nodeConfig.BlockManagerConfig, genesis, store, mempool, proxyApp, eventBus, logger.With("module", "BlockManager"), seqMetrics, execMetrics) if err != nil { return nil, fmt.Errorf("error while initializing BlockManager: %w", err) } diff --git a/state/executor.go b/state/executor.go index c08476e01..839d38973 100644 --- a/state/executor.go +++ b/state/executor.go @@ -275,53 +275,44 @@ func (e *BlockExecutor) ProcessProposal( } // ApplyBlock validates and executes the block. -func (e *BlockExecutor) ApplyBlock(ctx context.Context, state types.State, block *types.Block) (types.State, *abci.ResponseFinalizeBlock, error) { - isAppValid, err := e.ProcessProposal(block, state) +func (e *BlockExecutor) ApplyBlock(ctx context.Context, state types.State, block *types.Block) (*abci.ResponseFinalizeBlock, error) { + // isAppValid, err := e.ProcessProposal(block, state) + // if err != nil { + // return nil, err + // } + // if !isAppValid { + // return nil, fmt.Errorf("error while processing the proposal: %v", err) + // } + + err := e.Validate(state, block) if err != nil { - return types.State{}, nil, err - } - if !isAppValid { - return types.State{}, nil, fmt.Errorf("error while processing the proposal: %v", err) - } - - err = e.Validate(state, block) - if err != nil { - return types.State{}, nil, err + return nil, err } // This makes calls to the AppClient resp, err := e.execute(ctx, state, block) if err != nil { - return types.State{}, nil, err + return nil, err } if resp.ConsensusParamUpdates != nil { e.metrics.ConsensusParamUpdates.Add(1) } - state, err = e.updateState(state, block, resp) - if err != nil { - return types.State{}, nil, err - } - - if state.ConsensusParams.Block.MaxBytes == 0 { - e.logger.Error("maxBytes=0", "state.ConsensusParams.Block", state.ConsensusParams.Block, "block", block) - } - - return state, resp, nil + return resp, nil } // Commit commits the block -func (e *BlockExecutor) Commit(ctx context.Context, state types.State, block *types.Block, resp *abci.ResponseFinalizeBlock) ([]byte, uint64, error) { - appHash, retainHeight, err := e.commit(ctx, state, block, resp) +func (e *BlockExecutor) Commit(ctx context.Context, state types.State, block *types.Block, resp *abci.ResponseFinalizeBlock, skipExec bool) ([]byte, error) { + appHash, err := e.commit(ctx, state, block, resp, skipExec) if err != nil { - return []byte{}, 0, err + return []byte{}, err } state.AppHash = appHash e.publishEvents(resp, block, state) - return appHash, retainHeight, nil + return appHash, nil } // updateConsensusParams updates the consensus parameters based on the provided updates. @@ -336,7 +327,7 @@ func (e *BlockExecutor) updateConsensusParams(height uint64, params cmtypes.Cons return nextParams.ToProto(), nextParams.Version.App, nil } -func (e *BlockExecutor) updateState(state types.State, block *types.Block, finalizeBlockResponse *abci.ResponseFinalizeBlock) (types.State, error) { +func (e *BlockExecutor) UpdateState(state types.State, block *types.Block, finalizeBlockResponse *abci.ResponseFinalizeBlock) (types.State, error) { height := block.Height() if finalizeBlockResponse.ConsensusParamUpdates != nil { nextParamsProto, appVersion, err := e.updateConsensusParams(height, types.ConsensusParamsFromProto(state.ConsensusParams), finalizeBlockResponse.ConsensusParamUpdates) @@ -362,34 +353,41 @@ func (e *BlockExecutor) updateState(state types.State, block *types.Block, final ConsensusParams: state.ConsensusParams, LastHeightConsensusParamsChanged: state.LastHeightConsensusParamsChanged, AppHash: finalizeBlockResponse.AppHash, + DAHeight: state.DAHeight, } copy(s.LastResultsHash[:], cmtypes.NewResults(finalizeBlockResponse.TxResults).Hash()) + if s.ConsensusParams.Block.MaxBytes == 0 { + e.logger.Error("maxBytes=0", "state.ConsensusParams.Block", state.ConsensusParams.Block, "block", block) + } + return s, nil } -func (e *BlockExecutor) commit(ctx context.Context, state types.State, block *types.Block, resp *abci.ResponseFinalizeBlock) ([]byte, uint64, error) { +func (e *BlockExecutor) commit(ctx context.Context, state types.State, block *types.Block, resp *abci.ResponseFinalizeBlock, skipExec bool) ([]byte, error) { e.mempool.Lock() defer e.mempool.Unlock() err := e.mempool.FlushAppConn() if err != nil { - return nil, 0, err + return nil, err } - commitResp, err := e.proxyApp.Commit(ctx) - if err != nil { - return nil, 0, err + if skipExec == false { + _, err = e.proxyApp.Commit(ctx) + if err != nil { + return nil, err + } } maxBytes := state.ConsensusParams.Block.MaxBytes maxGas := state.ConsensusParams.Block.MaxGas err = e.mempool.Update(block.Height(), fromRollkitTxs(block.Data.Txs), resp.TxResults, mempool.PreCheckMaxBytes(maxBytes), mempool.PostCheckMaxGas(maxGas)) if err != nil { - return nil, 0, err + return nil, err } - return resp.AppHash, uint64(commitResp.RetainHeight), err + return resp.AppHash, err } // Validate validates the state and the block for the executor diff --git a/types/state.go b/types/state.go index 7c78ce867..4598b67e3 100644 --- a/types/state.go +++ b/types/state.go @@ -69,7 +69,7 @@ func NewFromGenesisDoc(genDoc *types.GenesisDoc) (State, error) { ChainID: genDoc.ChainID, InitialHeight: uint64(genDoc.InitialHeight), - DAHeight: 1, + DAHeight: 0, LastBlockHeight: uint64(genDoc.InitialHeight) - 1, LastBlockID: types.BlockID{},