Commit 10181b57 authored by Felix Lange's avatar Felix Lange Committed by Péter Szilágyi

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
parent ac193e36
......@@ -164,7 +164,7 @@ func ImportChain(chain *core.BlockChain, fn string) error {
func hasAllBlocks(chain *core.BlockChain, bs []*types.Block) bool {
for _, b := range bs {
if !chain.HasBlock(b.Hash()) {
if !chain.HasBlock(b.Hash(), b.NumberU64()) {
return false
}
}
......
This diff is collapsed.
......@@ -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 {
......
......@@ -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,
......
......@@ -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
}
......@@ -241,7 +241,7 @@ func (api *PrivateAdminAPI) ExportChain(file string) (bool, error) {
func hasAllBlocks(chain *core.BlockChain, bs []*types.Block) bool {
for _, b := range bs {
if !chain.HasBlock(b.Hash()) {
if !chain.HasBlock(b.Hash(), b.NumberU64()) {
return false
}
}
......
......@@ -156,7 +156,7 @@ type Downloader struct {
// LightChain encapsulates functions required to synchronise a light chain.
type LightChain interface {
// 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(common.Hash) *types.Header
......@@ -666,7 +666,7 @@ func (d *Downloader) findAncestor(p *peerConnection, height uint64) (uint64, err
continue
}
// 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()
// 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
// 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
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
unknown := make([]*types.Header, 0, len(headers))
for _, header := range chunk {
if !d.lightchain.HasHeader(header.Hash()) {
if !d.lightchain.HasHeader(header.Hash(), header.Number.Uint64()) {
unknown = append(unknown, header)
}
}
......
......@@ -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.
func (dl *downloadTester) HasHeader(hash common.Hash) bool {
func (dl *downloadTester) HasHeader(hash common.Hash, number uint64) bool {
return dl.GetHeaderByHash(hash) != nil
}
......
......@@ -26,6 +26,7 @@ import (
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/core/state"
"github.com/ethereum/go-ethereum/crypto/sha3"
"github.com/ethereum/go-ethereum/ethdb"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/trie"
)
......@@ -187,10 +188,13 @@ func (d *Downloader) runStateSync(s *stateSync) *stateSync {
type stateSync struct {
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
tasks map[common.Hash]*stateTask // Set of tasks currently queued for retrieval
numUncommitted int
bytesUncommitted int
deliver chan *stateReq // Delivery channel multiplexing peer responses
cancel chan struct{} // Channel to signal a termination request
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
for s.sched.Pending() > 0 {
if err := s.assignTasks(); err != nil {
if err := s.commit(false); err != nil {
return err
}
s.assignTasks()
// Tasks assigned, wait for something to happen
select {
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
}
// 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.
func (s *stateSync) assignTasks() error {
func (s *stateSync) assignTasks() {
// Iterate over all idle peers and try to assign them state fetches
peers, _ := s.d.peers.NodeDataIdlePeers()
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 len(req.items) > 0 {
req.peer.log.Trace("Requesting new batch of data", "type", "state", "count", len(req.items))
select {
case s.d.trackStateReq <- req:
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
......@@ -347,11 +366,11 @@ func (s *stateSync) fillTasks(n int, req *stateReq) {
// delivered.
func (s *stateSync) process(req *stateReq) (bool, error) {
// 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) {
if processed+written+duplicate+unexpected > 0 {
s.updateStats(processed, written, duplicate, unexpected, time.Since(start))
if duplicate > 0 || unexpected > 0 {
s.updateStats(0, duplicate, unexpected, time.Since(start))
}
}(time.Now())
......@@ -362,7 +381,9 @@ func (s *stateSync) process(req *stateReq) (bool, error) {
prog, hash, err := s.processNodeData(blob)
switch err {
case nil:
processed++
s.numUncommitted++
s.bytesUncommitted += len(blob)
progress = progress || prog
case trie.ErrNotRequested:
unexpected++
case trie.ErrAlreadyProcessed:
......@@ -370,38 +391,20 @@ func (s *stateSync) process(req *stateReq) (bool, error) {
default:
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 _, ok := req.tasks[hash]; ok {
delete(req.tasks, hash)
stale = false
}
}
// If some data managed to hit the database, flush and reset failure counters
if progress {
// Flush any accumulated data out to disk
batch := s.d.stateDB.NewBatch()
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
}
// If we're inside the critical section, reset fail counter since we progressed.
if progress && 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
npeers := s.d.peers.Len()
for hash, task := range req.tasks {
// 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
......@@ -425,25 +428,25 @@ func (s *stateSync) process(req *stateReq) (bool, error) {
// error occurred.
func (s *stateSync) processNodeData(blob []byte) (bool, common.Hash, error) {
res := trie.SyncResult{Data: blob}
s.keccak.Reset()
s.keccak.Write(blob)
s.keccak.Sum(res.Hash[:0])
committed, _, err := s.sched.Process([]trie.SyncResult{res})
return committed, res.Hash, err
}
// updateStats bumps the various state sync progress counters and displays a log
// 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()
defer s.d.syncStatsLock.Unlock()
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.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)
}
}
......@@ -609,7 +609,7 @@ func (pm *ProtocolManager) handleMsg(p *peer) error {
// Schedule all the unknown hashes for retrieval
unknown := make(newBlockHashesData, 0, len(announces))
for _, block := range announces {
if !pm.blockchain.HasBlock(block.Hash) {
if !pm.blockchain.HasBlock(block.Hash, block.Number) {
unknown = append(unknown, block)
}
}
......@@ -699,7 +699,7 @@ func (pm *ProtocolManager) BroadcastBlock(block *types.Block, propagate bool) {
return
}
// 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 {
peer.SendNewBlockHashes([]common.Hash{hash}, []uint64{block.NumberU64()})
}
......
......@@ -109,6 +109,10 @@ func (db *LDBDatabase) Put(key []byte, value []byte) error {
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.
func (db *LDBDatabase) Get(key []byte) ([]byte, error) {
// 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 {
return &ldbBatch{db: db.db, b: new(leveldb.Batch)}
}
type ldbBatch struct {
db *leveldb.DB
b *leveldb.Batch
db *leveldb.DB
b *leveldb.Batch
size int
}
func (b *ldbBatch) Put(key, value []byte) error {
b.b.Put(key, value)
b.size += len(value)
return nil
}
......@@ -291,6 +295,10 @@ func (b *ldbBatch) Write() error {
return b.db.Write(b.b, nil)
}
func (b *ldbBatch) ValueSize() int {
return b.size
}
type table struct {
db Database
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)
}
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) {
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 {
return tb.batch.Write()
}
func (tb *tableBatch) ValueSize() int {
return tb.batch.ValueSize()
}
......@@ -16,15 +16,29 @@
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
}
// Database wraps all database operations. All methods are safe for concurrent use.
type Database interface {
Putter
Get(key []byte) ([]byte, error)
Has(key []byte) (bool, error)
Delete(key []byte) error
Close()
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 {
Put(key, value []byte) error
Putter
ValueSize() int // amount of data in the batch
Write() error
}
......@@ -45,6 +45,14 @@ func (db *MemDatabase) Put(key []byte, value []byte) error {
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) {
db.lock.RLock()
defer db.lock.RUnlock()
......@@ -93,21 +101,16 @@ type kv struct{ k, v []byte }
type memBatch struct {
db *MemDatabase
writes []kv
lock sync.RWMutex
size int
}
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.size += len(value)
return nil
}
func (b *memBatch) Write() error {
b.lock.RLock()
defer b.lock.RUnlock()
b.db.lock.Lock()
defer b.db.lock.Unlock()
......@@ -116,3 +119,7 @@ func (b *memBatch) Write() error {
}
return nil
}
func (b *memBatch) ValueSize() int {
return b.size
}
......@@ -70,7 +70,7 @@ func errResp(code errCode, format string, v ...interface{}) error {
}
type BlockChain interface {
HasHeader(hash common.Hash) bool
HasHeader(hash common.Hash, number uint64) bool
GetHeader(hash common.Hash, number uint64) *types.Header
GetHeaderByHash(hash common.Hash) *types.Header
CurrentHeader() *types.Header
......
......@@ -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
// it if present.
func (bc *LightChain) HasBlock(hash common.Hash) bool {
blk, _ := bc.GetBlockByHash(NoOdr, hash)
func (bc *LightChain) HasBlock(hash common.Hash, number uint64) bool {
blk, _ := bc.GetBlock(NoOdr, hash, number)
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
// it if present.
func (bc *LightChain) HasHeader(hash common.Hash) bool {
return bc.hc.HasHeader(hash)
func (bc *LightChain) HasHeader(hash common.Hash, number uint64) bool {
return bc.hc.HasHeader(hash, number)
}
// GetBlockHashesFromHash retrieves a number of block hashes starting at a given
......
......@@ -304,13 +304,8 @@ func (self *worker) wait() {
}
go self.mux.Post(core.NewMinedBlockEvent{Block: block})
} else {
work.state.CommitTo(self.chainDb, self.config.IsEIP158(block.Number()))
stat, err := self.chain.WriteBlock(block)
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
// Update the block hash in all logs since it is now available and not when the
// receipt/log of individual transactions were created.
for _, r := range work.receipts {
for _, l := range r.Logs {
l.BlockHash = block.Hash()
......@@ -319,15 +314,17 @@ func (self *worker) wait() {
for _, log := range work.state.Logs() {
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
if stat == core.CanonStatTy {
// This puts transactions in a extra db for rpc
core.WriteTxLookupEntries(self.chainDb, block)
// implicit by posting ChainHeadEvent
mustCommitNewWork = false
}
// broadcast before waiting for validation
go func(block *types.Block, logs []*types.Log, receipts []*types.Receipt) {
self.mux.Post(core.NewMinedBlockEvent{Block: block})
......@@ -336,16 +333,12 @@ func (self *worker) wait() {
coalescedLogs []*types.Log
)
events = append(events, core.ChainEvent{Block: block, Hash: block.Hash(), Logs: logs})
if stat == core.CanonStatTy {
events = append(events, core.ChainHeadEvent{Block: block})
coalescedLogs = logs
}
// post blockchain events
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)
}
// Insert the block into the set of pending ones to wait for confirmations
......
......@@ -138,7 +138,7 @@ func (s *TrieSync) AddRawEntry(hash common.Hash, depth int, parent common.Hash)
if _, ok := s.membatch.batch[hash]; ok {
return
}
if blob, _ := s.database.Get(hash.Bytes()); blob != nil {
if ok, _ := s.database.Has(hash.Bytes()); ok {
return
}
// 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 {
continue
}
blob, _ := s.database.Get(node)
if local, err := decodeNode(node[:], blob, 0); local != nil && err == nil {
if ok, _ := s.database.Has(node); ok {
continue
}
// Locally unknown node, schedule for retrieval
......
......@@ -66,6 +66,7 @@ type Database interface {
// DatabaseReader wraps the Get method of a backing store for the trie.
type DatabaseReader interface {
Get(key []byte) (value []byte, err error)
Has(key []byte) (bool, error)
}
// DatabaseWriter wraps the Put method of a backing store for the trie.
......
Markdown is supported
0% or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment