diff options
author | Felix Lange <fjl@users.noreply.github.com> | 2017-09-10 00:03:07 +0800 |
---|---|---|
committer | Péter Szilágyi <peterke@gmail.com> | 2017-09-10 00:03:07 +0800 |
commit | 10181b57a9fb648f5fd424ca611820a3cf42c42b (patch) | |
tree | 8508c139bb867a6d2126fcbe6500cb08025ddbc1 /core | |
parent | ac193e36ce4bce752717124433a8ce84c347dbf7 (diff) | |
download | go-tangerine-10181b57a9fb648f5fd424ca611820a3cf42c42b.tar.gz go-tangerine-10181b57a9fb648f5fd424ca611820a3cf42c42b.tar.zst go-tangerine-10181b57a9fb648f5fd424ca611820a3cf42c42b.zip |
core, eth/downloader: commit block data using batches (#15115)
* ethdb: add Putter interface and Has method
* ethdb: improve docs and add IdealBatchSize
* ethdb: remove memory batch lock
Batches are not safe for concurrent use.
* core: use ethdb.Putter for Write* functions
This covers the easy cases.
* core/state: simplify StateSync
* trie: optimize local node check
* ethdb: add ValueSize to Batch
* core: optimize HasHeader check
This avoids one random database read get the block number. For many uses
of HasHeader, the expectation is that it's actually there. Using Has
avoids a load + decode of the value.
* core: write fast sync block data in batches
Collect writes into batches up to the ideal size instead of issuing many
small, concurrent writes.
* eth/downloader: commit larger state batches
Collect nodes into a batch up to the ideal size instead of committing
whenever a node is received.
* core: optimize HasBlock check
This avoids a random database read to get the number.
* core: use numberCache in HasHeader
numberCache has higher capacity, increasing the odds of finding the
header without a database lookup.
* core: write imported block data using a batch
Restore batch writes of state and add blocks, tx entries, receipts to
the same batch. The change also simplifies the miner.
This commit also removes posting of logs when a forked block is imported.
* core: fix DB write error handling
* ethdb: use RLock for Has
* core: fix HasBlock comment
Diffstat (limited to 'core')
-rw-r--r-- | core/blockchain.go | 224 | ||||
-rw-r--r-- | core/database_util.go | 53 | ||||
-rw-r--r-- | core/headerchain.go | 13 | ||||
-rw-r--r-- | core/state/sync.go | 42 |
4 files changed, 133 insertions, 199 deletions
diff --git a/core/blockchain.go b/core/blockchain.go index d74b3520b..25be8d762 100644 --- a/core/blockchain.go +++ b/core/blockchain.go @@ -23,7 +23,6 @@ import ( "io" "math/big" mrand "math/rand" - "runtime" "sync" "sync/atomic" "time" @@ -515,10 +514,13 @@ func (bc *BlockChain) GetBodyRLP(hash common.Hash) rlp.RawValue { return body } -// HasBlock checks if a block is fully present in the database or not, caching -// it if present. -func (bc *BlockChain) HasBlock(hash common.Hash) bool { - return bc.GetBlockByHash(hash) != nil +// HasBlock checks if a block is fully present in the database or not. +func (bc *BlockChain) HasBlock(hash common.Hash, number uint64) bool { + if bc.blockCache.Contains(hash) { + return true + } + ok, _ := bc.chainDb.Has(blockBodyKey(hash, number)) + return ok } // HasBlockAndState checks if a block and associated state trie is fully present @@ -693,108 +695,73 @@ func SetReceiptsData(config *params.ChainConfig, block *types.Block, receipts ty // InsertReceiptChain attempts to complete an already existing header chain with // transaction and receipt data. -// XXX should this be moved to the test? func (bc *BlockChain) InsertReceiptChain(blockChain types.Blocks, receiptChain []types.Receipts) (int, error) { + bc.wg.Add(1) + defer bc.wg.Done() + // Do a sanity check that the provided chain is actually ordered and linked for i := 1; i < len(blockChain); i++ { if blockChain[i].NumberU64() != blockChain[i-1].NumberU64()+1 || blockChain[i].ParentHash() != blockChain[i-1].Hash() { - // Chain broke ancestry, log a messge (programming error) and skip insertion log.Error("Non contiguous receipt insert", "number", blockChain[i].Number(), "hash", blockChain[i].Hash(), "parent", blockChain[i].ParentHash(), "prevnumber", blockChain[i-1].Number(), "prevhash", blockChain[i-1].Hash()) - return 0, fmt.Errorf("non contiguous insert: item %d is #%d [%x…], item %d is #%d [%x…] (parent [%x…])", i-1, blockChain[i-1].NumberU64(), blockChain[i-1].Hash().Bytes()[:4], i, blockChain[i].NumberU64(), blockChain[i].Hash().Bytes()[:4], blockChain[i].ParentHash().Bytes()[:4]) } } - // Pre-checks passed, start the block body and receipt imports - bc.wg.Add(1) - defer bc.wg.Done() - - // Collect some import statistics to report on - stats := struct{ processed, ignored int32 }{} - start := time.Now() - // Create the block importing task queue and worker functions - tasks := make(chan int, len(blockChain)) - for i := 0; i < len(blockChain) && i < len(receiptChain); i++ { - tasks <- i - } - close(tasks) - - errs, failed := make([]error, len(tasks)), int32(0) - process := func(worker int) { - for index := range tasks { - block, receipts := blockChain[index], receiptChain[index] + var ( + stats = struct{ processed, ignored int32 }{} + start = time.Now() + bytes = 0 + batch = bc.chainDb.NewBatch() + ) + for i, block := range blockChain { + receipts := receiptChain[i] + // Short circuit insertion if shutting down or processing failed + if atomic.LoadInt32(&bc.procInterrupt) == 1 { + return 0, nil + } + // Short circuit if the owner header is unknown + if !bc.HasHeader(block.Hash(), block.NumberU64()) { + return i, fmt.Errorf("containing header #%d [%x…] unknown", block.Number(), block.Hash().Bytes()[:4]) + } + // Skip if the entire data is already known + if bc.HasBlock(block.Hash(), block.NumberU64()) { + stats.ignored++ + continue + } + // Compute all the non-consensus fields of the receipts + SetReceiptsData(bc.config, block, receipts) + // Write all the data out into the database + if err := WriteBody(batch, block.Hash(), block.NumberU64(), block.Body()); err != nil { + return i, fmt.Errorf("failed to write block body: %v", err) + } + if err := WriteBlockReceipts(batch, block.Hash(), block.NumberU64(), receipts); err != nil { + return i, fmt.Errorf("failed to write block receipts: %v", err) + } + if err := WriteTxLookupEntries(batch, block); err != nil { + return i, fmt.Errorf("failed to write lookup metadata: %v", err) + } + stats.processed++ - // Short circuit insertion if shutting down or processing failed - if atomic.LoadInt32(&bc.procInterrupt) == 1 { - return - } - if atomic.LoadInt32(&failed) > 0 { - return - } - // Short circuit if the owner header is unknown - if !bc.HasHeader(block.Hash()) { - errs[index] = fmt.Errorf("containing header #%d [%x…] unknown", block.Number(), block.Hash().Bytes()[:4]) - atomic.AddInt32(&failed, 1) - return - } - // Skip if the entire data is already known - if bc.HasBlock(block.Hash()) { - atomic.AddInt32(&stats.ignored, 1) - continue - } - // Compute all the non-consensus fields of the receipts - SetReceiptsData(bc.config, block, receipts) - // Write all the data out into the database - if err := WriteBody(bc.chainDb, block.Hash(), block.NumberU64(), block.Body()); err != nil { - errs[index] = fmt.Errorf("failed to write block body: %v", err) - atomic.AddInt32(&failed, 1) - log.Crit("Failed to write block body", "err", err) - return + if batch.ValueSize() >= ethdb.IdealBatchSize { + if err := batch.Write(); err != nil { + return 0, err } - if err := WriteBlockReceipts(bc.chainDb, block.Hash(), block.NumberU64(), receipts); err != nil { - errs[index] = fmt.Errorf("failed to write block receipts: %v", err) - atomic.AddInt32(&failed, 1) - log.Crit("Failed to write block receipts", "err", err) - return - } - if err := WriteTxLookupEntries(bc.chainDb, block); err != nil { - errs[index] = fmt.Errorf("failed to write lookup metadata: %v", err) - atomic.AddInt32(&failed, 1) - log.Crit("Failed to write lookup metadata", "err", err) - return - } - atomic.AddInt32(&stats.processed, 1) + bytes += batch.ValueSize() + batch = bc.chainDb.NewBatch() } } - // Start as many worker threads as goroutines allowed - pending := new(sync.WaitGroup) - for i := 0; i < runtime.GOMAXPROCS(0); i++ { - pending.Add(1) - go func(id int) { - defer pending.Done() - process(id) - }(i) - } - pending.Wait() - - // If anything failed, report - if failed > 0 { - for i, err := range errs { - if err != nil { - return i, err - } + if batch.ValueSize() > 0 { + bytes += batch.ValueSize() + if err := batch.Write(); err != nil { + return 0, err } } - if atomic.LoadInt32(&bc.procInterrupt) == 1 { - log.Debug("Premature abort during receipts processing") - return 0, nil - } + // Update the head fast sync block if better bc.mu.Lock() - - head := blockChain[len(errs)-1] + head := blockChain[len(blockChain)-1] if td := bc.GetTd(head.Hash(), head.NumberU64()); td != nil { // Rewind may have occurred, skip in that case if bc.GetTd(bc.currentFastBlock.Hash(), bc.currentFastBlock.NumberU64()).Cmp(td) < 0 { if err := WriteHeadFastBlockHash(bc.chainDb, head.Hash()); err != nil { @@ -805,16 +772,18 @@ func (bc *BlockChain) InsertReceiptChain(blockChain types.Blocks, receiptChain [ } bc.mu.Unlock() - // Report some public statistics so the user has a clue what's going on - last := blockChain[len(blockChain)-1] - log.Info("Imported new block receipts", "count", stats.processed, "elapsed", common.PrettyDuration(time.Since(start)), - "number", last.Number(), "hash", last.Hash(), "ignored", stats.ignored) - + log.Info("Imported new block receipts", + "count", stats.processed, + "elapsed", common.PrettyDuration(time.Since(start)), + "bytes", bytes, + "number", head.Number(), + "hash", head.Hash(), + "ignored", stats.ignored) return 0, nil } // WriteBlock writes the block to the chain. -func (bc *BlockChain) WriteBlock(block *types.Block) (status WriteStatus, err error) { +func (bc *BlockChain) WriteBlockAndState(block *types.Block, receipts []*types.Receipt, state *state.StateDB) (status WriteStatus, err error) { bc.wg.Add(1) defer bc.wg.Done() @@ -827,7 +796,7 @@ func (bc *BlockChain) WriteBlock(block *types.Block) (status WriteStatus, err er bc.mu.Lock() defer bc.mu.Unlock() - if bc.HasBlock(block.Hash()) { + if bc.HasBlock(block.Hash(), block.NumberU64()) { log.Trace("Block existed", "hash", block.Hash()) return } @@ -837,10 +806,18 @@ func (bc *BlockChain) WriteBlock(block *types.Block) (status WriteStatus, err er // Irrelevant of the canonical status, write the block itself to the database if err := bc.hc.WriteTd(block.Hash(), block.NumberU64(), externTd); err != nil { - log.Crit("Failed to write block total difficulty", "err", err) + return NonStatTy, err } - if err := WriteBlock(bc.chainDb, block); err != nil { - log.Crit("Failed to write block contents", "err", err) + // Write other block data using a batch. + batch := bc.chainDb.NewBatch() + if err := WriteBlock(batch, block); err != nil { + return NonStatTy, err + } + if _, err := state.CommitTo(batch, bc.config.IsEIP158(block.Number())); err != nil { + return NonStatTy, err + } + if err := WriteBlockReceipts(batch, block.Hash(), block.NumberU64(), receipts); err != nil { + return NonStatTy, err } // If the total difficulty is higher than our known, add it to the canonical chain @@ -853,15 +830,28 @@ func (bc *BlockChain) WriteBlock(block *types.Block) (status WriteStatus, err er return NonStatTy, err } } - bc.insert(block) // Insert the block as the new head of the chain + // Write the positional metadata for transaction and receipt lookups + if err := WriteTxLookupEntries(batch, block); err != nil { + return NonStatTy, err + } + // Write hash preimages + if err := WritePreimages(bc.chainDb, block.NumberU64(), state.Preimages()); err != nil { + return NonStatTy, err + } status = CanonStatTy } else { status = SideStatTy } + if err := batch.Write(); err != nil { + return NonStatTy, err + } + // Set new head. + if status == CanonStatTy { + bc.insert(block) + } bc.futureBlocks.Remove(block.Hash()) - - return + return status, nil } // InsertChain will attempt to insert the given chain in to the canonical chain or, otherwise, create a fork. If an error is returned @@ -975,29 +965,18 @@ func (bc *BlockChain) InsertChain(chain types.Blocks) (int, error) { bc.reportBlock(block, receipts, err) return i, err } - // Write state changes to database - if _, err = state.CommitTo(bc.chainDb, bc.config.IsEIP158(block.Number())); err != nil { - return i, err - } - // coalesce logs for later processing - coalescedLogs = append(coalescedLogs, logs...) - - if err = WriteBlockReceipts(bc.chainDb, block.Hash(), block.NumberU64(), receipts); err != nil { - return i, err - } - - // write the block to the chain and get the status - status, err := bc.WriteBlock(block) + // Write the block to the chain and get the status. + status, err := bc.WriteBlockAndState(block, receipts, state) if err != nil { return i, err } - switch status { case CanonStatTy: log.Debug("Inserted new block", "number", block.Number(), "hash", block.Hash(), "uncles", len(block.Uncles()), "txs", len(block.Transactions()), "gas", block.GasUsed(), "elapsed", common.PrettyDuration(time.Since(bstart))) - + // coalesce logs for later processing + coalescedLogs = append(coalescedLogs, logs...) blockInsertTimer.UpdateSince(bstart) events = append(events, ChainEvent{block, block.Hash(), logs}) // We need some control over the mining operation. Acquiring locks and waiting @@ -1006,15 +985,6 @@ func (bc *BlockChain) InsertChain(chain types.Blocks) (int, error) { if bc.LastBlockHash() == block.Hash() { events = append(events, ChainHeadEvent{block}) } - - // Write the positional metadata for transaction and receipt lookups - if err := WriteTxLookupEntries(bc.chainDb, block); err != nil { - return i, err - } - // Write hash preimages - if err := WritePreimages(bc.chainDb, block.NumberU64(), state.Preimages()); err != nil { - return i, err - } case SideStatTy: log.Debug("Inserted forked block", "number", block.Number(), "hash", block.Hash(), "diff", block.Difficulty(), "elapsed", common.PrettyDuration(time.Since(bstart)), "txs", len(block.Transactions()), "gas", block.GasUsed(), "uncles", len(block.Uncles())) @@ -1357,8 +1327,8 @@ func (bc *BlockChain) GetHeaderByHash(hash common.Hash) *types.Header { // HasHeader checks if a block header is present in the database or not, caching // it if present. -func (bc *BlockChain) HasHeader(hash common.Hash) bool { - return bc.hc.HasHeader(hash) +func (bc *BlockChain) HasHeader(hash common.Hash, number uint64) bool { + return bc.hc.HasHeader(hash, number) } // GetBlockHashesFromHash retrieves a number of block hashes starting at a given diff --git a/core/database_util.go b/core/database_util.go index 989071104..1730a048e 100644 --- a/core/database_util.go +++ b/core/database_util.go @@ -38,11 +38,6 @@ type DatabaseReader interface { Get(key []byte) (value []byte, err error) } -// DatabaseWriter wraps the Put method of a backing data store. -type DatabaseWriter interface { - Put(key, value []byte) error -} - // DatabaseDeleter wraps the Delete method of a backing data store. type DatabaseDeleter interface { Delete(key []byte) error @@ -154,7 +149,7 @@ func GetHeadFastBlockHash(db DatabaseReader) common.Hash { // GetHeaderRLP retrieves a block header in its raw RLP database encoding, or nil // if the header's not found. func GetHeaderRLP(db DatabaseReader, hash common.Hash, number uint64) rlp.RawValue { - data, _ := db.Get(append(append(headerPrefix, encodeBlockNumber(number)...), hash.Bytes()...)) + data, _ := db.Get(headerKey(hash, number)) return data } @@ -175,10 +170,18 @@ func GetHeader(db DatabaseReader, hash common.Hash, number uint64) *types.Header // GetBodyRLP retrieves the block body (transactions and uncles) in RLP encoding. func GetBodyRLP(db DatabaseReader, hash common.Hash, number uint64) rlp.RawValue { - data, _ := db.Get(append(append(bodyPrefix, encodeBlockNumber(number)...), hash.Bytes()...)) + data, _ := db.Get(blockBodyKey(hash, number)) return data } +func headerKey(hash common.Hash, number uint64) []byte { + return append(append(headerPrefix, encodeBlockNumber(number)...), hash.Bytes()...) +} + +func blockBodyKey(hash common.Hash, number uint64) []byte { + return append(append(bodyPrefix, encodeBlockNumber(number)...), hash.Bytes()...) +} + // GetBody retrieves the block body (transactons, uncles) corresponding to the // hash, nil if none found. func GetBody(db DatabaseReader, hash common.Hash, number uint64) *types.Body { @@ -340,7 +343,7 @@ func GetBloomBits(db DatabaseReader, bit uint, section uint64, head common.Hash) } // WriteCanonicalHash stores the canonical hash for the given block number. -func WriteCanonicalHash(db DatabaseWriter, hash common.Hash, number uint64) error { +func WriteCanonicalHash(db ethdb.Putter, hash common.Hash, number uint64) error { key := append(append(headerPrefix, encodeBlockNumber(number)...), numSuffix...) if err := db.Put(key, hash.Bytes()); err != nil { log.Crit("Failed to store number to hash mapping", "err", err) @@ -349,7 +352,7 @@ func WriteCanonicalHash(db DatabaseWriter, hash common.Hash, number uint64) erro } // WriteHeadHeaderHash stores the head header's hash. -func WriteHeadHeaderHash(db DatabaseWriter, hash common.Hash) error { +func WriteHeadHeaderHash(db ethdb.Putter, hash common.Hash) error { if err := db.Put(headHeaderKey, hash.Bytes()); err != nil { log.Crit("Failed to store last header's hash", "err", err) } @@ -357,7 +360,7 @@ func WriteHeadHeaderHash(db DatabaseWriter, hash common.Hash) error { } // WriteHeadBlockHash stores the head block's hash. -func WriteHeadBlockHash(db DatabaseWriter, hash common.Hash) error { +func WriteHeadBlockHash(db ethdb.Putter, hash common.Hash) error { if err := db.Put(headBlockKey, hash.Bytes()); err != nil { log.Crit("Failed to store last block's hash", "err", err) } @@ -365,7 +368,7 @@ func WriteHeadBlockHash(db DatabaseWriter, hash common.Hash) error { } // WriteHeadFastBlockHash stores the fast head block's hash. -func WriteHeadFastBlockHash(db DatabaseWriter, hash common.Hash) error { +func WriteHeadFastBlockHash(db ethdb.Putter, hash common.Hash) error { if err := db.Put(headFastKey, hash.Bytes()); err != nil { log.Crit("Failed to store last fast block's hash", "err", err) } @@ -373,7 +376,7 @@ func WriteHeadFastBlockHash(db DatabaseWriter, hash common.Hash) error { } // WriteHeader serializes a block header into the database. -func WriteHeader(db DatabaseWriter, header *types.Header) error { +func WriteHeader(db ethdb.Putter, header *types.Header) error { data, err := rlp.EncodeToBytes(header) if err != nil { return err @@ -393,7 +396,7 @@ func WriteHeader(db DatabaseWriter, header *types.Header) error { } // WriteBody serializes the body of a block into the database. -func WriteBody(db DatabaseWriter, hash common.Hash, number uint64, body *types.Body) error { +func WriteBody(db ethdb.Putter, hash common.Hash, number uint64, body *types.Body) error { data, err := rlp.EncodeToBytes(body) if err != nil { return err @@ -402,7 +405,7 @@ func WriteBody(db DatabaseWriter, hash common.Hash, number uint64, body *types.B } // WriteBodyRLP writes a serialized body of a block into the database. -func WriteBodyRLP(db DatabaseWriter, hash common.Hash, number uint64, rlp rlp.RawValue) error { +func WriteBodyRLP(db ethdb.Putter, hash common.Hash, number uint64, rlp rlp.RawValue) error { key := append(append(bodyPrefix, encodeBlockNumber(number)...), hash.Bytes()...) if err := db.Put(key, rlp); err != nil { log.Crit("Failed to store block body", "err", err) @@ -411,7 +414,7 @@ func WriteBodyRLP(db DatabaseWriter, hash common.Hash, number uint64, rlp rlp.Ra } // WriteTd serializes the total difficulty of a block into the database. -func WriteTd(db DatabaseWriter, hash common.Hash, number uint64, td *big.Int) error { +func WriteTd(db ethdb.Putter, hash common.Hash, number uint64, td *big.Int) error { data, err := rlp.EncodeToBytes(td) if err != nil { return err @@ -424,7 +427,7 @@ func WriteTd(db DatabaseWriter, hash common.Hash, number uint64, td *big.Int) er } // WriteBlock serializes a block into the database, header and body separately. -func WriteBlock(db DatabaseWriter, block *types.Block) error { +func WriteBlock(db ethdb.Putter, block *types.Block) error { // Store the body first to retain database consistency if err := WriteBody(db, block.Hash(), block.NumberU64(), block.Body()); err != nil { return err @@ -439,7 +442,7 @@ func WriteBlock(db DatabaseWriter, block *types.Block) error { // WriteBlockReceipts stores all the transaction receipts belonging to a block // as a single receipt slice. This is used during chain reorganisations for // rescheduling dropped transactions. -func WriteBlockReceipts(db DatabaseWriter, hash common.Hash, number uint64, receipts types.Receipts) error { +func WriteBlockReceipts(db ethdb.Putter, hash common.Hash, number uint64, receipts types.Receipts) error { // Convert the receipts into their storage form and serialize them storageReceipts := make([]*types.ReceiptForStorage, len(receipts)) for i, receipt := range receipts { @@ -459,9 +462,7 @@ func WriteBlockReceipts(db DatabaseWriter, hash common.Hash, number uint64, rece // WriteTxLookupEntries stores a positional metadata for every transaction from // a block, enabling hash based transaction and receipt lookups. -func WriteTxLookupEntries(db ethdb.Database, block *types.Block) error { - batch := db.NewBatch() - +func WriteTxLookupEntries(db ethdb.Putter, block *types.Block) error { // Iterate over each transaction and encode its metadata for i, tx := range block.Transactions() { entry := txLookupEntry{ @@ -473,20 +474,16 @@ func WriteTxLookupEntries(db ethdb.Database, block *types.Block) error { if err != nil { return err } - if err := batch.Put(append(lookupPrefix, tx.Hash().Bytes()...), data); err != nil { + if err := db.Put(append(lookupPrefix, tx.Hash().Bytes()...), data); err != nil { return err } } - // Write the scheduled data into the database - if err := batch.Write(); err != nil { - log.Crit("Failed to store lookup entries", "err", err) - } return nil } // WriteBloomBits writes the compressed bloom bits vector belonging to the given // section and bit index. -func WriteBloomBits(db DatabaseWriter, bit uint, section uint64, head common.Hash, bits []byte) { +func WriteBloomBits(db ethdb.Putter, bit uint, section uint64, head common.Hash, bits []byte) { key := append(append(bloomBitsPrefix, make([]byte, 10)...), head.Bytes()...) binary.BigEndian.PutUint16(key[1:], uint16(bit)) @@ -572,13 +569,13 @@ func GetBlockChainVersion(db DatabaseReader) int { } // WriteBlockChainVersion writes vsn as the version number to db. -func WriteBlockChainVersion(db DatabaseWriter, vsn int) { +func WriteBlockChainVersion(db ethdb.Putter, vsn int) { enc, _ := rlp.EncodeToBytes(uint(vsn)) db.Put([]byte("BlockchainVersion"), enc) } // WriteChainConfig writes the chain config settings to the database. -func WriteChainConfig(db DatabaseWriter, hash common.Hash, cfg *params.ChainConfig) error { +func WriteChainConfig(db ethdb.Putter, hash common.Hash, cfg *params.ChainConfig) error { // short circuit and ignore if nil config. GetChainConfig // will return a default. if cfg == nil { diff --git a/core/headerchain.go b/core/headerchain.go index 6ec44b61d..0e5215293 100644 --- a/core/headerchain.go +++ b/core/headerchain.go @@ -267,7 +267,7 @@ func (hc *HeaderChain) InsertHeaderChain(chain []*types.Header, writeHeader WhCa return i, errors.New("aborted") } // If the header's already known, skip it, otherwise store - if hc.GetHeader(header.Hash(), header.Number.Uint64()) != nil { + if hc.HasHeader(header.Hash(), header.Number.Uint64()) { stats.ignored++ continue } @@ -361,10 +361,13 @@ func (hc *HeaderChain) GetHeaderByHash(hash common.Hash) *types.Header { return hc.GetHeader(hash, hc.GetBlockNumber(hash)) } -// HasHeader checks if a block header is present in the database or not, caching -// it if present. -func (hc *HeaderChain) HasHeader(hash common.Hash) bool { - return hc.GetHeaderByHash(hash) != nil +// HasHeader checks if a block header is present in the database or not. +func (hc *HeaderChain) HasHeader(hash common.Hash, number uint64) bool { + if hc.numberCache.Contains(hash) || hc.headerCache.Contains(hash) { + return true + } + ok, _ := hc.chainDb.Has(headerKey(hash, number)) + return ok } // GetHeaderByNumber retrieves a block header from the database by number, diff --git a/core/state/sync.go b/core/state/sync.go index 2c29d706a..28fcf6ae0 100644 --- a/core/state/sync.go +++ b/core/state/sync.go @@ -18,60 +18,24 @@ package state import ( "bytes" - "math/big" "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/rlp" "github.com/ethereum/go-ethereum/trie" ) -// StateSync is the main state synchronisation scheduler, which provides yet the -// unknown state hashes to retrieve, accepts node data associated with said hashes -// and reconstructs the state database step by step until all is done. -type StateSync trie.TrieSync - // NewStateSync create a new state trie download scheduler. -func NewStateSync(root common.Hash, database trie.DatabaseReader) *StateSync { +func NewStateSync(root common.Hash, database trie.DatabaseReader) *trie.TrieSync { var syncer *trie.TrieSync - callback := func(leaf []byte, parent common.Hash) error { - var obj struct { - Nonce uint64 - Balance *big.Int - Root common.Hash - CodeHash []byte - } + var obj Account if err := rlp.Decode(bytes.NewReader(leaf), &obj); err != nil { return err } syncer.AddSubTrie(obj.Root, 64, parent, nil) syncer.AddRawEntry(common.BytesToHash(obj.CodeHash), 64, parent) - return nil } syncer = trie.NewTrieSync(root, database, callback) - return (*StateSync)(syncer) -} - -// Missing retrieves the known missing nodes from the state trie for retrieval. -func (s *StateSync) Missing(max int) []common.Hash { - return (*trie.TrieSync)(s).Missing(max) -} - -// Process injects a batch of retrieved trie nodes data, returning if something -// was committed to the memcache and also the index of an entry if processing of -// it failed. -func (s *StateSync) Process(list []trie.SyncResult) (bool, int, error) { - return (*trie.TrieSync)(s).Process(list) -} - -// Commit flushes the data stored in the internal memcache out to persistent -// storage, returning th enumber of items written and any occurred error. -func (s *StateSync) Commit(dbw trie.DatabaseWriter) (int, error) { - return (*trie.TrieSync)(s).Commit(dbw) -} - -// Pending returns the number of state entries currently pending for download. -func (s *StateSync) Pending() int { - return (*trie.TrieSync)(s).Pending() + return syncer } |