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
This commit is contained in:
Felix Lange 2017-09-09 18:03:07 +02:00 committed by Péter Szilágyi
parent ac193e36ce
commit 10181b57a9
18 changed files with 247 additions and 280 deletions

View File

@ -164,7 +164,7 @@ func ImportChain(chain *core.BlockChain, fn string) error {
func hasAllBlocks(chain *core.BlockChain, bs []*types.Block) bool { func hasAllBlocks(chain *core.BlockChain, bs []*types.Block) bool {
for _, b := range bs { for _, b := range bs {
if !chain.HasBlock(b.Hash()) { if !chain.HasBlock(b.Hash(), b.NumberU64()) {
return false return false
} }
} }

View File

@ -23,7 +23,6 @@ import (
"io" "io"
"math/big" "math/big"
mrand "math/rand" mrand "math/rand"
"runtime"
"sync" "sync"
"sync/atomic" "sync/atomic"
"time" "time"
@ -515,10 +514,13 @@ func (bc *BlockChain) GetBodyRLP(hash common.Hash) rlp.RawValue {
return body return body
} }
// HasBlock checks if a block is fully present in the database or not, caching // HasBlock checks if a block is fully present in the database or not.
// it if present. func (bc *BlockChain) HasBlock(hash common.Hash, number uint64) bool {
func (bc *BlockChain) HasBlock(hash common.Hash) bool { if bc.blockCache.Contains(hash) {
return bc.GetBlockByHash(hash) != nil return true
}
ok, _ := bc.chainDb.Has(blockBodyKey(hash, number))
return ok
} }
// HasBlockAndState checks if a block and associated state trie is fully present // 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 // InsertReceiptChain attempts to complete an already existing header chain with
// transaction and receipt data. // transaction and receipt data.
// XXX should this be moved to the test?
func (bc *BlockChain) InsertReceiptChain(blockChain types.Blocks, receiptChain []types.Receipts) (int, error) { 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 // Do a sanity check that the provided chain is actually ordered and linked
for i := 1; i < len(blockChain); i++ { for i := 1; i < len(blockChain); i++ {
if blockChain[i].NumberU64() != blockChain[i-1].NumberU64()+1 || blockChain[i].ParentHash() != blockChain[i-1].Hash() { 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(), 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()) "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(), 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]) 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 var (
stats := struct{ processed, ignored int32 }{} stats = struct{ processed, ignored int32 }{}
start := time.Now() 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++
// Create the block importing task queue and worker functions if batch.ValueSize() >= ethdb.IdealBatchSize {
tasks := make(chan int, len(blockChain)) if err := batch.Write(); err != nil {
for i := 0; i < len(blockChain) && i < len(receiptChain); i++ { return 0, err
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]
// Short circuit insertion if shutting down or processing failed
if atomic.LoadInt32(&bc.procInterrupt) == 1 {
return
} }
if atomic.LoadInt32(&failed) > 0 { bytes += batch.ValueSize()
return batch = bc.chainDb.NewBatch()
}
// 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 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)
} }
} }
// Start as many worker threads as goroutines allowed if batch.ValueSize() > 0 {
pending := new(sync.WaitGroup) bytes += batch.ValueSize()
for i := 0; i < runtime.GOMAXPROCS(0); i++ { if err := batch.Write(); err != nil {
pending.Add(1) return 0, err
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 atomic.LoadInt32(&bc.procInterrupt) == 1 {
log.Debug("Premature abort during receipts processing")
return 0, nil
}
// Update the head fast sync block if better // Update the head fast sync block if better
bc.mu.Lock() bc.mu.Lock()
head := blockChain[len(blockChain)-1]
head := blockChain[len(errs)-1]
if td := bc.GetTd(head.Hash(), head.NumberU64()); td != nil { // Rewind may have occurred, skip in that case 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 bc.GetTd(bc.currentFastBlock.Hash(), bc.currentFastBlock.NumberU64()).Cmp(td) < 0 {
if err := WriteHeadFastBlockHash(bc.chainDb, head.Hash()); err != nil { if err := WriteHeadFastBlockHash(bc.chainDb, head.Hash()); err != nil {
@ -805,16 +772,18 @@ func (bc *BlockChain) InsertReceiptChain(blockChain types.Blocks, receiptChain [
} }
bc.mu.Unlock() bc.mu.Unlock()
// Report some public statistics so the user has a clue what's going on log.Info("Imported new block receipts",
last := blockChain[len(blockChain)-1] "count", stats.processed,
log.Info("Imported new block receipts", "count", stats.processed, "elapsed", common.PrettyDuration(time.Since(start)), "elapsed", common.PrettyDuration(time.Since(start)),
"number", last.Number(), "hash", last.Hash(), "ignored", stats.ignored) "bytes", bytes,
"number", head.Number(),
"hash", head.Hash(),
"ignored", stats.ignored)
return 0, nil return 0, nil
} }
// WriteBlock writes the block to the chain. // 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) bc.wg.Add(1)
defer bc.wg.Done() defer bc.wg.Done()
@ -827,7 +796,7 @@ func (bc *BlockChain) WriteBlock(block *types.Block) (status WriteStatus, err er
bc.mu.Lock() bc.mu.Lock()
defer bc.mu.Unlock() defer bc.mu.Unlock()
if bc.HasBlock(block.Hash()) { if bc.HasBlock(block.Hash(), block.NumberU64()) {
log.Trace("Block existed", "hash", block.Hash()) log.Trace("Block existed", "hash", block.Hash())
return 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 // Irrelevant of the canonical status, write the block itself to the database
if err := bc.hc.WriteTd(block.Hash(), block.NumberU64(), externTd); err != nil { 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 { // Write other block data using a batch.
log.Crit("Failed to write block contents", "err", err) 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 // 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 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 status = CanonStatTy
} else { } else {
status = SideStatTy 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()) bc.futureBlocks.Remove(block.Hash())
return status, nil
return
} }
// InsertChain will attempt to insert the given chain in to the canonical chain or, otherwise, create a fork. If an error is returned // 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) bc.reportBlock(block, receipts, err)
return i, 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 // Write the block to the chain and get the status.
coalescedLogs = append(coalescedLogs, logs...) status, err := bc.WriteBlockAndState(block, receipts, state)
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)
if err != nil { if err != nil {
return i, err return i, err
} }
switch status { switch status {
case CanonStatTy: case CanonStatTy:
log.Debug("Inserted new block", "number", block.Number(), "hash", block.Hash(), "uncles", len(block.Uncles()), 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))) "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) blockInsertTimer.UpdateSince(bstart)
events = append(events, ChainEvent{block, block.Hash(), logs}) events = append(events, ChainEvent{block, block.Hash(), logs})
// We need some control over the mining operation. Acquiring locks and waiting // 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() { if bc.LastBlockHash() == block.Hash() {
events = append(events, ChainHeadEvent{block}) 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: case SideStatTy:
log.Debug("Inserted forked block", "number", block.Number(), "hash", block.Hash(), "diff", block.Difficulty(), "elapsed", 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())) 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 // HasHeader checks if a block header is present in the database or not, caching
// it if present. // it if present.
func (bc *BlockChain) HasHeader(hash common.Hash) bool { func (bc *BlockChain) HasHeader(hash common.Hash, number uint64) bool {
return bc.hc.HasHeader(hash) return bc.hc.HasHeader(hash, number)
} }
// GetBlockHashesFromHash retrieves a number of block hashes starting at a given // GetBlockHashesFromHash retrieves a number of block hashes starting at a given

View File

@ -38,11 +38,6 @@ type DatabaseReader interface {
Get(key []byte) (value []byte, err error) 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. // DatabaseDeleter wraps the Delete method of a backing data store.
type DatabaseDeleter interface { type DatabaseDeleter interface {
Delete(key []byte) error 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 // GetHeaderRLP retrieves a block header in its raw RLP database encoding, or nil
// if the header's not found. // if the header's not found.
func GetHeaderRLP(db DatabaseReader, hash common.Hash, number uint64) rlp.RawValue { 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 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. // GetBodyRLP retrieves the block body (transactions and uncles) in RLP encoding.
func GetBodyRLP(db DatabaseReader, hash common.Hash, number uint64) rlp.RawValue { 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 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 // GetBody retrieves the block body (transactons, uncles) corresponding to the
// hash, nil if none found. // hash, nil if none found.
func GetBody(db DatabaseReader, hash common.Hash, number uint64) *types.Body { 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. // 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...) key := append(append(headerPrefix, encodeBlockNumber(number)...), numSuffix...)
if err := db.Put(key, hash.Bytes()); err != nil { if err := db.Put(key, hash.Bytes()); err != nil {
log.Crit("Failed to store number to hash mapping", "err", err) 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. // 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 { if err := db.Put(headHeaderKey, hash.Bytes()); err != nil {
log.Crit("Failed to store last header's hash", "err", err) 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. // 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 { if err := db.Put(headBlockKey, hash.Bytes()); err != nil {
log.Crit("Failed to store last block's hash", "err", err) 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. // 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 { if err := db.Put(headFastKey, hash.Bytes()); err != nil {
log.Crit("Failed to store last fast block's hash", "err", err) 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. // 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) data, err := rlp.EncodeToBytes(header)
if err != nil { if err != nil {
return err return err
@ -393,7 +396,7 @@ func WriteHeader(db DatabaseWriter, header *types.Header) error {
} }
// WriteBody serializes the body of a block into the database. // 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) data, err := rlp.EncodeToBytes(body)
if err != nil { if err != nil {
return err 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. // 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()...) key := append(append(bodyPrefix, encodeBlockNumber(number)...), hash.Bytes()...)
if err := db.Put(key, rlp); err != nil { if err := db.Put(key, rlp); err != nil {
log.Crit("Failed to store block body", "err", err) 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. // 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) data, err := rlp.EncodeToBytes(td)
if err != nil { if err != nil {
return err 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. // 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 // Store the body first to retain database consistency
if err := WriteBody(db, block.Hash(), block.NumberU64(), block.Body()); err != nil { if err := WriteBody(db, block.Hash(), block.NumberU64(), block.Body()); err != nil {
return err return err
@ -439,7 +442,7 @@ func WriteBlock(db DatabaseWriter, block *types.Block) error {
// WriteBlockReceipts stores all the transaction receipts belonging to a block // WriteBlockReceipts stores all the transaction receipts belonging to a block
// as a single receipt slice. This is used during chain reorganisations for // as a single receipt slice. This is used during chain reorganisations for
// rescheduling dropped transactions. // 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 // Convert the receipts into their storage form and serialize them
storageReceipts := make([]*types.ReceiptForStorage, len(receipts)) storageReceipts := make([]*types.ReceiptForStorage, len(receipts))
for i, receipt := range 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 // WriteTxLookupEntries stores a positional metadata for every transaction from
// a block, enabling hash based transaction and receipt lookups. // a block, enabling hash based transaction and receipt lookups.
func WriteTxLookupEntries(db ethdb.Database, block *types.Block) error { func WriteTxLookupEntries(db ethdb.Putter, block *types.Block) error {
batch := db.NewBatch()
// Iterate over each transaction and encode its metadata // Iterate over each transaction and encode its metadata
for i, tx := range block.Transactions() { for i, tx := range block.Transactions() {
entry := txLookupEntry{ entry := txLookupEntry{
@ -473,20 +474,16 @@ func WriteTxLookupEntries(db ethdb.Database, block *types.Block) error {
if err != nil { if err != nil {
return err 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 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 return nil
} }
// WriteBloomBits writes the compressed bloom bits vector belonging to the given // WriteBloomBits writes the compressed bloom bits vector belonging to the given
// section and bit index. // 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()...) key := append(append(bloomBitsPrefix, make([]byte, 10)...), head.Bytes()...)
binary.BigEndian.PutUint16(key[1:], uint16(bit)) 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. // 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)) enc, _ := rlp.EncodeToBytes(uint(vsn))
db.Put([]byte("BlockchainVersion"), enc) db.Put([]byte("BlockchainVersion"), enc)
} }
// WriteChainConfig writes the chain config settings to the database. // 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 // short circuit and ignore if nil config. GetChainConfig
// will return a default. // will return a default.
if cfg == nil { if cfg == nil {

View File

@ -267,7 +267,7 @@ func (hc *HeaderChain) InsertHeaderChain(chain []*types.Header, writeHeader WhCa
return i, errors.New("aborted") return i, errors.New("aborted")
} }
// If the header's already known, skip it, otherwise store // 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++ stats.ignored++
continue continue
} }
@ -361,10 +361,13 @@ func (hc *HeaderChain) GetHeaderByHash(hash common.Hash) *types.Header {
return hc.GetHeader(hash, hc.GetBlockNumber(hash)) return hc.GetHeader(hash, hc.GetBlockNumber(hash))
} }
// HasHeader checks if a block header is present in the database or not, caching // HasHeader checks if a block header is present in the database or not.
// it if present. func (hc *HeaderChain) HasHeader(hash common.Hash, number uint64) bool {
func (hc *HeaderChain) HasHeader(hash common.Hash) bool { if hc.numberCache.Contains(hash) || hc.headerCache.Contains(hash) {
return hc.GetHeaderByHash(hash) != nil return true
}
ok, _ := hc.chainDb.Has(headerKey(hash, number))
return ok
} }
// GetHeaderByNumber retrieves a block header from the database by number, // GetHeaderByNumber retrieves a block header from the database by number,

View File

@ -18,60 +18,24 @@ package state
import ( import (
"bytes" "bytes"
"math/big"
"github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/rlp" "github.com/ethereum/go-ethereum/rlp"
"github.com/ethereum/go-ethereum/trie" "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. // 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 var syncer *trie.TrieSync
callback := func(leaf []byte, parent common.Hash) error { callback := func(leaf []byte, parent common.Hash) error {
var obj struct { var obj Account
Nonce uint64
Balance *big.Int
Root common.Hash
CodeHash []byte
}
if err := rlp.Decode(bytes.NewReader(leaf), &obj); err != nil { if err := rlp.Decode(bytes.NewReader(leaf), &obj); err != nil {
return err return err
} }
syncer.AddSubTrie(obj.Root, 64, parent, nil) syncer.AddSubTrie(obj.Root, 64, parent, nil)
syncer.AddRawEntry(common.BytesToHash(obj.CodeHash), 64, parent) syncer.AddRawEntry(common.BytesToHash(obj.CodeHash), 64, parent)
return nil return nil
} }
syncer = trie.NewTrieSync(root, database, callback) syncer = trie.NewTrieSync(root, database, callback)
return (*StateSync)(syncer) return 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()
} }

View File

@ -241,7 +241,7 @@ func (api *PrivateAdminAPI) ExportChain(file string) (bool, error) {
func hasAllBlocks(chain *core.BlockChain, bs []*types.Block) bool { func hasAllBlocks(chain *core.BlockChain, bs []*types.Block) bool {
for _, b := range bs { for _, b := range bs {
if !chain.HasBlock(b.Hash()) { if !chain.HasBlock(b.Hash(), b.NumberU64()) {
return false return false
} }
} }

View File

@ -156,7 +156,7 @@ type Downloader struct {
// LightChain encapsulates functions required to synchronise a light chain. // LightChain encapsulates functions required to synchronise a light chain.
type LightChain interface { type LightChain interface {
// HasHeader verifies a header's presence in the local chain. // HasHeader verifies a header's presence in the local chain.
HasHeader(common.Hash) bool HasHeader(h common.Hash, number uint64) bool
// GetHeaderByHash retrieves a header from the local chain. // GetHeaderByHash retrieves a header from the local chain.
GetHeaderByHash(common.Hash) *types.Header GetHeaderByHash(common.Hash) *types.Header
@ -666,7 +666,7 @@ func (d *Downloader) findAncestor(p *peerConnection, height uint64) (uint64, err
continue continue
} }
// Otherwise check if we already know the header or not // Otherwise check if we already know the header or not
if (d.mode == FullSync && d.blockchain.HasBlockAndState(headers[i].Hash())) || (d.mode != FullSync && d.lightchain.HasHeader(headers[i].Hash())) { if (d.mode == FullSync && d.blockchain.HasBlockAndState(headers[i].Hash())) || (d.mode != FullSync && d.lightchain.HasHeader(headers[i].Hash(), headers[i].Number.Uint64())) {
number, hash = headers[i].Number.Uint64(), headers[i].Hash() number, hash = headers[i].Number.Uint64(), headers[i].Hash()
// If every header is known, even future ones, the peer straight out lied about its head // If every header is known, even future ones, the peer straight out lied about its head
@ -731,7 +731,7 @@ func (d *Downloader) findAncestor(p *peerConnection, height uint64) (uint64, err
arrived = true arrived = true
// Modify the search interval based on the response // Modify the search interval based on the response
if (d.mode == FullSync && !d.blockchain.HasBlockAndState(headers[0].Hash())) || (d.mode != FullSync && !d.lightchain.HasHeader(headers[0].Hash())) { if (d.mode == FullSync && !d.blockchain.HasBlockAndState(headers[0].Hash())) || (d.mode != FullSync && !d.lightchain.HasHeader(headers[0].Hash(), headers[0].Number.Uint64())) {
end = check end = check
break break
} }
@ -1256,7 +1256,7 @@ func (d *Downloader) processHeaders(origin uint64, td *big.Int) error {
// Collect the yet unknown headers to mark them as uncertain // Collect the yet unknown headers to mark them as uncertain
unknown := make([]*types.Header, 0, len(headers)) unknown := make([]*types.Header, 0, len(headers))
for _, header := range chunk { for _, header := range chunk {
if !d.lightchain.HasHeader(header.Hash()) { if !d.lightchain.HasHeader(header.Hash(), header.Number.Uint64()) {
unknown = append(unknown, header) unknown = append(unknown, header)
} }
} }

View File

@ -217,7 +217,7 @@ func (dl *downloadTester) sync(id string, td *big.Int, mode SyncMode) error {
} }
// HasHeader checks if a header is present in the testers canonical chain. // HasHeader checks if a header is present in the testers canonical chain.
func (dl *downloadTester) HasHeader(hash common.Hash) bool { func (dl *downloadTester) HasHeader(hash common.Hash, number uint64) bool {
return dl.GetHeaderByHash(hash) != nil return dl.GetHeaderByHash(hash) != nil
} }

View File

@ -26,6 +26,7 @@ import (
"github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/state" "github.com/ethereum/go-ethereum/core/state"
"github.com/ethereum/go-ethereum/crypto/sha3" "github.com/ethereum/go-ethereum/crypto/sha3"
"github.com/ethereum/go-ethereum/ethdb"
"github.com/ethereum/go-ethereum/log" "github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/trie" "github.com/ethereum/go-ethereum/trie"
) )
@ -187,10 +188,13 @@ func (d *Downloader) runStateSync(s *stateSync) *stateSync {
type stateSync struct { type stateSync struct {
d *Downloader // Downloader instance to access and manage current peerset d *Downloader // Downloader instance to access and manage current peerset
sched *state.StateSync // State trie sync scheduler defining the tasks sched *trie.TrieSync // State trie sync scheduler defining the tasks
keccak hash.Hash // Keccak256 hasher to verify deliveries with keccak hash.Hash // Keccak256 hasher to verify deliveries with
tasks map[common.Hash]*stateTask // Set of tasks currently queued for retrieval tasks map[common.Hash]*stateTask // Set of tasks currently queued for retrieval
numUncommitted int
bytesUncommitted int
deliver chan *stateReq // Delivery channel multiplexing peer responses deliver chan *stateReq // Delivery channel multiplexing peer responses
cancel chan struct{} // Channel to signal a termination request cancel chan struct{} // Channel to signal a termination request
cancelOnce sync.Once // Ensures cancel only ever gets called once cancelOnce sync.Once // Ensures cancel only ever gets called once
@ -252,9 +256,10 @@ func (s *stateSync) loop() error {
// Keep assigning new tasks until the sync completes or aborts // Keep assigning new tasks until the sync completes or aborts
for s.sched.Pending() > 0 { for s.sched.Pending() > 0 {
if err := s.assignTasks(); err != nil { if err := s.commit(false); err != nil {
return err return err
} }
s.assignTasks()
// Tasks assigned, wait for something to happen // Tasks assigned, wait for something to happen
select { select {
case <-newPeer: case <-newPeer:
@ -284,12 +289,28 @@ func (s *stateSync) loop() error {
} }
} }
} }
return s.commit(true)
}
func (s *stateSync) commit(force bool) error {
if !force && s.bytesUncommitted < ethdb.IdealBatchSize {
return nil
}
start := time.Now()
b := s.d.stateDB.NewBatch()
s.sched.Commit(b)
if err := b.Write(); err != nil {
return fmt.Errorf("DB write error: %v", err)
}
s.updateStats(s.numUncommitted, 0, 0, time.Since(start))
s.numUncommitted = 0
s.bytesUncommitted = 0
return nil return nil
} }
// assignTasks attempts to assing new tasks to all idle peers, either from the // assignTasks attempts to assing new tasks to all idle peers, either from the
// batch currently being retried, or fetching new data from the trie sync itself. // batch currently being retried, or fetching new data from the trie sync itself.
func (s *stateSync) assignTasks() error { func (s *stateSync) assignTasks() {
// Iterate over all idle peers and try to assign them state fetches // Iterate over all idle peers and try to assign them state fetches
peers, _ := s.d.peers.NodeDataIdlePeers() peers, _ := s.d.peers.NodeDataIdlePeers()
for _, p := range peers { for _, p := range peers {
@ -301,7 +322,6 @@ func (s *stateSync) assignTasks() error {
// If the peer was assigned tasks to fetch, send the network request // If the peer was assigned tasks to fetch, send the network request
if len(req.items) > 0 { if len(req.items) > 0 {
req.peer.log.Trace("Requesting new batch of data", "type", "state", "count", len(req.items)) req.peer.log.Trace("Requesting new batch of data", "type", "state", "count", len(req.items))
select { select {
case s.d.trackStateReq <- req: case s.d.trackStateReq <- req:
req.peer.FetchNodeData(req.items) req.peer.FetchNodeData(req.items)
@ -309,7 +329,6 @@ func (s *stateSync) assignTasks() error {
} }
} }
} }
return nil
} }
// fillTasks fills the given request object with a maximum of n state download // fillTasks fills the given request object with a maximum of n state download
@ -347,11 +366,11 @@ func (s *stateSync) fillTasks(n int, req *stateReq) {
// delivered. // delivered.
func (s *stateSync) process(req *stateReq) (bool, error) { func (s *stateSync) process(req *stateReq) (bool, error) {
// Collect processing stats and update progress if valid data was received // Collect processing stats and update progress if valid data was received
processed, written, duplicate, unexpected := 0, 0, 0, 0 duplicate, unexpected := 0, 0
defer func(start time.Time) { defer func(start time.Time) {
if processed+written+duplicate+unexpected > 0 { if duplicate > 0 || unexpected > 0 {
s.updateStats(processed, written, duplicate, unexpected, time.Since(start)) s.updateStats(0, duplicate, unexpected, time.Since(start))
} }
}(time.Now()) }(time.Now())
@ -362,7 +381,9 @@ func (s *stateSync) process(req *stateReq) (bool, error) {
prog, hash, err := s.processNodeData(blob) prog, hash, err := s.processNodeData(blob)
switch err { switch err {
case nil: case nil:
processed++ s.numUncommitted++
s.bytesUncommitted += len(blob)
progress = progress || prog
case trie.ErrNotRequested: case trie.ErrNotRequested:
unexpected++ unexpected++
case trie.ErrAlreadyProcessed: case trie.ErrAlreadyProcessed:
@ -370,38 +391,20 @@ func (s *stateSync) process(req *stateReq) (bool, error) {
default: default:
return stale, fmt.Errorf("invalid state node %s: %v", hash.TerminalString(), err) return stale, fmt.Errorf("invalid state node %s: %v", hash.TerminalString(), err)
} }
if prog {
progress = true
}
// If the node delivered a requested item, mark the delivery non-stale // If the node delivered a requested item, mark the delivery non-stale
if _, ok := req.tasks[hash]; ok { if _, ok := req.tasks[hash]; ok {
delete(req.tasks, hash) delete(req.tasks, hash)
stale = false stale = false
} }
} }
// If some data managed to hit the database, flush and reset failure counters // If we're inside the critical section, reset fail counter since we progressed.
if progress { if progress && atomic.LoadUint32(&s.d.fsPivotFails) > 1 {
// Flush any accumulated data out to disk log.Trace("Fast-sync progressed, resetting fail counter", "previous", atomic.LoadUint32(&s.d.fsPivotFails))
batch := s.d.stateDB.NewBatch() atomic.StoreUint32(&s.d.fsPivotFails, 1) // Don't ever reset to 0, as that will unlock the pivot block
count, err := s.sched.Commit(batch)
if err != nil {
return stale, err
}
if err := batch.Write(); err != nil {
return stale, err
}
written = count
// If we're inside the critical section, reset fail counter since we progressed
if atomic.LoadUint32(&s.d.fsPivotFails) > 1 {
log.Trace("Fast-sync progressed, resetting fail counter", "previous", atomic.LoadUint32(&s.d.fsPivotFails))
atomic.StoreUint32(&s.d.fsPivotFails, 1) // Don't ever reset to 0, as that will unlock the pivot block
}
} }
// Put unfulfilled tasks back into the retry queue // Put unfulfilled tasks back into the retry queue
npeers := s.d.peers.Len() npeers := s.d.peers.Len()
for hash, task := range req.tasks { for hash, task := range req.tasks {
// If the node did deliver something, missing items may be due to a protocol // If the node did deliver something, missing items may be due to a protocol
// limit or a previous timeout + delayed delivery. Both cases should permit // limit or a previous timeout + delayed delivery. Both cases should permit
@ -425,25 +428,25 @@ func (s *stateSync) process(req *stateReq) (bool, error) {
// error occurred. // error occurred.
func (s *stateSync) processNodeData(blob []byte) (bool, common.Hash, error) { func (s *stateSync) processNodeData(blob []byte) (bool, common.Hash, error) {
res := trie.SyncResult{Data: blob} res := trie.SyncResult{Data: blob}
s.keccak.Reset() s.keccak.Reset()
s.keccak.Write(blob) s.keccak.Write(blob)
s.keccak.Sum(res.Hash[:0]) s.keccak.Sum(res.Hash[:0])
committed, _, err := s.sched.Process([]trie.SyncResult{res}) committed, _, err := s.sched.Process([]trie.SyncResult{res})
return committed, res.Hash, err return committed, res.Hash, err
} }
// updateStats bumps the various state sync progress counters and displays a log // updateStats bumps the various state sync progress counters and displays a log
// message for the user to see. // message for the user to see.
func (s *stateSync) updateStats(processed, written, duplicate, unexpected int, duration time.Duration) { func (s *stateSync) updateStats(written, duplicate, unexpected int, duration time.Duration) {
s.d.syncStatsLock.Lock() s.d.syncStatsLock.Lock()
defer s.d.syncStatsLock.Unlock() defer s.d.syncStatsLock.Unlock()
s.d.syncStatsState.pending = uint64(s.sched.Pending()) s.d.syncStatsState.pending = uint64(s.sched.Pending())
s.d.syncStatsState.processed += uint64(processed) s.d.syncStatsState.processed += uint64(written)
s.d.syncStatsState.duplicate += uint64(duplicate) s.d.syncStatsState.duplicate += uint64(duplicate)
s.d.syncStatsState.unexpected += uint64(unexpected) s.d.syncStatsState.unexpected += uint64(unexpected)
log.Info("Imported new state entries", "count", processed, "flushed", written, "elapsed", common.PrettyDuration(duration), "processed", s.d.syncStatsState.processed, "pending", s.d.syncStatsState.pending, "retry", len(s.tasks), "duplicate", s.d.syncStatsState.duplicate, "unexpected", s.d.syncStatsState.unexpected) if written > 0 || duplicate > 0 || unexpected > 0 {
log.Info("Imported new state entries", "count", written, "elapsed", common.PrettyDuration(duration), "processed", s.d.syncStatsState.processed, "pending", s.d.syncStatsState.pending, "retry", len(s.tasks), "duplicate", s.d.syncStatsState.duplicate, "unexpected", s.d.syncStatsState.unexpected)
}
} }

View File

@ -609,7 +609,7 @@ func (pm *ProtocolManager) handleMsg(p *peer) error {
// Schedule all the unknown hashes for retrieval // Schedule all the unknown hashes for retrieval
unknown := make(newBlockHashesData, 0, len(announces)) unknown := make(newBlockHashesData, 0, len(announces))
for _, block := range announces { for _, block := range announces {
if !pm.blockchain.HasBlock(block.Hash) { if !pm.blockchain.HasBlock(block.Hash, block.Number) {
unknown = append(unknown, block) unknown = append(unknown, block)
} }
} }
@ -699,7 +699,7 @@ func (pm *ProtocolManager) BroadcastBlock(block *types.Block, propagate bool) {
return return
} }
// Otherwise if the block is indeed in out own chain, announce it // Otherwise if the block is indeed in out own chain, announce it
if pm.blockchain.HasBlock(hash) { if pm.blockchain.HasBlock(hash, block.NumberU64()) {
for _, peer := range peers { for _, peer := range peers {
peer.SendNewBlockHashes([]common.Hash{hash}, []uint64{block.NumberU64()}) peer.SendNewBlockHashes([]common.Hash{hash}, []uint64{block.NumberU64()})
} }

View File

@ -109,6 +109,10 @@ func (db *LDBDatabase) Put(key []byte, value []byte) error {
return db.db.Put(key, value, nil) return db.db.Put(key, value, nil)
} }
func (db *LDBDatabase) Has(key []byte) (bool, error) {
return db.db.Has(key, nil)
}
// Get returns the given key if it's present. // Get returns the given key if it's present.
func (db *LDBDatabase) Get(key []byte) ([]byte, error) { func (db *LDBDatabase) Get(key []byte) ([]byte, error) {
// Measure the database get latency, if requested // Measure the database get latency, if requested
@ -271,19 +275,19 @@ func (db *LDBDatabase) meter(refresh time.Duration) {
} }
} }
// TODO: remove this stuff and expose leveldb directly
func (db *LDBDatabase) NewBatch() Batch { func (db *LDBDatabase) NewBatch() Batch {
return &ldbBatch{db: db.db, b: new(leveldb.Batch)} return &ldbBatch{db: db.db, b: new(leveldb.Batch)}
} }
type ldbBatch struct { type ldbBatch struct {
db *leveldb.DB db *leveldb.DB
b *leveldb.Batch b *leveldb.Batch
size int
} }
func (b *ldbBatch) Put(key, value []byte) error { func (b *ldbBatch) Put(key, value []byte) error {
b.b.Put(key, value) b.b.Put(key, value)
b.size += len(value)
return nil return nil
} }
@ -291,6 +295,10 @@ func (b *ldbBatch) Write() error {
return b.db.Write(b.b, nil) return b.db.Write(b.b, nil)
} }
func (b *ldbBatch) ValueSize() int {
return b.size
}
type table struct { type table struct {
db Database db Database
prefix string prefix string
@ -309,6 +317,10 @@ func (dt *table) Put(key []byte, value []byte) error {
return dt.db.Put(append([]byte(dt.prefix), key...), value) return dt.db.Put(append([]byte(dt.prefix), key...), value)
} }
func (dt *table) Has(key []byte) (bool, error) {
return dt.db.Has(append([]byte(dt.prefix), key...))
}
func (dt *table) Get(key []byte) ([]byte, error) { func (dt *table) Get(key []byte) ([]byte, error) {
return dt.db.Get(append([]byte(dt.prefix), key...)) return dt.db.Get(append([]byte(dt.prefix), key...))
} }
@ -342,3 +354,7 @@ func (tb *tableBatch) Put(key, value []byte) error {
func (tb *tableBatch) Write() error { func (tb *tableBatch) Write() error {
return tb.batch.Write() return tb.batch.Write()
} }
func (tb *tableBatch) ValueSize() int {
return tb.batch.ValueSize()
}

View File

@ -16,15 +16,29 @@
package ethdb package ethdb
type Database interface { // Code using batches should try to add this much data to the batch.
// The value was determined empirically.
const IdealBatchSize = 100 * 1024
// Putter wraps the database write operation supported by both batches and regular databases.
type Putter interface {
Put(key []byte, value []byte) error Put(key []byte, value []byte) error
}
// Database wraps all database operations. All methods are safe for concurrent use.
type Database interface {
Putter
Get(key []byte) ([]byte, error) Get(key []byte) ([]byte, error)
Has(key []byte) (bool, error)
Delete(key []byte) error Delete(key []byte) error
Close() Close()
NewBatch() Batch NewBatch() Batch
} }
// Batch is a write-only database that commits changes to its host database
// when Write is called. Batch cannot be used concurrently.
type Batch interface { type Batch interface {
Put(key, value []byte) error Putter
ValueSize() int // amount of data in the batch
Write() error Write() error
} }

View File

@ -45,6 +45,14 @@ func (db *MemDatabase) Put(key []byte, value []byte) error {
return nil return nil
} }
func (db *MemDatabase) Has(key []byte) (bool, error) {
db.lock.RLock()
defer db.lock.RUnlock()
_, ok := db.db[string(key)]
return ok, nil
}
func (db *MemDatabase) Get(key []byte) ([]byte, error) { func (db *MemDatabase) Get(key []byte) ([]byte, error) {
db.lock.RLock() db.lock.RLock()
defer db.lock.RUnlock() defer db.lock.RUnlock()
@ -93,21 +101,16 @@ type kv struct{ k, v []byte }
type memBatch struct { type memBatch struct {
db *MemDatabase db *MemDatabase
writes []kv writes []kv
lock sync.RWMutex size int
} }
func (b *memBatch) Put(key, value []byte) error { func (b *memBatch) Put(key, value []byte) error {
b.lock.Lock()
defer b.lock.Unlock()
b.writes = append(b.writes, kv{common.CopyBytes(key), common.CopyBytes(value)}) b.writes = append(b.writes, kv{common.CopyBytes(key), common.CopyBytes(value)})
b.size += len(value)
return nil return nil
} }
func (b *memBatch) Write() error { func (b *memBatch) Write() error {
b.lock.RLock()
defer b.lock.RUnlock()
b.db.lock.Lock() b.db.lock.Lock()
defer b.db.lock.Unlock() defer b.db.lock.Unlock()
@ -116,3 +119,7 @@ func (b *memBatch) Write() error {
} }
return nil return nil
} }
func (b *memBatch) ValueSize() int {
return b.size
}

View File

@ -70,7 +70,7 @@ func errResp(code errCode, format string, v ...interface{}) error {
} }
type BlockChain interface { type BlockChain interface {
HasHeader(hash common.Hash) bool HasHeader(hash common.Hash, number uint64) bool
GetHeader(hash common.Hash, number uint64) *types.Header GetHeader(hash common.Hash, number uint64) *types.Header
GetHeaderByHash(hash common.Hash) *types.Header GetHeaderByHash(hash common.Hash) *types.Header
CurrentHeader() *types.Header CurrentHeader() *types.Header

View File

@ -252,8 +252,8 @@ func (self *LightChain) GetBodyRLP(ctx context.Context, hash common.Hash) (rlp.R
// HasBlock checks if a block is fully present in the database or not, caching // HasBlock checks if a block is fully present in the database or not, caching
// it if present. // it if present.
func (bc *LightChain) HasBlock(hash common.Hash) bool { func (bc *LightChain) HasBlock(hash common.Hash, number uint64) bool {
blk, _ := bc.GetBlockByHash(NoOdr, hash) blk, _ := bc.GetBlock(NoOdr, hash, number)
return blk != nil return blk != nil
} }
@ -418,8 +418,8 @@ func (self *LightChain) GetHeaderByHash(hash common.Hash) *types.Header {
// HasHeader checks if a block header is present in the database or not, caching // HasHeader checks if a block header is present in the database or not, caching
// it if present. // it if present.
func (bc *LightChain) HasHeader(hash common.Hash) bool { func (bc *LightChain) HasHeader(hash common.Hash, number uint64) bool {
return bc.hc.HasHeader(hash) return bc.hc.HasHeader(hash, number)
} }
// GetBlockHashesFromHash retrieves a number of block hashes starting at a given // GetBlockHashesFromHash retrieves a number of block hashes starting at a given

View File

@ -304,13 +304,8 @@ func (self *worker) wait() {
} }
go self.mux.Post(core.NewMinedBlockEvent{Block: block}) go self.mux.Post(core.NewMinedBlockEvent{Block: block})
} else { } else {
work.state.CommitTo(self.chainDb, self.config.IsEIP158(block.Number())) // Update the block hash in all logs since it is now available and not when the
stat, err := self.chain.WriteBlock(block) // receipt/log of individual transactions were created.
if err != nil {
log.Error("Failed writing block to chain", "err", err)
continue
}
// update block hash since it is now available and not when the receipt/log of individual transactions were created
for _, r := range work.receipts { for _, r := range work.receipts {
for _, l := range r.Logs { for _, l := range r.Logs {
l.BlockHash = block.Hash() l.BlockHash = block.Hash()
@ -319,15 +314,17 @@ func (self *worker) wait() {
for _, log := range work.state.Logs() { for _, log := range work.state.Logs() {
log.BlockHash = block.Hash() log.BlockHash = block.Hash()
} }
stat, err := self.chain.WriteBlockAndState(block, work.receipts, work.state)
if err != nil {
log.Error("Failed writing block to chain", "err", err)
continue
}
// check if canon block and write transactions // check if canon block and write transactions
if stat == core.CanonStatTy { if stat == core.CanonStatTy {
// This puts transactions in a extra db for rpc
core.WriteTxLookupEntries(self.chainDb, block)
// implicit by posting ChainHeadEvent // implicit by posting ChainHeadEvent
mustCommitNewWork = false mustCommitNewWork = false
} }
// broadcast before waiting for validation // broadcast before waiting for validation
go func(block *types.Block, logs []*types.Log, receipts []*types.Receipt) { go func(block *types.Block, logs []*types.Log, receipts []*types.Receipt) {
self.mux.Post(core.NewMinedBlockEvent{Block: block}) self.mux.Post(core.NewMinedBlockEvent{Block: block})
@ -336,16 +333,12 @@ func (self *worker) wait() {
coalescedLogs []*types.Log coalescedLogs []*types.Log
) )
events = append(events, core.ChainEvent{Block: block, Hash: block.Hash(), Logs: logs}) events = append(events, core.ChainEvent{Block: block, Hash: block.Hash(), Logs: logs})
if stat == core.CanonStatTy { if stat == core.CanonStatTy {
events = append(events, core.ChainHeadEvent{Block: block}) events = append(events, core.ChainHeadEvent{Block: block})
coalescedLogs = logs coalescedLogs = logs
} }
// post blockchain events // post blockchain events
self.chain.PostChainEvents(events, coalescedLogs) self.chain.PostChainEvents(events, coalescedLogs)
if err := core.WriteBlockReceipts(self.chainDb, block.Hash(), block.NumberU64(), receipts); err != nil {
log.Warn("Failed writing block receipts", "err", err)
}
}(block, work.state.Logs(), work.receipts) }(block, work.state.Logs(), work.receipts)
} }
// Insert the block into the set of pending ones to wait for confirmations // Insert the block into the set of pending ones to wait for confirmations

View File

@ -138,7 +138,7 @@ func (s *TrieSync) AddRawEntry(hash common.Hash, depth int, parent common.Hash)
if _, ok := s.membatch.batch[hash]; ok { if _, ok := s.membatch.batch[hash]; ok {
return return
} }
if blob, _ := s.database.Get(hash.Bytes()); blob != nil { if ok, _ := s.database.Has(hash.Bytes()); ok {
return return
} }
// Assemble the new sub-trie sync request // Assemble the new sub-trie sync request
@ -296,8 +296,7 @@ func (s *TrieSync) children(req *request, object node) ([]*request, error) {
if _, ok := s.membatch.batch[hash]; ok { if _, ok := s.membatch.batch[hash]; ok {
continue continue
} }
blob, _ := s.database.Get(node) if ok, _ := s.database.Has(node); ok {
if local, err := decodeNode(node[:], blob, 0); local != nil && err == nil {
continue continue
} }
// Locally unknown node, schedule for retrieval // Locally unknown node, schedule for retrieval

View File

@ -66,6 +66,7 @@ type Database interface {
// DatabaseReader wraps the Get method of a backing store for the trie. // DatabaseReader wraps the Get method of a backing store for the trie.
type DatabaseReader interface { type DatabaseReader interface {
Get(key []byte) (value []byte, err error) Get(key []byte) (value []byte, err error)
Has(key []byte) (bool, error)
} }
// DatabaseWriter wraps the Put method of a backing store for the trie. // DatabaseWriter wraps the Put method of a backing store for the trie.