From 493903eedecaae3d14966cd99aa84d146ea0ce13 Mon Sep 17 00:00:00 2001 From: Martin Holst Swende Date: Sat, 20 Oct 2018 10:43:59 +0200 Subject: [PATCH 01/48] core: better side-chain importing --- core/blockchain.go | 367 +++++++++++++++++++++++++++++++----------- node/node.go | 2 +- p2p/discover/table.go | 2 +- 3 files changed, 274 insertions(+), 97 deletions(-) diff --git a/core/blockchain.go b/core/blockchain.go index d173b2de2..74ac30e70 100644 --- a/core/blockchain.go +++ b/core/blockchain.go @@ -1048,6 +1048,80 @@ func (bc *BlockChain) InsertChain(chain types.Blocks) (int, error) { return n, err } +// addFutureBlock checks if the block is within the max allowed window to get accepted for future processing, and +// returns an error if the block is too far ahead and was not added. +func (bc *BlockChain) addFutureBlock(block *types.Block) error { + max := big.NewInt(time.Now().Unix() + maxTimeFutureBlocks) + if block.Time().Cmp(max) > 0 { + return fmt.Errorf("future block: %v > %v", block.Time(), max) + } + bc.futureBlocks.Add(block.Hash(), block) + return nil +} + +// importBatch is a helper function to assist during chain import +type importBatch struct { + chain types.Blocks + results <-chan error + index int + validator Validator +} + +// newBatch creates a new batch based on the given blocks, which are assumed to be a contiguous chain +func newBatch(chain types.Blocks, results <-chan error, validator Validator) *importBatch { + return &importBatch{ + chain: chain, + results: results, + index: -1, + validator: validator, + } +} + +// next returns the next block in the batch, along with any potential validation error for that block +// When the end is reached, it will return (nil, nil), but Current() will always return the last element. +func (batch *importBatch) next() (*types.Block, error) { + if batch.index+1 >= len(batch.chain) { + return nil, nil + } + batch.index++ + if err := <-batch.results; err != nil { + return batch.chain[batch.index], err + } + return batch.chain[batch.index], batch.validator.ValidateBody(batch.chain[batch.index]) +} + +// current returns the current block that's being processed. Even after the next() has progressed the entire +// chain, current will always return the last element +func (batch *importBatch) current() *types.Block { + if batch.index < 0 { + return nil + } + return batch.chain[batch.index] +} + +// previous returns the previous block was being processed, or nil +func (batch *importBatch) previous() *types.Block { + if batch.index < 1 { + return nil + } + return batch.chain[batch.index-1] +} + +// first returns the first block in the batch +func (batch *importBatch) first() *types.Block { + return batch.chain[0] +} + +// remaining returns the number of remaining blocks +func (batch *importBatch) remaining() int { + return len(batch.chain) - batch.index +} + +// processed returns the number of processed blocks +func (batch *importBatch) processed() int { + return batch.index + 1 +} + // insertChain will execute the actual chain insertion and event aggregation. The // only reason this method exists as a separate one is to make locking cleaner // with deferred statements. @@ -1067,12 +1141,27 @@ func (bc *BlockChain) insertChain(chain types.Blocks) (int, []interface{}, []*ty chain[i-1].Hash().Bytes()[:4], i, chain[i].NumberU64(), chain[i].Hash().Bytes()[:4], chain[i].ParentHash().Bytes()[:4]) } } + log.Info("insertChain", "from", chain[0].NumberU64(), "to", chain[len(chain)-1].NumberU64()) + // Pre-checks passed, start the full block imports bc.wg.Add(1) defer bc.wg.Done() - bc.chainmu.Lock() defer bc.chainmu.Unlock() + return bc.insertChainInternal(chain, true) +} + +//insertChainInternal is the internal implementation of insertChain, which assumes that +// 1. chains are contiguous, and +// 2. The `chainMu` lock is held +// This method is split out so that import batches that require re-injecting historical blocks can do +// so without releasing the lock, which could lead to racey behaviour. If a sidechain import is in progress, +// and the historic state is imported, but then new canon-head is added before the actual sidechain completes, +// then the historic state could be pruned again +func (bc *BlockChain) insertChainInternal(chain types.Blocks, verifySeals bool) (int, []interface{}, []*types.Log, error) { + + // Start a parallel signature recovery (signer will fluke on fork transition, minimal perf loss) + senderCacher.recoverFromBlocks(types.MakeSigner(bc.chainConfig, chain[0].Number()), chain) // A queued approach to delivering events. This is generally // faster than direct delivery and requires much less mutex @@ -1082,6 +1171,8 @@ func (bc *BlockChain) insertChain(chain types.Blocks) (int, []interface{}, []*ty events = make([]interface{}, 0, len(chain)) lastCanon *types.Block coalescedLogs []*types.Log + block *types.Block + err error ) // Start the parallel header verifier headers := make([]*types.Header, len(chain)) @@ -1089,16 +1180,57 @@ func (bc *BlockChain) insertChain(chain types.Blocks) (int, []interface{}, []*ty for i, block := range chain { headers[i] = block.Header() - seals[i] = true + seals[i] = verifySeals } abort, results := bc.engine.VerifyHeaders(bc, headers, seals) defer close(abort) - // Start a parallel signature recovery (signer will fluke on fork transition, minimal perf loss) - senderCacher.recoverFromBlocks(types.MakeSigner(bc.chainConfig, chain[0].Number()), chain) + // Peek the error for the first block + batch := newBatch(chain, results, bc.Validator()) + if block, err = batch.next(); err != nil { + if err == consensus.ErrPrunedAncestor { + return bc.insertSidechainInternal(batch, err) + } else if err == consensus.ErrFutureBlock || + (err == consensus.ErrUnknownAncestor && bc.futureBlocks.Contains(batch.first().ParentHash())) { - // Iterate over the blocks and insert when the verifier permits - for i, block := range chain { + // The first block is a future block + // We can shove that one and any child blocks (that fail because of UnknownAncestor) into the future-queue + for block != nil && (batch.index == 0 || err == consensus.ErrUnknownAncestor) { + block := batch.current() + if futureError := bc.addFutureBlock(block); futureError != nil { + return batch.index, events, coalescedLogs, futureError + } + block, err = batch.next() + } + stats.queued += batch.processed() + stats.ignored += batch.remaining() + + // If there are any still remaining, mark as ignored + return batch.index, events, coalescedLogs, err + } else if err == ErrKnownBlock { + + // Block and state both already known -- there can be two explanations. + // 1. We did a roll-back, and should now do a re-import + // 2. The block is stored as a sidechain, and is lying about it's stateroot, and passes a stateroot + // from the canonical chain, which has not been verified. + + // Skip all known blocks that are blocks behind us + currentNum := bc.CurrentBlock().NumberU64() + for block != nil && err == ErrKnownBlock && currentNum >= block.NumberU64() { + // We ignore these + stats.ignored++ + block, err = batch.next() + } + // Falls through to the block import + } else { + // Some other error + stats.ignored += len(batch.chain) + bc.reportBlock(block, nil, err) + return batch.index, events, coalescedLogs, err + } + } + // No validation errors + for ; block != nil && err == nil; block, err = batch.next() { // If the chain is terminating, stop processing blocks if atomic.LoadInt32(&bc.procInterrupt) == 1 { log.Debug("Premature abort during blocks processing") @@ -1107,115 +1239,45 @@ func (bc *BlockChain) insertChain(chain types.Blocks) (int, []interface{}, []*ty // If the header is a banned one, straight out abort if BadHashes[block.Hash()] { bc.reportBlock(block, nil, ErrBlacklistedHash) - return i, events, coalescedLogs, ErrBlacklistedHash + return batch.index, events, coalescedLogs, ErrBlacklistedHash } - // Wait for the block's verification to complete bstart := time.Now() - - err := <-results - if err == nil { - err = bc.Validator().ValidateBody(block) - } - switch { - case err == ErrKnownBlock: - // Block and state both already known. However if the current block is below - // this number we did a rollback and we should reimport it nonetheless. - if bc.CurrentBlock().NumberU64() >= block.NumberU64() { - stats.ignored++ - continue - } - - case err == consensus.ErrFutureBlock: - // Allow up to MaxFuture second in the future blocks. If this limit is exceeded - // the chain is discarded and processed at a later time if given. - max := big.NewInt(time.Now().Unix() + maxTimeFutureBlocks) - if block.Time().Cmp(max) > 0 { - return i, events, coalescedLogs, fmt.Errorf("future block: %v > %v", block.Time(), max) - } - bc.futureBlocks.Add(block.Hash(), block) - stats.queued++ - continue - - case err == consensus.ErrUnknownAncestor && bc.futureBlocks.Contains(block.ParentHash()): - bc.futureBlocks.Add(block.Hash(), block) - stats.queued++ - continue - - case err == consensus.ErrPrunedAncestor: - // Block competing with the canonical chain, store in the db, but don't process - // until the competitor TD goes above the canonical TD - currentBlock := bc.CurrentBlock() - localTd := bc.GetTd(currentBlock.Hash(), currentBlock.NumberU64()) - externTd := new(big.Int).Add(bc.GetTd(block.ParentHash(), block.NumberU64()-1), block.Difficulty()) - if localTd.Cmp(externTd) > 0 { - if err = bc.WriteBlockWithoutState(block, externTd); err != nil { - return i, events, coalescedLogs, err - } - continue - } - // Competitor chain beat canonical, gather all blocks from the common ancestor - var winner []*types.Block - - parent := bc.GetBlock(block.ParentHash(), block.NumberU64()-1) - for !bc.HasState(parent.Root()) { - winner = append(winner, parent) - parent = bc.GetBlock(parent.ParentHash(), parent.NumberU64()-1) - } - for j := 0; j < len(winner)/2; j++ { - winner[j], winner[len(winner)-1-j] = winner[len(winner)-1-j], winner[j] - } - // Import all the pruned blocks to make the state available - bc.chainmu.Unlock() - _, evs, logs, err := bc.insertChain(winner) - bc.chainmu.Lock() - events, coalescedLogs = evs, logs - - if err != nil { - return i, events, coalescedLogs, err - } - - case err != nil: - bc.reportBlock(block, nil, err) - return i, events, coalescedLogs, err - } - // Create a new statedb using the parent block and report an - // error if it fails. var parent *types.Block - if i == 0 { + parent = batch.previous() + if parent == nil { parent = bc.GetBlock(block.ParentHash(), block.NumberU64()-1) - } else { - parent = chain[i-1] } + state, err := state.New(parent.Root(), bc.stateCache) if err != nil { - return i, events, coalescedLogs, err + return batch.index, events, coalescedLogs, err } // Process block using the parent state as reference point. receipts, logs, usedGas, err := bc.processor.Process(block, state, bc.vmConfig) if err != nil { bc.reportBlock(block, receipts, err) - return i, events, coalescedLogs, err + return batch.index, events, coalescedLogs, err } // Validate the state using the default validator - err = bc.Validator().ValidateState(block, parent, state, receipts, usedGas) - if err != nil { + if err := bc.Validator().ValidateState(block, parent, state, receipts, usedGas); err != nil { bc.reportBlock(block, receipts, err) - return i, events, coalescedLogs, err + return batch.index, events, coalescedLogs, err } proctime := time.Since(bstart) // Write the block to the chain and get the status. status, err := bc.WriteBlockWithState(block, receipts, state) if err != nil { - return i, events, coalescedLogs, err + return batch.index, events, coalescedLogs, 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))) + 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)), + "root", block.Root().String()) coalescedLogs = append(coalescedLogs, logs...) - blockInsertTimer.UpdateSince(bstart) events = append(events, ChainEvent{block, block.Hash(), logs}) lastCanon = block @@ -1223,23 +1285,138 @@ func (bc *BlockChain) insertChain(chain types.Blocks) (int, []interface{}, []*ty bc.gcproc += proctime 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())) - - blockInsertTimer.UpdateSince(bstart) + 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()), + "root", block.Root().String()) events = append(events, ChainSideEvent{block}) } + blockInsertTimer.UpdateSince(bstart) stats.processed++ stats.usedGas += usedGas cache, _ := bc.stateCache.TrieDB().Size() - stats.report(chain, i, cache) + stats.report(chain, batch.index, cache) } + + // Any blocks remaining here? If so, the only ones we need to care about are + // shoving future blocks into queue + if block != nil && err == consensus.ErrFutureBlock { + if futureErr := bc.addFutureBlock(block); futureErr != nil { + return batch.index, events, coalescedLogs, futureErr + } + for block, err = batch.next(); block != nil && err == consensus.ErrUnknownAncestor; { + if futureErr := bc.addFutureBlock(block); futureErr != nil { + return batch.index, events, coalescedLogs, futureErr + } + stats.queued++ + block, err = batch.next() + } + } + stats.ignored += batch.remaining() // Append a single chain head event if we've progressed the chain if lastCanon != nil && bc.CurrentBlock().Hash() == lastCanon.Hash() { events = append(events, ChainHeadEvent{lastCanon}) } - return 0, events, coalescedLogs, nil + return 0, events, coalescedLogs, err +} + +// insertSidechainInternal should be called when an import batch hits upon a pruned ancestor error, which happens when +// an sidechain with a sufficiently old fork-block is found. It writes all (header-and-body-valid) blocks to disk, then +// tries to switch over to the new chain if the TD exceeded the current chain. +// It assumes that relevant locks are held already (hence 'Internal') +func (bc *BlockChain) insertSidechainInternal(batch *importBatch, err error) (int, []interface{}, []*types.Log, error) { + // If we're given a chain of blocks, and the first one is pruned, that means we're getting a + // sidechain imported. On the sidechain, we validate headers, but do not validate body and state + // (and actually import them) until the sidechain reaches a higher TD. + // Until then, we store them in the database (assuming that the header PoW check works out) + var ( + externTd *big.Int + canonHeadNumber = bc.CurrentBlock().NumberU64() + events = make([]interface{}, 0) + coalescedLogs []*types.Log + ) + // The first sidechain block error is already verified to be ErrPrunedAncestor. Since we don't import + // them here, we expect ErrUnknownAncestor for the remaining ones. Any other errors means that + // the block is invalid, and should not be written to disk. + block := batch.current() + for block != nil && (err == consensus.ErrPrunedAncestor) { + // Check the canonical stateroot for that number + if remoteNum := block.NumberU64(); canonHeadNumber >= remoteNum { + canonBlock := bc.GetBlockByNumber(remoteNum) + if canonBlock != nil && canonBlock.Root() == block.Root() { + // This is most likely a shadow-state attack. + // When a fork is imported into the database, and it eventually reaches a block height which is + // not pruned, we just found that the state already exist! This means that the sidechain block + // refers to a state which already exists in our canon chain. + // If left unchecked, we would now proceed importing the blocks, without actually having verified + // the state of the previous blocks. + log.Warn("Sidechain ghost-state attack detected", "blocknum", block.NumberU64(), + "sidechain root", block.Root(), "canon root", canonBlock.Root()) + // If someone legitimately side-mines blocks, they would still be imported as usual. However, + // we cannot risk writing unverified blocks to disk when they obviously target the pruning + // mechanism. + return batch.index, events, coalescedLogs, fmt.Errorf("sidechain ghost-state attack detected") + } + } + if externTd == nil { + externTd = bc.GetTd(block.ParentHash(), block.NumberU64()-1) + } + externTd = new(big.Int).Add(externTd, block.Difficulty()) + if !bc.HasBlock(block.Hash(), block.NumberU64()) { + if err := bc.WriteBlockWithoutState(block, externTd); err != nil { + return batch.index, events, coalescedLogs, err + } + } + block, err = batch.next() + } + // At this point, we've written all sidechain blocks to database. Loop ended either on some other error, + // or all were processed. If there was some other error, we can ignore the rest of those blocks. + // + // If the externTd was larger than our local TD, we now need to reimport the previous + // blocks to regenerate the required state + currentBlock := bc.CurrentBlock() + localTd := bc.GetTd(currentBlock.Hash(), currentBlock.NumberU64()) + // don't process until the competitor TD goes above the canonical TD + if localTd.Cmp(externTd) > 0 { + // If we have hit a sidechain, we may have to reimport pruned blocks + log.Info("Sidechain stored", "start", batch.first().NumberU64(), "end", batch.current().NumberU64(), "sidechain TD", externTd, "local TD", localTd) + return batch.index, events, coalescedLogs, err + } + // Competitor chain beat canonical. Before we reprocess to get the common ancestor, investigate if + // any blocks in the chain are 'known bad' blocks. + for index, b := range batch.chain { + if bc.badBlocks.Contains(b.Hash()) { + log.Info("Sidechain import aborted, bad block found", "index", index, "hash", b.Hash()) + return index, events, coalescedLogs, fmt.Errorf("known bad block %d 0x%x", b.NumberU64(), b.Hash()) + } + } + // gather all blocks from the common ancestor + var parents []*types.Block + // Import all the pruned blocks to make the state available + parent := bc.GetBlock(batch.first().ParentHash(), batch.first().NumberU64()-1) + for !bc.HasState(parent.Root()) { + if bc.badBlocks.Contains(parent.Hash()) { + log.Info("Sidechain parent processing aborted, bad block found", "number", parent.NumberU64(), "hash", parent.Hash()) + return 0, events, coalescedLogs, fmt.Errorf("known bad block %d 0x%x", parent.NumberU64(), parent.Hash()) + } + parents = append(parents, parent) + parent = bc.GetBlock(parent.ParentHash(), parent.NumberU64()-1) + } + for j := 0; j < len(parents)/2; j++ { + parents[j], parents[len(parents)-1-j] = parents[len(parents)-1-j], parents[j] + } + // Import all the pruned blocks to make the state available + // During re-import, we can disable PoW-verification, since these are already verified + log.Info("Inserting parent blocks for reprocessing", "first", parents[0].NumberU64(), "count", len(parents), "last", parents[len(parents)-1].NumberU64) + _, evs, logs, err := bc.insertChainInternal(parents, false) + events, coalescedLogs = evs, logs + if err != nil { + return 0, events, coalescedLogs, err + } + log.Info("Inserting sidechain blocks for processing") + errindex, events, coalescedLogs, err := bc.insertChainInternal(batch.chain[0:batch.index], false) + return errindex, events, coalescedLogs, err } // insertStats tracks and reports on block insertion. diff --git a/node/node.go b/node/node.go index 85299dba7..846100839 100644 --- a/node/node.go +++ b/node/node.go @@ -287,7 +287,7 @@ func (n *Node) startInProc(apis []rpc.API) error { if err := handler.RegisterName(api.Namespace, api.Service); err != nil { return err } - n.log.Debug("InProc registered", "service", api.Service, "namespace", api.Namespace) + n.log.Debug("InProc registered", "namespace", api.Namespace) } n.inprocHandler = handler return nil diff --git a/p2p/discover/table.go b/p2p/discover/table.go index afd4c9a27..9f7f1d41b 100644 --- a/p2p/discover/table.go +++ b/p2p/discover/table.go @@ -434,7 +434,7 @@ func (tab *Table) loadSeedNodes() { for i := range seeds { seed := seeds[i] age := log.Lazy{Fn: func() interface{} { return time.Since(tab.db.LastPongReceived(seed.ID())) }} - log.Debug("Found seed node in database", "id", seed.ID(), "addr", seed.addr(), "age", age) + log.Trace("Found seed node in database", "id", seed.ID(), "addr", seed.addr(), "age", age) tab.add(seed) } } -- 2.45.2 From 333b5fb123a497efa1a9ef54a437b24dfb3936cc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?P=C3=A9ter=20Szil=C3=A1gyi?= Date: Tue, 20 Nov 2018 14:15:26 +0200 Subject: [PATCH 02/48] core: polish side chain importer a bit --- core/blockchain.go | 479 +++++++++++++++----------------------- core/blockchain_insert.go | 143 ++++++++++++ core/blockchain_test.go | 8 +- 3 files changed, 332 insertions(+), 298 deletions(-) create mode 100644 core/blockchain_insert.go diff --git a/core/blockchain.go b/core/blockchain.go index 74ac30e70..d78946791 100644 --- a/core/blockchain.go +++ b/core/blockchain.go @@ -1036,6 +1036,18 @@ func (bc *BlockChain) WriteBlockWithState(block *types.Block, receipts []*types. return status, nil } +// addFutureBlock checks if the block is within the max allowed window to get +// accepted for future processing, and returns an error if the block is too far +// ahead and was not added. +func (bc *BlockChain) addFutureBlock(block *types.Block) error { + max := big.NewInt(time.Now().Unix() + maxTimeFutureBlocks) + if block.Time().Cmp(max) > 0 { + return fmt.Errorf("future block timestamp %v > allowed %v", block.Time(), max) + } + bc.futureBlocks.Add(block.Hash(), block) + return nil +} + // InsertChain attempts to insert the given batch of blocks in to the canonical // chain or, otherwise, create a fork. If an error is returned it will return // the index number of the failing block as well an error describing what went @@ -1043,92 +1055,9 @@ func (bc *BlockChain) WriteBlockWithState(block *types.Block, receipts []*types. // // After insertion is done, all accumulated events will be fired. func (bc *BlockChain) InsertChain(chain types.Blocks) (int, error) { - n, events, logs, err := bc.insertChain(chain) - bc.PostChainEvents(events, logs) - return n, err -} - -// addFutureBlock checks if the block is within the max allowed window to get accepted for future processing, and -// returns an error if the block is too far ahead and was not added. -func (bc *BlockChain) addFutureBlock(block *types.Block) error { - max := big.NewInt(time.Now().Unix() + maxTimeFutureBlocks) - if block.Time().Cmp(max) > 0 { - return fmt.Errorf("future block: %v > %v", block.Time(), max) - } - bc.futureBlocks.Add(block.Hash(), block) - return nil -} - -// importBatch is a helper function to assist during chain import -type importBatch struct { - chain types.Blocks - results <-chan error - index int - validator Validator -} - -// newBatch creates a new batch based on the given blocks, which are assumed to be a contiguous chain -func newBatch(chain types.Blocks, results <-chan error, validator Validator) *importBatch { - return &importBatch{ - chain: chain, - results: results, - index: -1, - validator: validator, - } -} - -// next returns the next block in the batch, along with any potential validation error for that block -// When the end is reached, it will return (nil, nil), but Current() will always return the last element. -func (batch *importBatch) next() (*types.Block, error) { - if batch.index+1 >= len(batch.chain) { - return nil, nil - } - batch.index++ - if err := <-batch.results; err != nil { - return batch.chain[batch.index], err - } - return batch.chain[batch.index], batch.validator.ValidateBody(batch.chain[batch.index]) -} - -// current returns the current block that's being processed. Even after the next() has progressed the entire -// chain, current will always return the last element -func (batch *importBatch) current() *types.Block { - if batch.index < 0 { - return nil - } - return batch.chain[batch.index] -} - -// previous returns the previous block was being processed, or nil -func (batch *importBatch) previous() *types.Block { - if batch.index < 1 { - return nil - } - return batch.chain[batch.index-1] -} - -// first returns the first block in the batch -func (batch *importBatch) first() *types.Block { - return batch.chain[0] -} - -// remaining returns the number of remaining blocks -func (batch *importBatch) remaining() int { - return len(batch.chain) - batch.index -} - -// processed returns the number of processed blocks -func (batch *importBatch) processed() int { - return batch.index + 1 -} - -// insertChain will execute the actual chain insertion and event aggregation. The -// only reason this method exists as a separate one is to make locking cleaner -// with deferred statements. -func (bc *BlockChain) insertChain(chain types.Blocks) (int, []interface{}, []*types.Log, error) { // Sanity check that we have something meaningful to import if len(chain) == 0 { - return 0, nil, nil, nil + return 0, nil } // Do a sanity check that the provided chain is actually ordered and linked for i := 1; i < len(chain); i++ { @@ -1137,29 +1066,34 @@ func (bc *BlockChain) insertChain(chain types.Blocks) (int, []interface{}, []*ty log.Error("Non contiguous block insert", "number", chain[i].Number(), "hash", chain[i].Hash(), "parent", chain[i].ParentHash(), "prevnumber", chain[i-1].Number(), "prevhash", chain[i-1].Hash()) - return 0, nil, nil, fmt.Errorf("non contiguous insert: item %d is #%d [%x…], item %d is #%d [%x…] (parent [%x…])", i-1, chain[i-1].NumberU64(), + return 0, fmt.Errorf("non contiguous insert: item %d is #%d [%x…], item %d is #%d [%x…] (parent [%x…])", i-1, chain[i-1].NumberU64(), chain[i-1].Hash().Bytes()[:4], i, chain[i].NumberU64(), chain[i].Hash().Bytes()[:4], chain[i].ParentHash().Bytes()[:4]) } } - log.Info("insertChain", "from", chain[0].NumberU64(), "to", chain[len(chain)-1].NumberU64()) - // Pre-checks passed, start the full block imports bc.wg.Add(1) - defer bc.wg.Done() bc.chainmu.Lock() - defer bc.chainmu.Unlock() - return bc.insertChainInternal(chain, true) + n, events, logs, err := bc.insertChain(chain, true) + bc.chainmu.Unlock() + bc.wg.Done() + + bc.PostChainEvents(events, logs) + return n, err } -//insertChainInternal is the internal implementation of insertChain, which assumes that -// 1. chains are contiguous, and -// 2. The `chainMu` lock is held -// This method is split out so that import batches that require re-injecting historical blocks can do -// so without releasing the lock, which could lead to racey behaviour. If a sidechain import is in progress, -// and the historic state is imported, but then new canon-head is added before the actual sidechain completes, -// then the historic state could be pruned again -func (bc *BlockChain) insertChainInternal(chain types.Blocks, verifySeals bool) (int, []interface{}, []*types.Log, error) { - +// insertChain is the internal implementation of insertChain, which assumes that +// 1) chains are contiguous, and 2) The chain mutex is held. +// +// This method is split out so that import batches that require re-injecting +// historical blocks can do so without releasing the lock, which could lead to +// racey behaviour. If a sidechain import is in progress, and the historic state +// is imported, but then new canon-head is added before the actual sidechain +// completes, then the historic state could be pruned again +func (bc *BlockChain) insertChain(chain types.Blocks, verifySeals bool) (int, []interface{}, []*types.Log, error) { + // If the chain is terminating, don't even bother starting u + if atomic.LoadInt32(&bc.procInterrupt) == 1 { + return 0, nil, nil, nil + } // Start a parallel signature recovery (signer will fluke on fork transition, minimal perf loss) senderCacher.recoverFromBlocks(types.MakeSigner(bc.chainConfig, chain[0].Number()), chain) @@ -1171,8 +1105,6 @@ func (bc *BlockChain) insertChainInternal(chain types.Blocks, verifySeals bool) events = make([]interface{}, 0, len(chain)) lastCanon *types.Block coalescedLogs []*types.Log - block *types.Block - err error ) // Start the parallel header verifier headers := make([]*types.Header, len(chain)) @@ -1185,52 +1117,51 @@ func (bc *BlockChain) insertChainInternal(chain types.Blocks, verifySeals bool) abort, results := bc.engine.VerifyHeaders(bc, headers, seals) defer close(abort) - // Peek the error for the first block - batch := newBatch(chain, results, bc.Validator()) - if block, err = batch.next(); err != nil { - if err == consensus.ErrPrunedAncestor { - return bc.insertSidechainInternal(batch, err) - } else if err == consensus.ErrFutureBlock || - (err == consensus.ErrUnknownAncestor && bc.futureBlocks.Contains(batch.first().ParentHash())) { + // Peek the error for the first block to decide the directing import logic + it := newInsertIterator(chain, results, bc.Validator()) - // The first block is a future block - // We can shove that one and any child blocks (that fail because of UnknownAncestor) into the future-queue - for block != nil && (batch.index == 0 || err == consensus.ErrUnknownAncestor) { - block := batch.current() - if futureError := bc.addFutureBlock(block); futureError != nil { - return batch.index, events, coalescedLogs, futureError - } - block, err = batch.next() + block, err := it.next() + switch { + // First block is pruned, insert as sidechain and reorg only if TD grows enough + case err == consensus.ErrPrunedAncestor: + return bc.insertSidechain(it) + + // First block is future, shove it (and all children) to the future queue (unknown ancestor) + case err == consensus.ErrFutureBlock || (err == consensus.ErrUnknownAncestor && bc.futureBlocks.Contains(it.first().ParentHash())): + for block != nil && (it.index == 0 || err == consensus.ErrUnknownAncestor) { + if err := bc.addFutureBlock(block); err != nil { + return it.index, events, coalescedLogs, err } - stats.queued += batch.processed() - stats.ignored += batch.remaining() - - // If there are any still remaining, mark as ignored - return batch.index, events, coalescedLogs, err - } else if err == ErrKnownBlock { - - // Block and state both already known -- there can be two explanations. - // 1. We did a roll-back, and should now do a re-import - // 2. The block is stored as a sidechain, and is lying about it's stateroot, and passes a stateroot - // from the canonical chain, which has not been verified. - - // Skip all known blocks that are blocks behind us - currentNum := bc.CurrentBlock().NumberU64() - for block != nil && err == ErrKnownBlock && currentNum >= block.NumberU64() { - // We ignore these - stats.ignored++ - block, err = batch.next() - } - // Falls through to the block import - } else { - // Some other error - stats.ignored += len(batch.chain) - bc.reportBlock(block, nil, err) - return batch.index, events, coalescedLogs, err + block, err = it.next() } + stats.queued += it.processed() + stats.ignored += it.remaining() + + // If there are any still remaining, mark as ignored + return it.index, events, coalescedLogs, err + + // First block (and state) is known + // 1. We did a roll-back, and should now do a re-import + // 2. The block is stored as a sidechain, and is lying about it's stateroot, and passes a stateroot + // from the canonical chain, which has not been verified. + case err == ErrKnownBlock: + // Skip all known blocks that behind us + current := bc.CurrentBlock().NumberU64() + + for block != nil && err == ErrKnownBlock && current >= block.NumberU64() { + stats.ignored++ + block, err = it.next() + } + // Falls through to the block import + + // Some other error occurred, abort + case err != nil: + stats.ignored += len(it.chain) + bc.reportBlock(block, nil, err) + return it.index, events, coalescedLogs, err } - // No validation errors - for ; block != nil && err == nil; block, err = batch.next() { + // No validation errors for the first block (or chain prefix skipped) + for ; block != nil && err == nil; block, err = it.next() { // If the chain is terminating, stop processing blocks if atomic.LoadInt32(&bc.procInterrupt) == 1 { log.Debug("Premature abort during blocks processing") @@ -1239,43 +1170,43 @@ func (bc *BlockChain) insertChainInternal(chain types.Blocks, verifySeals bool) // If the header is a banned one, straight out abort if BadHashes[block.Hash()] { bc.reportBlock(block, nil, ErrBlacklistedHash) - return batch.index, events, coalescedLogs, ErrBlacklistedHash + return it.index, events, coalescedLogs, ErrBlacklistedHash } - bstart := time.Now() - var parent *types.Block - parent = batch.previous() + // Retrieve the parent block and it's state to execute on top + start := time.Now() + + parent := it.previous() if parent == nil { parent = bc.GetBlock(block.ParentHash(), block.NumberU64()-1) } - state, err := state.New(parent.Root(), bc.stateCache) if err != nil { - return batch.index, events, coalescedLogs, err + return it.index, events, coalescedLogs, err } // Process block using the parent state as reference point. receipts, logs, usedGas, err := bc.processor.Process(block, state, bc.vmConfig) if err != nil { bc.reportBlock(block, receipts, err) - return batch.index, events, coalescedLogs, err + return it.index, events, coalescedLogs, err } // Validate the state using the default validator if err := bc.Validator().ValidateState(block, parent, state, receipts, usedGas); err != nil { bc.reportBlock(block, receipts, err) - return batch.index, events, coalescedLogs, err + return it.index, events, coalescedLogs, err } - proctime := time.Since(bstart) + proctime := time.Since(start) // Write the block to the chain and get the status. status, err := bc.WriteBlockWithState(block, receipts, state) if err != nil { - return batch.index, events, coalescedLogs, err + return it.index, events, coalescedLogs, 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)), - "root", block.Root().String()) + "elapsed", common.PrettyDuration(time.Since(start)), + "root", block.Root()) coalescedLogs = append(coalescedLogs, logs...) events = append(events, ChainEvent{block, block.Hash(), logs}) @@ -1286,192 +1217,152 @@ func (bc *BlockChain) insertChainInternal(chain types.Blocks, verifySeals bool) case SideStatTy: log.Debug("Inserted forked block", "number", block.Number(), "hash", block.Hash(), - "diff", block.Difficulty(), "elapsed", common.PrettyDuration(time.Since(bstart)), + "diff", block.Difficulty(), "elapsed", common.PrettyDuration(time.Since(start)), "txs", len(block.Transactions()), "gas", block.GasUsed(), "uncles", len(block.Uncles()), - "root", block.Root().String()) + "root", block.Root()) events = append(events, ChainSideEvent{block}) } - blockInsertTimer.UpdateSince(bstart) + blockInsertTimer.UpdateSince(start) stats.processed++ stats.usedGas += usedGas cache, _ := bc.stateCache.TrieDB().Size() - stats.report(chain, batch.index, cache) + stats.report(chain, it.index, cache) } - - // Any blocks remaining here? If so, the only ones we need to care about are - // shoving future blocks into queue + // Any blocks remaining here? The only ones we care about are the future ones if block != nil && err == consensus.ErrFutureBlock { - if futureErr := bc.addFutureBlock(block); futureErr != nil { - return batch.index, events, coalescedLogs, futureErr + if err := bc.addFutureBlock(block); err != nil { + return it.index, events, coalescedLogs, err } - for block, err = batch.next(); block != nil && err == consensus.ErrUnknownAncestor; { - if futureErr := bc.addFutureBlock(block); futureErr != nil { - return batch.index, events, coalescedLogs, futureErr + block, err = it.next() + + for ; block != nil && err == consensus.ErrUnknownAncestor; block, err = it.next() { + if err := bc.addFutureBlock(block); err != nil { + return it.index, events, coalescedLogs, err } stats.queued++ - block, err = batch.next() } } - stats.ignored += batch.remaining() + stats.ignored += it.remaining() + // Append a single chain head event if we've progressed the chain if lastCanon != nil && bc.CurrentBlock().Hash() == lastCanon.Hash() { events = append(events, ChainHeadEvent{lastCanon}) } - return 0, events, coalescedLogs, err + return it.index, events, coalescedLogs, err } -// insertSidechainInternal should be called when an import batch hits upon a pruned ancestor error, which happens when -// an sidechain with a sufficiently old fork-block is found. It writes all (header-and-body-valid) blocks to disk, then -// tries to switch over to the new chain if the TD exceeded the current chain. -// It assumes that relevant locks are held already (hence 'Internal') -func (bc *BlockChain) insertSidechainInternal(batch *importBatch, err error) (int, []interface{}, []*types.Log, error) { - // If we're given a chain of blocks, and the first one is pruned, that means we're getting a - // sidechain imported. On the sidechain, we validate headers, but do not validate body and state - // (and actually import them) until the sidechain reaches a higher TD. - // Until then, we store them in the database (assuming that the header PoW check works out) +// insertSidechain is called when an import batch hits upon a pruned ancestor +// error, which happens when a sidechain with a sufficiently old fork-block is +// found. +// +// The method writes all (header-and-body-valid) blocks to disk, then tries to +// switch over to the new chain if the TD exceeded the current chain. +func (bc *BlockChain) insertSidechain(it *insertIterator) (int, []interface{}, []*types.Log, error) { var ( - externTd *big.Int - canonHeadNumber = bc.CurrentBlock().NumberU64() - events = make([]interface{}, 0) - coalescedLogs []*types.Log + externTd *big.Int + current = bc.CurrentBlock().NumberU64() ) - // The first sidechain block error is already verified to be ErrPrunedAncestor. Since we don't import - // them here, we expect ErrUnknownAncestor for the remaining ones. Any other errors means that - // the block is invalid, and should not be written to disk. - block := batch.current() - for block != nil && (err == consensus.ErrPrunedAncestor) { - // Check the canonical stateroot for that number - if remoteNum := block.NumberU64(); canonHeadNumber >= remoteNum { - canonBlock := bc.GetBlockByNumber(remoteNum) - if canonBlock != nil && canonBlock.Root() == block.Root() { - // This is most likely a shadow-state attack. - // When a fork is imported into the database, and it eventually reaches a block height which is - // not pruned, we just found that the state already exist! This means that the sidechain block + // The first sidechain block error is already verified to be ErrPrunedAncestor. + // Since we don't import them here, we expect ErrUnknownAncestor for the remaining + // ones. Any other errors means that the block is invalid, and should not be written + // to disk. + block, err := it.current(), consensus.ErrPrunedAncestor + for ; block != nil && (err == consensus.ErrPrunedAncestor); block, err = it.next() { + // Check the canonical state root for that number + if number := block.NumberU64(); current >= number { + if canonical := bc.GetBlockByNumber(number); canonical != nil && canonical.Root() == block.Root() { + // This is most likely a shadow-state attack. When a fork is imported into the + // database, and it eventually reaches a block height which is not pruned, we + // just found that the state already exist! This means that the sidechain block // refers to a state which already exists in our canon chain. - // If left unchecked, we would now proceed importing the blocks, without actually having verified - // the state of the previous blocks. - log.Warn("Sidechain ghost-state attack detected", "blocknum", block.NumberU64(), - "sidechain root", block.Root(), "canon root", canonBlock.Root()) + // + // If left unchecked, we would now proceed importing the blocks, without actually + // having verified the state of the previous blocks. + log.Warn("Sidechain ghost-state attack detected", "number", block.NumberU64(), "sideroot", block.Root(), "canonroot", canonical.Root()) + // If someone legitimately side-mines blocks, they would still be imported as usual. However, // we cannot risk writing unverified blocks to disk when they obviously target the pruning // mechanism. - return batch.index, events, coalescedLogs, fmt.Errorf("sidechain ghost-state attack detected") + return it.index, nil, nil, errors.New("sidechain ghost-state attack") } } if externTd == nil { externTd = bc.GetTd(block.ParentHash(), block.NumberU64()-1) } externTd = new(big.Int).Add(externTd, block.Difficulty()) + if !bc.HasBlock(block.Hash(), block.NumberU64()) { + start := time.Now() if err := bc.WriteBlockWithoutState(block, externTd); err != nil { - return batch.index, events, coalescedLogs, err + return it.index, nil, nil, err } + log.Debug("Inserted sidechain block", "number", block.Number(), "hash", block.Hash(), + "diff", block.Difficulty(), "elapsed", common.PrettyDuration(time.Since(start)), + "txs", len(block.Transactions()), "gas", block.GasUsed(), "uncles", len(block.Uncles()), + "root", block.Root()) } - block, err = batch.next() } - // At this point, we've written all sidechain blocks to database. Loop ended either on some other error, - // or all were processed. If there was some other error, we can ignore the rest of those blocks. + // At this point, we've written all sidechain blocks to database. Loop ended + // either on some other error or all were processed. If there was some other + // error, we can ignore the rest of those blocks. // // If the externTd was larger than our local TD, we now need to reimport the previous // blocks to regenerate the required state - currentBlock := bc.CurrentBlock() - localTd := bc.GetTd(currentBlock.Hash(), currentBlock.NumberU64()) - // don't process until the competitor TD goes above the canonical TD + localTd := bc.GetTd(bc.CurrentBlock().Hash(), current) if localTd.Cmp(externTd) > 0 { - // If we have hit a sidechain, we may have to reimport pruned blocks - log.Info("Sidechain stored", "start", batch.first().NumberU64(), "end", batch.current().NumberU64(), "sidechain TD", externTd, "local TD", localTd) - return batch.index, events, coalescedLogs, err + log.Info("Sidechain written to disk", "start", it.first().NumberU64(), "end", it.previous().NumberU64(), "sidetd", externTd, "localtd", localTd) + return it.index, nil, nil, err } - // Competitor chain beat canonical. Before we reprocess to get the common ancestor, investigate if - // any blocks in the chain are 'known bad' blocks. - for index, b := range batch.chain { - if bc.badBlocks.Contains(b.Hash()) { - log.Info("Sidechain import aborted, bad block found", "index", index, "hash", b.Hash()) - return index, events, coalescedLogs, fmt.Errorf("known bad block %d 0x%x", b.NumberU64(), b.Hash()) - } - } - // gather all blocks from the common ancestor - var parents []*types.Block - // Import all the pruned blocks to make the state available - parent := bc.GetBlock(batch.first().ParentHash(), batch.first().NumberU64()-1) - for !bc.HasState(parent.Root()) { - if bc.badBlocks.Contains(parent.Hash()) { - log.Info("Sidechain parent processing aborted, bad block found", "number", parent.NumberU64(), "hash", parent.Hash()) - return 0, events, coalescedLogs, fmt.Errorf("known bad block %d 0x%x", parent.NumberU64(), parent.Hash()) - } - parents = append(parents, parent) - parent = bc.GetBlock(parent.ParentHash(), parent.NumberU64()-1) - } - for j := 0; j < len(parents)/2; j++ { - parents[j], parents[len(parents)-1-j] = parents[len(parents)-1-j], parents[j] - } - // Import all the pruned blocks to make the state available - // During re-import, we can disable PoW-verification, since these are already verified - log.Info("Inserting parent blocks for reprocessing", "first", parents[0].NumberU64(), "count", len(parents), "last", parents[len(parents)-1].NumberU64) - _, evs, logs, err := bc.insertChainInternal(parents, false) - events, coalescedLogs = evs, logs - if err != nil { - return 0, events, coalescedLogs, err - } - log.Info("Inserting sidechain blocks for processing") - errindex, events, coalescedLogs, err := bc.insertChainInternal(batch.chain[0:batch.index], false) - return errindex, events, coalescedLogs, err -} - -// insertStats tracks and reports on block insertion. -type insertStats struct { - queued, processed, ignored int - usedGas uint64 - lastIndex int - startTime mclock.AbsTime -} - -// statsReportLimit is the time limit during import and export after which we -// always print out progress. This avoids the user wondering what's going on. -const statsReportLimit = 8 * time.Second - -// report prints statistics if some number of blocks have been processed -// or more than a few seconds have passed since the last message. -func (st *insertStats) report(chain []*types.Block, index int, cache common.StorageSize) { - // Fetch the timings for the batch + // Gather all the sidechain hashes (full blocks may be memory heavy) var ( - now = mclock.Now() - elapsed = time.Duration(now) - time.Duration(st.startTime) + hashes []common.Hash + numbers []uint64 ) - // If we're at the last block of the batch or report period reached, log - if index == len(chain)-1 || elapsed >= statsReportLimit { - var ( - end = chain[index] - txs = countTransactions(chain[st.lastIndex : index+1]) - ) - context := []interface{}{ - "blocks", st.processed, "txs", txs, "mgas", float64(st.usedGas) / 1000000, - "elapsed", common.PrettyDuration(elapsed), "mgasps", float64(st.usedGas) * 1000 / float64(elapsed), - "number", end.Number(), "hash", end.Hash(), - } - if timestamp := time.Unix(end.Time().Int64(), 0); time.Since(timestamp) > time.Minute { - context = append(context, []interface{}{"age", common.PrettyAge(timestamp)}...) - } - context = append(context, []interface{}{"cache", cache}...) + parent := bc.GetHeader(it.previous().Hash(), it.previous().NumberU64()) + for parent != nil && !bc.HasState(parent.Root) { + hashes = append(hashes, parent.Hash()) + numbers = append(numbers, parent.Number.Uint64()) - if st.queued > 0 { - context = append(context, []interface{}{"queued", st.queued}...) - } - if st.ignored > 0 { - context = append(context, []interface{}{"ignored", st.ignored}...) - } - log.Info("Imported new chain segment", context...) - - *st = insertStats{startTime: now, lastIndex: index + 1} + parent = bc.GetHeader(parent.ParentHash, parent.Number.Uint64()-1) } -} - -func countTransactions(chain []*types.Block) (c int) { - for _, b := range chain { - c += len(b.Transactions()) + if parent == nil { + return it.index, nil, nil, errors.New("missing parent") } - return c + // Import all the pruned blocks to make the state available + var ( + blocks []*types.Block + memory common.StorageSize + ) + for i := len(hashes) - 1; i >= 0; i-- { + // Append the next block to our batch + block := bc.GetBlock(hashes[i], numbers[i]) + + blocks = append(blocks, block) + memory += block.Size() + + // If memory use grew too large, import and continue. Sadly we need to discard + // all raised events and logs from notifications since we're too heavy on the + // memory here. + if len(blocks) >= 2048 || memory > 64*1024*1024 { + log.Info("Importing heavy sidechain segment", "blocks", len(blocks), "start", blocks[0].NumberU64(), "end", block.NumberU64()) + if _, _, _, err := bc.insertChain(blocks, false); err != nil { + return 0, nil, nil, err + } + blocks, memory = blocks[:0], 0 + + // If the chain is terminating, stop processing blocks + if atomic.LoadInt32(&bc.procInterrupt) == 1 { + log.Debug("Premature abort during blocks processing") + return 0, nil, nil, nil + } + } + } + if len(blocks) > 0 { + log.Info("Importing sidechain segment", "start", blocks[0].NumberU64(), "end", blocks[len(blocks)-1].NumberU64()) + return bc.insertChain(blocks, false) + } + return 0, nil, nil, nil } // reorgs takes two blocks, an old chain and a new chain and will reconstruct the blocks and inserts them diff --git a/core/blockchain_insert.go b/core/blockchain_insert.go new file mode 100644 index 000000000..70bea3544 --- /dev/null +++ b/core/blockchain_insert.go @@ -0,0 +1,143 @@ +// Copyright 2018 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +package core + +import ( + "time" + + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/common/mclock" + "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/log" +) + +// insertStats tracks and reports on block insertion. +type insertStats struct { + queued, processed, ignored int + usedGas uint64 + lastIndex int + startTime mclock.AbsTime +} + +// statsReportLimit is the time limit during import and export after which we +// always print out progress. This avoids the user wondering what's going on. +const statsReportLimit = 8 * time.Second + +// report prints statistics if some number of blocks have been processed +// or more than a few seconds have passed since the last message. +func (st *insertStats) report(chain []*types.Block, index int, cache common.StorageSize) { + // Fetch the timings for the batch + var ( + now = mclock.Now() + elapsed = time.Duration(now) - time.Duration(st.startTime) + ) + // If we're at the last block of the batch or report period reached, log + if index == len(chain)-1 || elapsed >= statsReportLimit { + // Count the number of transactions in this segment + var txs int + for _, block := range chain[st.lastIndex : index+1] { + txs += len(block.Transactions()) + } + end := chain[index] + + // Assemble the log context and send it to the logger + context := []interface{}{ + "blocks", st.processed, "txs", txs, "mgas", float64(st.usedGas) / 1000000, + "elapsed", common.PrettyDuration(elapsed), "mgasps", float64(st.usedGas) * 1000 / float64(elapsed), + "number", end.Number(), "hash", end.Hash(), + } + if timestamp := time.Unix(end.Time().Int64(), 0); time.Since(timestamp) > time.Minute { + context = append(context, []interface{}{"age", common.PrettyAge(timestamp)}...) + } + context = append(context, []interface{}{"cache", cache}...) + + if st.queued > 0 { + context = append(context, []interface{}{"queued", st.queued}...) + } + if st.ignored > 0 { + context = append(context, []interface{}{"ignored", st.ignored}...) + } + log.Info("Imported new chain segment", context...) + + // Bump the stats reported to the next section + *st = insertStats{startTime: now, lastIndex: index + 1} + } +} + +// insertIterator is a helper to assist during chain import. +type insertIterator struct { + chain types.Blocks + results <-chan error + index int + validator Validator +} + +// newInsertIterator creates a new iterator based on the given blocks, which are +// assumed to be a contiguous chain. +func newInsertIterator(chain types.Blocks, results <-chan error, validator Validator) *insertIterator { + return &insertIterator{ + chain: chain, + results: results, + index: -1, + validator: validator, + } +} + +// next returns the next block in the iterator, along with any potential validation +// error for that block. When the end is reached, it will return (nil, nil). +func (it *insertIterator) next() (*types.Block, error) { + if it.index+1 >= len(it.chain) { + it.index = len(it.chain) + return nil, nil + } + it.index++ + if err := <-it.results; err != nil { + return it.chain[it.index], err + } + return it.chain[it.index], it.validator.ValidateBody(it.chain[it.index]) +} + +// current returns the current block that's being processed. +func (it *insertIterator) current() *types.Block { + if it.index < 0 || it.index+1 >= len(it.chain) { + return nil + } + return it.chain[it.index] +} + +// previous returns the previous block was being processed, or nil +func (it *insertIterator) previous() *types.Block { + if it.index < 1 { + return nil + } + return it.chain[it.index-1] +} + +// first returns the first block in the it. +func (it *insertIterator) first() *types.Block { + return it.chain[0] +} + +// remaining returns the number of remaining blocks. +func (it *insertIterator) remaining() int { + return len(it.chain) - it.index +} + +// processed returns the number of processed blocks. +func (it *insertIterator) processed() int { + return it.index + 1 +} diff --git a/core/blockchain_test.go b/core/blockchain_test.go index aef810050..5ab29e205 100644 --- a/core/blockchain_test.go +++ b/core/blockchain_test.go @@ -579,11 +579,11 @@ func testInsertNonceError(t *testing.T, full bool) { blockchain.hc.engine = blockchain.engine failRes, err = blockchain.InsertHeaderChain(headers, 1) } - // Check that the returned error indicates the failure. + // Check that the returned error indicates the failure if failRes != failAt { - t.Errorf("test %d: failure index mismatch: have %d, want %d", i, failRes, failAt) + t.Errorf("test %d: failure (%v) index mismatch: have %d, want %d", i, err, failRes, failAt) } - // Check that all no blocks after the failing block have been inserted. + // Check that all blocks after the failing block have been inserted for j := 0; j < i-failAt; j++ { if full { if block := blockchain.GetBlockByNumber(failNum + uint64(j)); block != nil { @@ -1345,7 +1345,7 @@ func TestLargeReorgTrieGC(t *testing.T) { t.Fatalf("failed to insert shared chain: %v", err) } if _, err := chain.InsertChain(original); err != nil { - t.Fatalf("failed to insert shared chain: %v", err) + t.Fatalf("failed to insert original chain: %v", err) } // Ensure that the state associated with the forking point is pruned away if node, _ := chain.stateCache.TrieDB().Node(shared[len(shared)-1].Root()); node != nil { -- 2.45.2 From 53a6b1c6892b13cbda56c6b8fad001df47e2ebb0 Mon Sep 17 00:00:00 2001 From: Ian Norden Date: Wed, 21 Nov 2018 18:44:57 -0600 Subject: [PATCH 03/48] port statediff from https://github.com/jpmorganchase/quorum/blob/9b7fd9af8082795eeeb6863d9746f12b82dd5078/statediff/statediff.go; minor fixes --- statediff/helpers.go | 114 +++++++++++++ statediff/statediff.go | 67 ++++++++ statediff/statediff_builder.go | 301 +++++++++++++++++++++++++++++++++ 3 files changed, 482 insertions(+) create mode 100644 statediff/helpers.go create mode 100644 statediff/statediff.go create mode 100644 statediff/statediff_builder.go diff --git a/statediff/helpers.go b/statediff/helpers.go new file mode 100644 index 000000000..976d21fe2 --- /dev/null +++ b/statediff/helpers.go @@ -0,0 +1,114 @@ +package statediff + +import ( + "sort" + "strings" + + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/core/state" + "github.com/ethereum/go-ethereum/trie" +) + +func sortKeys(data map[common.Address]*state.Account) []string { + var keys []string + for key := range data { + keys = append(keys, key.Hex()) + } + sort.Strings(keys) + + return keys +} + +func findIntersection(a, b []string) []string { + lenA := len(a) + lenB := len(b) + iOfA, iOfB := 0, 0 + updates := make([]string, 0) + if iOfA >= lenA || iOfB >= lenB { + return updates + } + for { + switch strings.Compare(a[iOfA], b[iOfB]) { + // a[iOfA] < b[iOfB] + case -1: + iOfA++ + if iOfA >= lenA { + return updates + } + break + // a[iOfA] == b[iOfB] + case 0: + updates = append(updates, a[iOfA]) + iOfA++ + iOfB++ + if iOfA >= lenA || iOfB >= lenB { + return updates + } + break + // a[iOfA] > b[iOfB] + case 1: + iOfB++ + if iOfB >= lenB { + return updates + } + break + } + } + +} + +func pathToStr(it trie.NodeIterator) string { + path := it.Path() + if hasTerm(path) { + path = path[:len(path)-1] + } + nibblePath := "" + for i, v := range common.ToHex(path) { + if i%2 == 0 && i > 1 { + continue + } + nibblePath = nibblePath + string(v) + } + + return nibblePath +} + +// Duplicated from trie/encoding.go +func hexToKeybytes(hex []byte) []byte { + if hasTerm(hex) { + hex = hex[:len(hex)-1] + } + if len(hex)&1 != 0 { + panic("can't convert hex key of odd length") + } + key := make([]byte, (len(hex)+1)/2) + decodeNibbles(hex, key) + + return key +} + +func decodeNibbles(nibbles []byte, bytes []byte) { + for bi, ni := 0, 0; ni < len(nibbles); bi, ni = bi+1, ni+2 { + bytes[bi] = nibbles[ni]<<4 | nibbles[ni+1] + } +} + +// prefixLen returns the length of the common prefix of a and b. +func prefixLen(a, b []byte) int { + var i, length = 0, len(a) + if len(b) < length { + length = len(b) + } + for ; i < length; i++ { + if a[i] != b[i] { + break + } + } + + return i +} + +// hasTerm returns whether a hex key has the terminator flag. +func hasTerm(s []byte) bool { + return len(s) > 0 && s[len(s)-1] == 16 +} diff --git a/statediff/statediff.go b/statediff/statediff.go new file mode 100644 index 000000000..d980ef867 --- /dev/null +++ b/statediff/statediff.go @@ -0,0 +1,67 @@ +package statediff + +import ( + "encoding/json" + "math/big" + + "github.com/ethereum/go-ethereum/common" +) + +type StateDiff struct { + BlockNumber int64 `json:"blockNumber" gencodec:"required"` + BlockHash common.Hash `json:"blockHash" gencodec:"required"` + CreatedAccounts map[common.Address]AccountDiffEventual `json:"createdAccounts" gencodec:"required"` + DeletedAccounts map[common.Address]AccountDiffEventual `json:"deletedAccounts" gencodec:"required"` + UpdatedAccounts map[common.Address]AccountDiffIncremental `json:"updatedAccounts" gencodec:"required"` + + encoded []byte + err error +} + +func (self *StateDiff) ensureEncoded() { + if self.encoded == nil && self.err == nil { + self.encoded, self.err = json.Marshal(self) + } +} + +// Implement Encoder interface for StateDiff +func (sd *StateDiff) Length() int { + sd.ensureEncoded() + return len(sd.encoded) +} + +// Implement Encoder interface for StateDiff +func (sd *StateDiff) Encode() ([]byte, error) { + sd.ensureEncoded() + return sd.encoded, sd.err +} + +type AccountDiffEventual struct { + Nonce diffUint64 `json:"nonce" gencodec:"required"` + Balance diffBigInt `json:"balance" gencodec:"required"` + Code string `json:"code" gencodec:"required"` + CodeHash string `json:"codeHash" gencodec:"required"` + ContractRoot diffString `json:"contractRoot" gencodec:"required"` + Storage map[string]diffString `json:"storage" gencodec:"required"` +} + +type AccountDiffIncremental struct { + Nonce diffUint64 `json:"nonce" gencodec:"required"` + Balance diffBigInt `json:"balance" gencodec:"required"` + CodeHash string `json:"codeHash" gencodec:"required"` + ContractRoot diffString `json:"contractRoot" gencodec:"required"` + Storage map[string]diffString `json:"storage" gencodec:"required"` +} + +type diffString struct { + NewValue *string `json:"newValue" gencodec:"optional"` + OldValue *string `json:"oldValue" gencodec:"optional"` +} +type diffUint64 struct { + NewValue *uint64 `json:"newValue" gencodec:"optional"` + OldValue *uint64 `json:"oldValue" gencodec:"optional"` +} +type diffBigInt struct { + NewValue *big.Int `json:"newValue" gencodec:"optional"` + OldValue *big.Int `json:"oldValue" gencodec:"optional"` +} diff --git a/statediff/statediff_builder.go b/statediff/statediff_builder.go new file mode 100644 index 000000000..c3011ed66 --- /dev/null +++ b/statediff/statediff_builder.go @@ -0,0 +1,301 @@ +package statediff + +import ( + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/core/state" + "github.com/ethereum/go-ethereum/ethdb" + "github.com/ethereum/go-ethereum/log" + "github.com/ethereum/go-ethereum/rlp" + "github.com/ethereum/go-ethereum/trie" +) + +type StateDiffBuilder interface { + CreateStateDiff(oldStateRoot, newStateRoot common.Hash, blockNumber int64, blockHash common.Hash) (*StateDiff, error) +} + +type stateDiffBuilder struct { + chainDB ethdb.Database + trieDB *trie.Database + cachedTrie *trie.Trie +} + +func NewStateDiffBuilder(db ethdb.Database) *stateDiffBuilder { + return &stateDiffBuilder{ + chainDB: db, + trieDB: trie.NewDatabase(db), + } +} + +func (sdb *stateDiffBuilder) CreateStateDiff(oldStateRoot, newStateRoot common.Hash, blockNumber int64, blockHash common.Hash) (*StateDiff, error) { + // Generate tries for old and new states + oldTrie, err := trie.New(oldStateRoot, sdb.trieDB) + if err != nil { + return nil, err + } + newTrie, err := trie.New(newStateRoot, sdb.trieDB) + if err != nil { + return nil, err + } + + // Find created accounts + oldIt := oldTrie.NodeIterator([]byte{}) + newIt := newTrie.NodeIterator([]byte{}) + creations, err := sdb.collectDiffNodes(oldIt, newIt) + if err != nil { + return nil, err + } + + // Find deleted accounts + oldIt = oldTrie.NodeIterator(make([]byte, 0)) + newIt = newTrie.NodeIterator(make([]byte, 0)) + deletions, err := sdb.collectDiffNodes(newIt, oldIt) + if err != nil { + return nil, err + } + + // Find all the diffed keys + createKeys := sortKeys(creations) + deleteKeys := sortKeys(deletions) + updatedKeys := findIntersection(createKeys, deleteKeys) + + // Build and return the statediff + updatedAccounts, err := sdb.buildDiffIncremental(creations, deletions, &updatedKeys) + if err != nil { + return nil, err + } + createdAccounts, err := sdb.buildDiffEventual(creations, true) + if err != nil { + return nil, err + } + deletedAccounts, err := sdb.buildDiffEventual(deletions, false) + if err != nil { + return nil, err + } + + return &StateDiff{ + BlockNumber: blockNumber, + BlockHash: blockHash, + CreatedAccounts: createdAccounts, + DeletedAccounts: deletedAccounts, + UpdatedAccounts: updatedAccounts, + }, nil +} + +func (sdb *stateDiffBuilder) collectDiffNodes(a, b trie.NodeIterator) (map[common.Address]*state.Account, error) { + var diffAccounts map[common.Address]*state.Account + it, _ := trie.NewDifferenceIterator(a, b) + + for { + log.Debug("Current Path and Hash", "path", pathToStr(it), "hashold", common.Hash(it.Hash())) + if it.Leaf() { + + // lookup address + path := make([]byte, len(it.Path())-1) + copy(path, it.Path()) + addr, err := sdb.addressByPath(path) + if err != nil { + log.Error("Error looking up address via path", "path", path, "error", err) + return nil, err + } + + // lookup account state + var account state.Account + if err := rlp.DecodeBytes(it.LeafBlob(), &account); err != nil { + log.Error("Error looking up account via address", "address", addr, "error", err) + return nil, err + } + + // record account to diffs (creation if we are looking at new - old; deletion if old - new) + log.Debug("Account lookup successful", "address", addr, "account", account) + diffAccounts[*addr] = &account + } + cont := it.Next(true) + if !cont { + break + } + } + return diffAccounts, nil +} + +func (sdb *stateDiffBuilder) buildDiffEventual(accounts map[common.Address]*state.Account, created bool) (map[common.Address]AccountDiffEventual, error) { + accountDiffs := make(map[common.Address]AccountDiffEventual) + for addr, val := range accounts { + sr := val.Root + if storageDiffs, err := sdb.buildStorageDiffsEventual(sr, created); err != nil { + log.Error("Failed building eventual storage diffs", "Address", val, "error", err) + return nil, err + } else { + code := "" + codeBytes, err := sdb.chainDB.Get(val.CodeHash) + if err == nil && len(codeBytes) != 0 { + code = common.ToHex(codeBytes) + } else { + log.Debug("No code field.", "codehash", val.CodeHash, "Address", val, "error", err) + } + codeHash := common.ToHex(val.CodeHash) + if created { + nonce := diffUint64{ + NewValue: &val.Nonce, + } + + balance := diffBigInt{ + NewValue: val.Balance, + } + + hexRoot := val.Root.Hex() + contractRoot := diffString{ + NewValue: &hexRoot, + } + accountDiffs[addr] = AccountDiffEventual{ + Nonce: nonce, + Balance: balance, + CodeHash: codeHash, + Code: code, + ContractRoot: contractRoot, + Storage: storageDiffs, + } + } else { + nonce := diffUint64{ + OldValue: &val.Nonce, + } + balance := diffBigInt{ + OldValue: val.Balance, + } + hexRoot := val.Root.Hex() + contractRoot := diffString{ + OldValue: &hexRoot, + } + accountDiffs[addr] = AccountDiffEventual{ + Nonce: nonce, + Balance: balance, + CodeHash: codeHash, + ContractRoot: contractRoot, + Storage: storageDiffs, + } + } + } + } + return accountDiffs, nil +} + +func (sdb *stateDiffBuilder) buildDiffIncremental(creations map[common.Address]*state.Account, deletions map[common.Address]*state.Account, updatedKeys *[]string) (map[common.Address]AccountDiffIncremental, error) { + updatedAccounts := make(map[common.Address]AccountDiffIncremental) + for _, val := range *updatedKeys { + createdAcc := creations[common.HexToAddress(val)] + deletedAcc := deletions[common.HexToAddress(val)] + oldSR := deletedAcc.Root + newSR := createdAcc.Root + if storageDiffs, err := sdb.buildStorageDiffsIncremental(oldSR, newSR); err != nil { + log.Error("Failed building storage diffs", "Address", val, "error", err) + return nil, err + } else { + nonce := diffUint64{ + NewValue: &createdAcc.Nonce, + OldValue: &deletedAcc.Nonce, + } + + balance := diffBigInt{ + NewValue: createdAcc.Balance, + OldValue: deletedAcc.Balance, + } + codeHash := common.ToHex(createdAcc.CodeHash) + + nHexRoot := createdAcc.Root.Hex() + oHexRoot := deletedAcc.Root.Hex() + contractRoot := diffString{ + NewValue: &nHexRoot, + OldValue: &oHexRoot, + } + + updatedAccounts[common.HexToAddress(val)] = AccountDiffIncremental{ + Nonce: nonce, + Balance: balance, + CodeHash: codeHash, + ContractRoot: contractRoot, + Storage: storageDiffs, + } + delete(creations, common.HexToAddress(val)) + delete(deletions, common.HexToAddress(val)) + } + } + return updatedAccounts, nil +} + +func (sdb *stateDiffBuilder) buildStorageDiffsEventual(sr common.Hash, creation bool) (map[string]diffString, error) { + log.Debug("Storage Root For Eventual Diff", "root", sr.Hex()) + sTrie, err := trie.New(sr, sdb.trieDB) + if err != nil { + return nil, err + } + it := sTrie.NodeIterator(make([]byte, 0)) + storageDiffs := make(map[string]diffString) + for { + log.Debug("Iterating over state at path ", "path", pathToStr(it)) + if it.Leaf() { + log.Debug("Found leaf in storage", "path", pathToStr(it)) + path := pathToStr(it) + value := common.ToHex(it.LeafBlob()) + if creation { + storageDiffs[path] = diffString{NewValue: &value} + } else { + storageDiffs[path] = diffString{OldValue: &value} + } + } + cont := it.Next(true) + if !cont { + break + } + } + return storageDiffs, nil +} + +func (sdb *stateDiffBuilder) buildStorageDiffsIncremental(oldSR common.Hash, newSR common.Hash) (map[string]diffString, error) { + log.Debug("Storage Roots for Incremental Diff", "old", oldSR.Hex(), "new", newSR.Hex()) + oldTrie, err := trie.New(oldSR, sdb.trieDB) + if err != nil { + return nil, err + } + newTrie, err := trie.New(newSR, sdb.trieDB) + if err != nil { + return nil, err + } + + oldIt := oldTrie.NodeIterator(make([]byte, 0)) + newIt := newTrie.NodeIterator(make([]byte, 0)) + it, _ := trie.NewDifferenceIterator(oldIt, newIt) + storageDiffs := make(map[string]diffString) + for { + if it.Leaf() { + log.Debug("Found leaf in storage", "path", pathToStr(it)) + path := pathToStr(it) + value := common.ToHex(it.LeafBlob()) + if oldVal, err := oldTrie.TryGet(it.LeafKey()); err != nil { + log.Error("Failed to look up value in oldTrie", "path", path, "error", err) + } else { + hexOldVal := common.ToHex(oldVal) + storageDiffs[path] = diffString{OldValue: &hexOldVal, NewValue: &value} + } + } + + cont := it.Next(true) + if !cont { + break + } + } + return storageDiffs, nil +} + +func (sdb *stateDiffBuilder) addressByPath(path []byte) (*common.Address, error) { + // db := core.PreimageTable(sdb.chainDb) + log.Debug("Looking up address from path", "path", common.ToHex(append([]byte("secure-key-"), path...))) + // if addrBytes,err := db.Get(path); err != nil { + if addrBytes, err := sdb.chainDB.Get(append([]byte("secure-key-"), hexToKeybytes(path)...)); err != nil { + log.Error("Error looking up address via path", "path", common.ToHex(append([]byte("secure-key-"), path...)), "error", err) + return nil, err + } else { + addr := common.BytesToAddress(addrBytes) + log.Debug("Address found", "Address", addr) + return &addr, nil + } + +} -- 2.45.2 From e0d091e0909fc94958a33608ae4dad0825482813 Mon Sep 17 00:00:00 2001 From: Martin Holst Swende Date: Thu, 22 Nov 2018 10:00:16 +0100 Subject: [PATCH 04/48] core: better printout of receipts in bad block reports (#18156) * core/blockchain: better printout of receipts in bad block reports * fix splleing --- core/blockchain.go | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/core/blockchain.go b/core/blockchain.go index d173b2de2..4de7f007b 100644 --- a/core/blockchain.go +++ b/core/blockchain.go @@ -1469,8 +1469,10 @@ func (bc *BlockChain) reportBlock(block *types.Block, receipts types.Receipts, e bc.addBadBlock(block) var receiptString string - for _, receipt := range receipts { - receiptString += fmt.Sprintf("\t%v\n", receipt) + for i, receipt := range receipts { + receiptString += fmt.Sprintf("\t %d: cumulative: %v gas: %v contract: %v status: %v tx: %v logs: %v bloom: %x state: %x\n", + i, receipt.CumulativeGasUsed, receipt.GasUsed, receipt.ContractAddress.Hex(), + receipt.Status, receipt.TxHash.Hex(), receipt.Logs, receipt.Bloom, receipt.PostState) } log.Error(fmt.Sprintf(` ########## BAD BLOCK ######### -- 2.45.2 From 9d5e3e06370dace249305d91624cb83cabf3c25f Mon Sep 17 00:00:00 2001 From: Enrique Fynn Date: Thu, 22 Nov 2018 14:03:50 +0100 Subject: [PATCH 05/48] params: add Constantinople block to AllXYZProtocolChanges (#18162) * params: Add Constantinople block to AllCliqueProtocolChanges * params: Add Constantinople block to AllEthashProtocolChanges --- params/config.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/params/config.go b/params/config.go index 007e4a66d..7f04e0e67 100644 --- a/params/config.go +++ b/params/config.go @@ -111,14 +111,14 @@ var ( // // This configuration is intentionally not using keyed fields to force anyone // adding flags to the config to also have to set these fields. - AllEthashProtocolChanges = &ChainConfig{big.NewInt(1337), big.NewInt(0), nil, false, big.NewInt(0), common.Hash{}, big.NewInt(0), big.NewInt(0), big.NewInt(0), nil, nil, new(EthashConfig), nil} + AllEthashProtocolChanges = &ChainConfig{big.NewInt(1337), big.NewInt(0), nil, false, big.NewInt(0), common.Hash{}, big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), nil, new(EthashConfig), nil} // AllCliqueProtocolChanges contains every protocol change (EIPs) introduced // and accepted by the Ethereum core developers into the Clique consensus. // // This configuration is intentionally not using keyed fields to force anyone // adding flags to the config to also have to set these fields. - AllCliqueProtocolChanges = &ChainConfig{big.NewInt(1337), big.NewInt(0), nil, false, big.NewInt(0), common.Hash{}, big.NewInt(0), big.NewInt(0), big.NewInt(0), nil, nil, nil, &CliqueConfig{Period: 0, Epoch: 30000}} + AllCliqueProtocolChanges = &ChainConfig{big.NewInt(1337), big.NewInt(0), nil, false, big.NewInt(0), common.Hash{}, big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), nil, nil, &CliqueConfig{Period: 0, Epoch: 30000}} TestChainConfig = &ChainConfig{big.NewInt(1), big.NewInt(0), nil, false, big.NewInt(0), common.Hash{}, big.NewInt(0), big.NewInt(0), big.NewInt(0), nil, nil, new(EthashConfig), nil} TestRules = TestChainConfig.Rules(new(big.Int)) -- 2.45.2 From 2843001ac218040f7e773159596625654f4e4148 Mon Sep 17 00:00:00 2001 From: Martin Holst Swende Date: Thu, 22 Nov 2018 14:14:31 +0100 Subject: [PATCH 06/48] trie: fix overflow in write cache parent tracking (#18165) trie/database: fix overflow in parent tracking --- trie/database.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/trie/database.go b/trie/database.go index 71190b3f3..739a98add 100644 --- a/trie/database.go +++ b/trie/database.go @@ -141,7 +141,7 @@ type cachedNode struct { node node // Cached collapsed trie node, or raw rlp data size uint16 // Byte size of the useful cached data - parents uint16 // Number of live nodes referencing this one + parents uint32 // Number of live nodes referencing this one children map[common.Hash]uint16 // External children referenced by this node flushPrev common.Hash // Previous node in the flush-list -- 2.45.2 From 6b2cc8950e55b1d93afb3c82bb1081cf931d62cf Mon Sep 17 00:00:00 2001 From: Anton Evangelatov Date: Thu, 22 Nov 2018 15:32:50 +0100 Subject: [PATCH 07/48] travis: increase open file limits (#18155) --- .travis.yml | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/.travis.yml b/.travis.yml index c1cc7c4aa..33a4f8949 100644 --- a/.travis.yml +++ b/.travis.yml @@ -29,6 +29,14 @@ matrix: - os: osx go: 1.11.x script: + - echo "Increase the maximum number of open file descriptors on macOS" + - NOFILE=20480 + - sudo sysctl -w kern.maxfiles=$NOFILE + - sudo sysctl -w kern.maxfilesperproc=$NOFILE + - sudo launchctl limit maxfiles $NOFILE $NOFILE + - sudo launchctl limit maxfiles + - ulimit -S -n $NOFILE + - ulimit -n - unset -f cd # workaround for https://github.com/travis-ci/travis-ci/issues/8703 - go run build/ci.go install - go run build/ci.go test -coverage $TEST_PACKAGES -- 2.45.2 From 76f5f662ccaf5190eb283ab8b5d607587e1ab8f9 Mon Sep 17 00:00:00 2001 From: Ferenc Szabo Date: Fri, 23 Nov 2018 01:32:34 +0100 Subject: [PATCH 08/48] cmd/swarm: FUSE do not require --ipcpath (#18112) - Have `${DataDir}/bzzd.ipc` as IPC path default. - Respect the `--datadir` flag. - Keep only the global `--ipcpath` flag and drop the local `--ipcpath` flag as flags might overwrite each other. (Note: before global `--ipcpath` was ignored even if it was set) fixes ethersphere#795 --- cmd/swarm/config_test.go | 24 ++++++++++++------------ cmd/swarm/fs.go | 36 +++++++++++++++++------------------- cmd/swarm/fs_test.go | 30 ++++++++++++++++++++++++++---- cmd/utils/flags.go | 24 ++++++++++++++---------- 4 files changed, 69 insertions(+), 45 deletions(-) diff --git a/cmd/swarm/config_test.go b/cmd/swarm/config_test.go index 02198f878..18be316e5 100644 --- a/cmd/swarm/config_test.go +++ b/cmd/swarm/config_test.go @@ -26,14 +26,14 @@ import ( "testing" "time" + "github.com/docker/docker/pkg/reexec" + "github.com/ethereum/go-ethereum/cmd/utils" "github.com/ethereum/go-ethereum/rpc" "github.com/ethereum/go-ethereum/swarm" "github.com/ethereum/go-ethereum/swarm/api" - - "github.com/docker/docker/pkg/reexec" ) -func TestDumpConfig(t *testing.T) { +func TestConfigDump(t *testing.T) { swarm := runSwarm(t, "dumpconfig") defaultConf := api.NewConfig() out, err := tomlSettings.Marshal(&defaultConf) @@ -91,8 +91,8 @@ func TestConfigCmdLineOverrides(t *testing.T) { fmt.Sprintf("--%s", SwarmAccountFlag.Name), account.Address.String(), fmt.Sprintf("--%s", SwarmDeliverySkipCheckFlag.Name), fmt.Sprintf("--%s", EnsAPIFlag.Name), "", - "--datadir", dir, - "--ipcpath", conf.IPCPath, + fmt.Sprintf("--%s", utils.DataDirFlag.Name), dir, + fmt.Sprintf("--%s", utils.IPCPathFlag.Name), conf.IPCPath, } node.Cmd = runSwarm(t, flags...) node.Cmd.InputLine(testPassphrase) @@ -189,9 +189,9 @@ func TestConfigFileOverrides(t *testing.T) { flags := []string{ fmt.Sprintf("--%s", SwarmTomlConfigPathFlag.Name), f.Name(), fmt.Sprintf("--%s", SwarmAccountFlag.Name), account.Address.String(), - "--ens-api", "", - "--ipcpath", conf.IPCPath, - "--datadir", dir, + fmt.Sprintf("--%s", EnsAPIFlag.Name), "", + fmt.Sprintf("--%s", utils.DataDirFlag.Name), dir, + fmt.Sprintf("--%s", utils.IPCPathFlag.Name), conf.IPCPath, } node.Cmd = runSwarm(t, flags...) node.Cmd.InputLine(testPassphrase) @@ -407,9 +407,9 @@ func TestConfigCmdLineOverridesFile(t *testing.T) { fmt.Sprintf("--%s", SwarmSyncDisabledFlag.Name), fmt.Sprintf("--%s", SwarmTomlConfigPathFlag.Name), f.Name(), fmt.Sprintf("--%s", SwarmAccountFlag.Name), account.Address.String(), - "--ens-api", "", - "--datadir", dir, - "--ipcpath", conf.IPCPath, + fmt.Sprintf("--%s", EnsAPIFlag.Name), "", + fmt.Sprintf("--%s", utils.DataDirFlag.Name), dir, + fmt.Sprintf("--%s", utils.IPCPathFlag.Name), conf.IPCPath, } node.Cmd = runSwarm(t, flags...) node.Cmd.InputLine(testPassphrase) @@ -466,7 +466,7 @@ func TestConfigCmdLineOverridesFile(t *testing.T) { node.Shutdown() } -func TestValidateConfig(t *testing.T) { +func TestConfigValidate(t *testing.T) { for _, c := range []struct { cfg *api.Config err string diff --git a/cmd/swarm/fs.go b/cmd/swarm/fs.go index b970b2e8c..edeeddff8 100644 --- a/cmd/swarm/fs.go +++ b/cmd/swarm/fs.go @@ -24,7 +24,7 @@ import ( "time" "github.com/ethereum/go-ethereum/cmd/utils" - "github.com/ethereum/go-ethereum/node" + "github.com/ethereum/go-ethereum/log" "github.com/ethereum/go-ethereum/rpc" "github.com/ethereum/go-ethereum/swarm/fuse" "gopkg.in/urfave/cli.v1" @@ -41,27 +41,24 @@ var fsCommand = cli.Command{ Action: mount, CustomHelpTemplate: helpTemplate, Name: "mount", - Flags: []cli.Flag{utils.IPCPathFlag}, Usage: "mount a swarm hash to a mount point", - ArgsUsage: "swarm fs mount --ipcpath ", + ArgsUsage: "swarm fs mount ", Description: "Mounts a Swarm manifest hash to a given mount point. This assumes you already have a Swarm node running locally. You must reference the correct path to your bzzd.ipc file", }, { Action: unmount, CustomHelpTemplate: helpTemplate, Name: "unmount", - Flags: []cli.Flag{utils.IPCPathFlag}, Usage: "unmount a swarmfs mount", - ArgsUsage: "swarm fs unmount --ipcpath ", + ArgsUsage: "swarm fs unmount ", Description: "Unmounts a swarmfs mount residing at . This assumes you already have a Swarm node running locally. You must reference the correct path to your bzzd.ipc file", }, { Action: listMounts, CustomHelpTemplate: helpTemplate, Name: "list", - Flags: []cli.Flag{utils.IPCPathFlag}, Usage: "list swarmfs mounts", - ArgsUsage: "swarm fs list --ipcpath ", + ArgsUsage: "swarm fs list", Description: "Lists all mounted swarmfs volumes. This assumes you already have a Swarm node running locally. You must reference the correct path to your bzzd.ipc file", }, }, @@ -70,7 +67,7 @@ var fsCommand = cli.Command{ func mount(cliContext *cli.Context) { args := cliContext.Args() if len(args) < 2 { - utils.Fatalf("Usage: swarm fs mount --ipcpath ") + utils.Fatalf("Usage: swarm fs mount ") } client, err := dialRPC(cliContext) @@ -97,7 +94,7 @@ func unmount(cliContext *cli.Context) { args := cliContext.Args() if len(args) < 1 { - utils.Fatalf("Usage: swarm fs unmount --ipcpath ") + utils.Fatalf("Usage: swarm fs unmount ") } client, err := dialRPC(cliContext) if err != nil { @@ -145,20 +142,21 @@ func listMounts(cliContext *cli.Context) { } func dialRPC(ctx *cli.Context) (*rpc.Client, error) { - var endpoint string + endpoint := getIPCEndpoint(ctx) + log.Info("IPC endpoint", "path", endpoint) + return rpc.Dial(endpoint) +} - if ctx.IsSet(utils.IPCPathFlag.Name) { - endpoint = ctx.String(utils.IPCPathFlag.Name) - } else { - utils.Fatalf("swarm ipc endpoint not specified") - } +func getIPCEndpoint(ctx *cli.Context) string { + cfg := defaultNodeConfig + utils.SetNodeConfig(ctx, &cfg) - if endpoint == "" { - endpoint = node.DefaultIPCEndpoint(clientIdentifier) - } else if strings.HasPrefix(endpoint, "rpc:") || strings.HasPrefix(endpoint, "ipc:") { + endpoint := cfg.IPCEndpoint() + + if strings.HasPrefix(endpoint, "rpc:") || strings.HasPrefix(endpoint, "ipc:") { // Backwards compatibility with geth < 1.5 which required // these prefixes. endpoint = endpoint[4:] } - return rpc.Dial(endpoint) + return endpoint } diff --git a/cmd/swarm/fs_test.go b/cmd/swarm/fs_test.go index ac4223b66..5f58d6c0d 100644 --- a/cmd/swarm/fs_test.go +++ b/cmd/swarm/fs_test.go @@ -20,6 +20,7 @@ package main import ( "bytes" + "fmt" "io" "io/ioutil" "os" @@ -28,6 +29,7 @@ import ( "testing" "time" + "github.com/ethereum/go-ethereum/cmd/utils" "github.com/ethereum/go-ethereum/log" ) @@ -36,6 +38,26 @@ type testFile struct { content string } +// TestCLISwarmFsDefaultIPCPath tests if the most basic fs command, i.e., list +// can find and correctly connect to a running Swarm node on the default +// IPCPath. +func TestCLISwarmFsDefaultIPCPath(t *testing.T) { + cluster := newTestCluster(t, 1) + defer cluster.Shutdown() + + handlingNode := cluster.Nodes[0] + list := runSwarm(t, []string{ + "--datadir", handlingNode.Dir, + "fs", + "list", + }...) + + list.WaitExit() + if list.Err != nil { + t.Fatal(list.Err) + } +} + // TestCLISwarmFs is a high-level test of swarmfs // // This test fails on travis for macOS as this executable exits with code 1 @@ -59,9 +81,9 @@ func TestCLISwarmFs(t *testing.T) { log.Debug("swarmfs cli test: mounting first run", "ipc path", filepath.Join(handlingNode.Dir, handlingNode.IpcPath)) mount := runSwarm(t, []string{ + fmt.Sprintf("--%s", utils.IPCPathFlag.Name), filepath.Join(handlingNode.Dir, handlingNode.IpcPath), "fs", "mount", - "--ipcpath", filepath.Join(handlingNode.Dir, handlingNode.IpcPath), mhash, mountPoint, }...) @@ -101,9 +123,9 @@ func TestCLISwarmFs(t *testing.T) { log.Debug("swarmfs cli test: unmounting first run...", "ipc path", filepath.Join(handlingNode.Dir, handlingNode.IpcPath)) unmount := runSwarm(t, []string{ + fmt.Sprintf("--%s", utils.IPCPathFlag.Name), filepath.Join(handlingNode.Dir, handlingNode.IpcPath), "fs", "unmount", - "--ipcpath", filepath.Join(handlingNode.Dir, handlingNode.IpcPath), mountPoint, }...) _, matches := unmount.ExpectRegexp(hashRegexp) @@ -136,9 +158,9 @@ func TestCLISwarmFs(t *testing.T) { //remount, check files newMount := runSwarm(t, []string{ + fmt.Sprintf("--%s", utils.IPCPathFlag.Name), filepath.Join(handlingNode.Dir, handlingNode.IpcPath), "fs", "mount", - "--ipcpath", filepath.Join(handlingNode.Dir, handlingNode.IpcPath), hash, // the latest hash secondMountPoint, }...) @@ -172,9 +194,9 @@ func TestCLISwarmFs(t *testing.T) { log.Debug("swarmfs cli test: unmounting second run", "ipc path", filepath.Join(handlingNode.Dir, handlingNode.IpcPath)) unmountSec := runSwarm(t, []string{ + fmt.Sprintf("--%s", utils.IPCPathFlag.Name), filepath.Join(handlingNode.Dir, handlingNode.IpcPath), "fs", "unmount", - "--ipcpath", filepath.Join(handlingNode.Dir, handlingNode.IpcPath), secondMountPoint, }...) diff --git a/cmd/utils/flags.go b/cmd/utils/flags.go index d7b698c7e..d0597c2f1 100644 --- a/cmd/utils/flags.go +++ b/cmd/utils/flags.go @@ -978,16 +978,7 @@ func SetNodeConfig(ctx *cli.Context, cfg *node.Config) { setWS(ctx, cfg) setNodeUserIdent(ctx, cfg) - switch { - case ctx.GlobalIsSet(DataDirFlag.Name): - cfg.DataDir = ctx.GlobalString(DataDirFlag.Name) - case ctx.GlobalBool(DeveloperFlag.Name): - cfg.DataDir = "" // unless explicitly requested, use memory databases - case ctx.GlobalBool(TestnetFlag.Name): - cfg.DataDir = filepath.Join(node.DefaultDataDir(), "testnet") - case ctx.GlobalBool(RinkebyFlag.Name): - cfg.DataDir = filepath.Join(node.DefaultDataDir(), "rinkeby") - } + setDataDir(ctx, cfg) if ctx.GlobalIsSet(KeyStoreDirFlag.Name) { cfg.KeyStoreDir = ctx.GlobalString(KeyStoreDirFlag.Name) @@ -1000,6 +991,19 @@ func SetNodeConfig(ctx *cli.Context, cfg *node.Config) { } } +func setDataDir(ctx *cli.Context, cfg *node.Config) { + switch { + case ctx.GlobalIsSet(DataDirFlag.Name): + cfg.DataDir = ctx.GlobalString(DataDirFlag.Name) + case ctx.GlobalBool(DeveloperFlag.Name): + cfg.DataDir = "" // unless explicitly requested, use memory databases + case ctx.GlobalBool(TestnetFlag.Name): + cfg.DataDir = filepath.Join(node.DefaultDataDir(), "testnet") + case ctx.GlobalBool(RinkebyFlag.Name): + cfg.DataDir = filepath.Join(node.DefaultDataDir(), "rinkeby") + } +} + func setGPO(ctx *cli.Context, cfg *gasprice.Config) { if ctx.GlobalIsSet(GpoBlocksFlag.Name) { cfg.Blocks = ctx.GlobalInt(GpoBlocksFlag.Name) -- 2.45.2 From b24ef5e05d158e1533cf339aea866e0584447cef Mon Sep 17 00:00:00 2001 From: Felix Lange Date: Fri, 23 Nov 2018 10:14:09 +0100 Subject: [PATCH 09/48] eth: increase timeout in TestBroadcastBlock (#18064) --- eth/handler_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/eth/handler_test.go b/eth/handler_test.go index 7811cd480..44824fd0b 100644 --- a/eth/handler_test.go +++ b/eth/handler_test.go @@ -585,7 +585,7 @@ func testBroadcastBlock(t *testing.T, totalPeers, broadcastExpected int) { } }(peer) } - timeoutCh := time.NewTimer(time.Millisecond * 100).C + timeout := time.After(300 * time.Millisecond) var receivedCount int outer: for { @@ -597,7 +597,7 @@ outer: if receivedCount == totalPeers { break outer } - case <-timeoutCh: + case <-timeout: break outer } } -- 2.45.2 From 2a113f6d72d3b612c6686fa2ab8c30f9e7e6e972 Mon Sep 17 00:00:00 2001 From: mr_franklin Date: Fri, 23 Nov 2018 17:16:14 +0800 Subject: [PATCH 10/48] core: return error if repair block failed (#18126) * core: return error if repair block failed * make error a bit shorter --- core/blockchain.go | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/core/blockchain.go b/core/blockchain.go index 3a9caf01e..bcd53d498 100644 --- a/core/blockchain.go +++ b/core/blockchain.go @@ -445,7 +445,11 @@ func (bc *BlockChain) repair(head **types.Block) error { return nil } // Otherwise rewind one block and recheck state availability there - (*head) = bc.GetBlock((*head).ParentHash(), (*head).NumberU64()-1) + block := bc.GetBlock((*head).ParentHash(), (*head).NumberU64()-1) + if block == nil { + return fmt.Errorf("missing block %d [%x]", (*head).NumberU64()-1, (*head).ParentHash()) + } + (*head) = block } } -- 2.45.2 From 26e8088cc5f676bcdbd51c889a8303d4603cefd8 Mon Sep 17 00:00:00 2001 From: Ian Norden Date: Sun, 25 Nov 2018 23:52:53 -0600 Subject: [PATCH 11/48] integrating state diff extracting, building, and persisting into geth processes --- cmd/geth/main.go | 1 + cmd/utils/flags.go | 17 ++++ core/blockchain.go | 14 ++++ eth/backend.go | 8 +- eth/config.go | 4 + .../{statediff_builder.go => builder.go} | 43 +++++++--- statediff/builder_test.go | 21 +++++ statediff/config.go | 80 +++++++++++++++++++ statediff/extractor.go | 50 ++++++++++++ statediff/extractor_test.go | 20 +++++ statediff/helpers.go | 19 +++++ statediff/persister.go | 40 ++++++++++ statediff/persister_test.go | 20 +++++ statediff/statediff.go | 19 +++++ 14 files changed, 343 insertions(+), 13 deletions(-) rename statediff/{statediff_builder.go => builder.go} (78%) create mode 100644 statediff/builder_test.go create mode 100644 statediff/config.go create mode 100644 statediff/extractor.go create mode 100644 statediff/extractor_test.go create mode 100644 statediff/persister.go create mode 100644 statediff/persister_test.go diff --git a/cmd/geth/main.go b/cmd/geth/main.go index 69802a48a..15f765548 100644 --- a/cmd/geth/main.go +++ b/cmd/geth/main.go @@ -132,6 +132,7 @@ var ( utils.GpoPercentileFlag, utils.EWASMInterpreterFlag, utils.EVMInterpreterFlag, + utils.StateDiffFlag, configFileFlag, } diff --git a/cmd/utils/flags.go b/cmd/utils/flags.go index d7b698c7e..c18d98383 100644 --- a/cmd/utils/flags.go +++ b/cmd/utils/flags.go @@ -626,6 +626,12 @@ var ( Usage: "External EVM configuration (default = built-in interpreter)", Value: "", } + + // Statediff flags + StateDiffFlag = cli.BoolFlag{ + Name: "statediff", + Usage: "Enables the calculation of state diffs between each block, persists these state diffs in ipfs", + } ) // MakeDataDir retrieves the currently requested data directory, terminating @@ -1127,6 +1133,13 @@ func SetShhConfig(ctx *cli.Context, stack *node.Node, cfg *whisper.Config) { } } +// Check if state diff flags are on and applies them to eth context +func setStateDiff(ctx *cli.Context, cfg *eth.Config) { + if ctx.GlobalBool(StateDiffFlag.Name) && cfg.NoPruning && cfg.SyncMode == downloader.FullSync { + cfg.StateDiff.On = true + } +} + // SetEthConfig applies eth-related command line flags to the config. func SetEthConfig(ctx *cli.Context, stack *node.Node, cfg *eth.Config) { // Avoid conflicting network flags @@ -1162,6 +1175,10 @@ func SetEthConfig(ctx *cli.Context, stack *node.Node, cfg *eth.Config) { } cfg.NoPruning = ctx.GlobalString(GCModeFlag.Name) == "archive" + if ctx.GlobalIsSet(StateDiffFlag.Name) { + setStateDiff(ctx, cfg) + } + if ctx.GlobalIsSet(CacheFlag.Name) || ctx.GlobalIsSet(CacheTrieFlag.Name) { cfg.TrieCleanCache = ctx.GlobalInt(CacheFlag.Name) * ctx.GlobalInt(CacheTrieFlag.Name) / 100 } diff --git a/core/blockchain.go b/core/blockchain.go index d173b2de2..fed6b4b74 100644 --- a/core/blockchain.go +++ b/core/blockchain.go @@ -20,6 +20,7 @@ package core import ( "errors" "fmt" + "github.com/ethereum/go-ethereum/statediff" "io" "math/big" mrand "math/rand" @@ -72,6 +73,7 @@ type CacheConfig struct { TrieCleanLimit int // Memory allowance (MB) to use for caching trie nodes in memory TrieDirtyLimit int // Memory limit (MB) at which to start flushing dirty trie nodes to disk TrieTimeLimit time.Duration // Time limit after which to flush the current in-memory trie to disk + StateDiff bool // Whether or not to calculate and persist state diffs } // BlockChain represents the canonical chain given a database with a genesis @@ -133,6 +135,8 @@ type BlockChain struct { badBlocks *lru.Cache // Bad block cache shouldPreserve func(*types.Block) bool // Function used to determine whether should preserve the given block. + + diffExtractor statediff.Extractor // State diff processing interface } // NewBlockChain returns a fully initialised block chain using information @@ -173,6 +177,10 @@ func NewBlockChain(db ethdb.Database, cacheConfig *CacheConfig, chainConfig *par bc.SetValidator(NewBlockValidator(chainConfig, bc, engine)) bc.SetProcessor(NewStateProcessor(chainConfig, bc, engine)) + if cacheConfig.StateDiff { + bc.diffExtractor = statediff.NewExtractor(db) + } + var err error bc.hc, err = NewHeaderChain(db, chainConfig, engine, bc.getProcInterrupt) if err != nil { @@ -1187,6 +1195,7 @@ func (bc *BlockChain) insertChain(chain types.Blocks) (int, []interface{}, []*ty parent = chain[i-1] } state, err := state.New(parent.Root(), bc.stateCache) + if err != nil { return i, events, coalescedLogs, err } @@ -1204,6 +1213,11 @@ func (bc *BlockChain) insertChain(chain types.Blocks) (int, []interface{}, []*ty } proctime := time.Since(bstart) + // If extracting statediffs, do so now + if bc.cacheConfig.StateDiff { + bc.diffExtractor.Extract(*parent, *block) + } + // Write the block to the chain and get the status. status, err := bc.WriteBlockWithState(block, receipts, state) if err != nil { diff --git a/eth/backend.go b/eth/backend.go index 472140842..d96175643 100644 --- a/eth/backend.go +++ b/eth/backend.go @@ -154,7 +154,13 @@ func New(ctx *node.ServiceContext, config *Config) (*Ethereum, error) { EWASMInterpreter: config.EWASMInterpreter, EVMInterpreter: config.EVMInterpreter, } - cacheConfig = &core.CacheConfig{Disabled: config.NoPruning, TrieCleanLimit: config.TrieCleanCache, TrieDirtyLimit: config.TrieDirtyCache, TrieTimeLimit: config.TrieTimeout} + cacheConfig = &core.CacheConfig{ + Disabled: config.NoPruning, + TrieCleanLimit: config.TrieCleanCache, + TrieDirtyLimit: config.TrieDirtyCache, + TrieTimeLimit: config.TrieTimeout, + StateDiff: config.StateDiff.On, + } ) eth.blockchain, err = core.NewBlockChain(chainDb, cacheConfig, eth.chainConfig, eth.engine, vmConfig, eth.shouldPreserve) if err != nil { diff --git a/eth/config.go b/eth/config.go index 601f4735e..c675b6d08 100644 --- a/eth/config.go +++ b/eth/config.go @@ -17,6 +17,7 @@ package eth import ( + "github.com/ethereum/go-ethereum/statediff" "math/big" "os" "os/user" @@ -128,6 +129,9 @@ type Config struct { EWASMInterpreter string // Type of the EVM interpreter ("" for default) EVMInterpreter string + + // Config for state diff building + StateDiff statediff.Config } type configMarshaling struct { diff --git a/statediff/statediff_builder.go b/statediff/builder.go similarity index 78% rename from statediff/statediff_builder.go rename to statediff/builder.go index c3011ed66..9017c240d 100644 --- a/statediff/statediff_builder.go +++ b/statediff/builder.go @@ -1,3 +1,22 @@ +// Copyright 2015 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +// Contains a batch of utility type declarations used by the tests. As the node +// operates on unique types, a lot of them are needed to check various features. + package statediff import ( @@ -9,24 +28,24 @@ import ( "github.com/ethereum/go-ethereum/trie" ) -type StateDiffBuilder interface { - CreateStateDiff(oldStateRoot, newStateRoot common.Hash, blockNumber int64, blockHash common.Hash) (*StateDiff, error) +type Builder interface { + BuildStateDiff(oldStateRoot, newStateRoot common.Hash, blockNumber int64, blockHash common.Hash) (*StateDiff, error) } -type stateDiffBuilder struct { +type builder struct { chainDB ethdb.Database trieDB *trie.Database cachedTrie *trie.Trie } -func NewStateDiffBuilder(db ethdb.Database) *stateDiffBuilder { - return &stateDiffBuilder{ +func NewBuilder(db ethdb.Database) *builder { + return &builder{ chainDB: db, trieDB: trie.NewDatabase(db), } } -func (sdb *stateDiffBuilder) CreateStateDiff(oldStateRoot, newStateRoot common.Hash, blockNumber int64, blockHash common.Hash) (*StateDiff, error) { +func (sdb *builder) BuildStateDiff(oldStateRoot, newStateRoot common.Hash, blockNumber int64, blockHash common.Hash) (*StateDiff, error) { // Generate tries for old and new states oldTrie, err := trie.New(oldStateRoot, sdb.trieDB) if err != nil { @@ -81,7 +100,7 @@ func (sdb *stateDiffBuilder) CreateStateDiff(oldStateRoot, newStateRoot common.H }, nil } -func (sdb *stateDiffBuilder) collectDiffNodes(a, b trie.NodeIterator) (map[common.Address]*state.Account, error) { +func (sdb *builder) collectDiffNodes(a, b trie.NodeIterator) (map[common.Address]*state.Account, error) { var diffAccounts map[common.Address]*state.Account it, _ := trie.NewDifferenceIterator(a, b) @@ -117,7 +136,7 @@ func (sdb *stateDiffBuilder) collectDiffNodes(a, b trie.NodeIterator) (map[commo return diffAccounts, nil } -func (sdb *stateDiffBuilder) buildDiffEventual(accounts map[common.Address]*state.Account, created bool) (map[common.Address]AccountDiffEventual, error) { +func (sdb *builder) buildDiffEventual(accounts map[common.Address]*state.Account, created bool) (map[common.Address]AccountDiffEventual, error) { accountDiffs := make(map[common.Address]AccountDiffEventual) for addr, val := range accounts { sr := val.Root @@ -178,7 +197,7 @@ func (sdb *stateDiffBuilder) buildDiffEventual(accounts map[common.Address]*stat return accountDiffs, nil } -func (sdb *stateDiffBuilder) buildDiffIncremental(creations map[common.Address]*state.Account, deletions map[common.Address]*state.Account, updatedKeys *[]string) (map[common.Address]AccountDiffIncremental, error) { +func (sdb *builder) buildDiffIncremental(creations map[common.Address]*state.Account, deletions map[common.Address]*state.Account, updatedKeys *[]string) (map[common.Address]AccountDiffIncremental, error) { updatedAccounts := make(map[common.Address]AccountDiffIncremental) for _, val := range *updatedKeys { createdAcc := creations[common.HexToAddress(val)] @@ -221,7 +240,7 @@ func (sdb *stateDiffBuilder) buildDiffIncremental(creations map[common.Address]* return updatedAccounts, nil } -func (sdb *stateDiffBuilder) buildStorageDiffsEventual(sr common.Hash, creation bool) (map[string]diffString, error) { +func (sdb *builder) buildStorageDiffsEventual(sr common.Hash, creation bool) (map[string]diffString, error) { log.Debug("Storage Root For Eventual Diff", "root", sr.Hex()) sTrie, err := trie.New(sr, sdb.trieDB) if err != nil { @@ -249,7 +268,7 @@ func (sdb *stateDiffBuilder) buildStorageDiffsEventual(sr common.Hash, creation return storageDiffs, nil } -func (sdb *stateDiffBuilder) buildStorageDiffsIncremental(oldSR common.Hash, newSR common.Hash) (map[string]diffString, error) { +func (sdb *builder) buildStorageDiffsIncremental(oldSR common.Hash, newSR common.Hash) (map[string]diffString, error) { log.Debug("Storage Roots for Incremental Diff", "old", oldSR.Hex(), "new", newSR.Hex()) oldTrie, err := trie.New(oldSR, sdb.trieDB) if err != nil { @@ -285,7 +304,7 @@ func (sdb *stateDiffBuilder) buildStorageDiffsIncremental(oldSR common.Hash, new return storageDiffs, nil } -func (sdb *stateDiffBuilder) addressByPath(path []byte) (*common.Address, error) { +func (sdb *builder) addressByPath(path []byte) (*common.Address, error) { // db := core.PreimageTable(sdb.chainDb) log.Debug("Looking up address from path", "path", common.ToHex(append([]byte("secure-key-"), path...))) // if addrBytes,err := db.Get(path); err != nil { diff --git a/statediff/builder_test.go b/statediff/builder_test.go new file mode 100644 index 000000000..6433b167a --- /dev/null +++ b/statediff/builder_test.go @@ -0,0 +1,21 @@ +// Copyright 2015 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +// Contains a batch of utility type declarations used by the tests. As the node +// operates on unique types, a lot of them are needed to check various features. + +package statediff_test + diff --git a/statediff/config.go b/statediff/config.go new file mode 100644 index 000000000..ee31f271e --- /dev/null +++ b/statediff/config.go @@ -0,0 +1,80 @@ +// Copyright 2015 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +// Contains a batch of utility type declarations used by the tests. As the node +// operates on unique types, a lot of them are needed to check various features. + +package statediff + +import "fmt" + +type Config struct { + On bool + Mode StateDiffMode +} + +type StateDiffMode int + +const ( + IPFS StateDiffMode = iota + LDB + SQL +) + +func (mode StateDiffMode) IsValid() bool { + return mode >= IPFS && mode <= SQL +} + +// String implements the stringer interface. +func (mode StateDiffMode) String() string { + switch mode { + case IPFS: + return "ipfs" + case LDB: + return "ldb" + case SQL: + return "sql" + default: + return "unknown" + } +} + +func (mode StateDiffMode) MarshalText() ([]byte, error) { + switch mode { + case IPFS: + return []byte("ipfs"), nil + case LDB: + return []byte("ldb"), nil + case SQL: + return []byte("sql"), nil + default: + return nil, fmt.Errorf("unknown state diff storage mode %d", mode) + } +} + +func (mode *StateDiffMode) UnmarshalText(text []byte) error { + switch string(text) { + case "ipfs": + *mode = IPFS + case "ldb": + *mode = LDB + case "sql": + *mode = SQL + default: + return fmt.Errorf(`unknown state diff storage mode %q, want "ipfs", "ldb" or "sql"`, text) + } + return nil +} \ No newline at end of file diff --git a/statediff/extractor.go b/statediff/extractor.go new file mode 100644 index 000000000..2abe3ebb3 --- /dev/null +++ b/statediff/extractor.go @@ -0,0 +1,50 @@ +// Copyright 2015 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +// Contains a batch of utility type declarations used by the tests. As the node +// operates on unique types, a lot of them are needed to check various features. + +package statediff + +import ( + "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/ethdb" +) + +type Extractor interface { + ExtractStateDiff(parent, current types.Block) error +} + +type extractor struct { + b *builder + p *persister +} + +func NewExtractor(db ethdb.Database) *extractor { + return &extractor{ + b: NewBuilder(db), + p: NewPersister(), + } +} + +func (e *extractor) Extract(parent, current types.Block) error { + stateDiff, err := e.b.BuildStateDiff(parent.Root(), current.Root(), current.Number().Int64(), current.Hash()) + if err != nil { + return err + } + + return e.p.PersistStateDiff(stateDiff) +} \ No newline at end of file diff --git a/statediff/extractor_test.go b/statediff/extractor_test.go new file mode 100644 index 000000000..de5e17095 --- /dev/null +++ b/statediff/extractor_test.go @@ -0,0 +1,20 @@ +// Copyright 2015 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +// Contains a batch of utility type declarations used by the tests. As the node +// operates on unique types, a lot of them are needed to check various features. + +package statediff_test \ No newline at end of file diff --git a/statediff/helpers.go b/statediff/helpers.go index 976d21fe2..50626ab2c 100644 --- a/statediff/helpers.go +++ b/statediff/helpers.go @@ -1,3 +1,22 @@ +// Copyright 2015 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +// Contains a batch of utility type declarations used by the tests. As the node +// operates on unique types, a lot of them are needed to check various features. + package statediff import ( diff --git a/statediff/persister.go b/statediff/persister.go new file mode 100644 index 000000000..a3eba0826 --- /dev/null +++ b/statediff/persister.go @@ -0,0 +1,40 @@ +// Copyright 2015 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +// Contains a batch of utility type declarations used by the tests. As the node +// operates on unique types, a lot of them are needed to check various features. + +package statediff + +type Persister interface { + PersistStateDiff(sd *StateDiff) error +} + +type persister struct { + +} + +func NewPersister() *persister { + return &persister{ + + } +} + +func (p *persister) PersistStateDiff(sd *StateDiff) error { + //TODO: Persist state diff in IPFS + + return nil +} \ No newline at end of file diff --git a/statediff/persister_test.go b/statediff/persister_test.go new file mode 100644 index 000000000..de5e17095 --- /dev/null +++ b/statediff/persister_test.go @@ -0,0 +1,20 @@ +// Copyright 2015 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +// Contains a batch of utility type declarations used by the tests. As the node +// operates on unique types, a lot of them are needed to check various features. + +package statediff_test \ No newline at end of file diff --git a/statediff/statediff.go b/statediff/statediff.go index d980ef867..f2e606936 100644 --- a/statediff/statediff.go +++ b/statediff/statediff.go @@ -1,3 +1,22 @@ +// Copyright 2015 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +// Contains a batch of utility type declarations used by the tests. As the node +// operates on unique types, a lot of them are needed to check various features. + package statediff import ( -- 2.45.2 From a5898ba621c51e420097a48b4a3f4fa75165a8b0 Mon Sep 17 00:00:00 2001 From: Martin Holst Swende Date: Mon, 26 Nov 2018 09:55:45 +0100 Subject: [PATCH 12/48] config: add constantinople block to testchainconfig --- params/config.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/params/config.go b/params/config.go index 7f04e0e67..3570cb204 100644 --- a/params/config.go +++ b/params/config.go @@ -120,7 +120,7 @@ var ( // adding flags to the config to also have to set these fields. AllCliqueProtocolChanges = &ChainConfig{big.NewInt(1337), big.NewInt(0), nil, false, big.NewInt(0), common.Hash{}, big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), nil, nil, &CliqueConfig{Period: 0, Epoch: 30000}} - TestChainConfig = &ChainConfig{big.NewInt(1), big.NewInt(0), nil, false, big.NewInt(0), common.Hash{}, big.NewInt(0), big.NewInt(0), big.NewInt(0), nil, nil, new(EthashConfig), nil} + TestChainConfig = &ChainConfig{big.NewInt(1), big.NewInt(0), nil, false, big.NewInt(0), common.Hash{}, big.NewInt(0), big.NewInt(0), big.NewInt(0), big.NewInt(0), nil, new(EthashConfig), nil} TestRules = TestChainConfig.Rules(new(big.Int)) ) -- 2.45.2 From 38592a13a3c41f7b6cf29ace473852b68cdb6690 Mon Sep 17 00:00:00 2001 From: Jaynti Kanani Date: Mon, 26 Nov 2018 14:29:04 +0530 Subject: [PATCH 13/48] fix mixHash/nonce for parity compatible network (#18166) --- core/types/block.go | 4 ++-- core/types/gen_header_json.go | 20 ++++++++++---------- 2 files changed, 12 insertions(+), 12 deletions(-) diff --git a/core/types/block.go b/core/types/block.go index 8a21bba1e..9d11f60d8 100644 --- a/core/types/block.go +++ b/core/types/block.go @@ -81,8 +81,8 @@ type Header struct { GasUsed uint64 `json:"gasUsed" gencodec:"required"` Time *big.Int `json:"timestamp" gencodec:"required"` Extra []byte `json:"extraData" gencodec:"required"` - MixDigest common.Hash `json:"mixHash" gencodec:"required"` - Nonce BlockNonce `json:"nonce" gencodec:"required"` + MixDigest common.Hash `json:"mixHash"` + Nonce BlockNonce `json:"nonce"` } // field type overrides for gencodec diff --git a/core/types/gen_header_json.go b/core/types/gen_header_json.go index 1b92cd9cf..59a1c9c43 100644 --- a/core/types/gen_header_json.go +++ b/core/types/gen_header_json.go @@ -13,6 +13,7 @@ import ( var _ = (*headerMarshaling)(nil) +// MarshalJSON marshals as JSON. func (h Header) MarshalJSON() ([]byte, error) { type Header struct { ParentHash common.Hash `json:"parentHash" gencodec:"required"` @@ -28,8 +29,8 @@ func (h Header) MarshalJSON() ([]byte, error) { GasUsed hexutil.Uint64 `json:"gasUsed" gencodec:"required"` Time *hexutil.Big `json:"timestamp" gencodec:"required"` Extra hexutil.Bytes `json:"extraData" gencodec:"required"` - MixDigest common.Hash `json:"mixHash" gencodec:"required"` - Nonce BlockNonce `json:"nonce" gencodec:"required"` + MixDigest common.Hash `json:"mixHash"` + Nonce BlockNonce `json:"nonce"` Hash common.Hash `json:"hash"` } var enc Header @@ -52,6 +53,7 @@ func (h Header) MarshalJSON() ([]byte, error) { return json.Marshal(&enc) } +// UnmarshalJSON unmarshals from JSON. func (h *Header) UnmarshalJSON(input []byte) error { type Header struct { ParentHash *common.Hash `json:"parentHash" gencodec:"required"` @@ -67,8 +69,8 @@ func (h *Header) UnmarshalJSON(input []byte) error { GasUsed *hexutil.Uint64 `json:"gasUsed" gencodec:"required"` Time *hexutil.Big `json:"timestamp" gencodec:"required"` Extra *hexutil.Bytes `json:"extraData" gencodec:"required"` - MixDigest *common.Hash `json:"mixHash" gencodec:"required"` - Nonce *BlockNonce `json:"nonce" gencodec:"required"` + MixDigest *common.Hash `json:"mixHash"` + Nonce *BlockNonce `json:"nonce"` } var dec Header if err := json.Unmarshal(input, &dec); err != nil { @@ -126,13 +128,11 @@ func (h *Header) UnmarshalJSON(input []byte) error { return errors.New("missing required field 'extraData' for Header") } h.Extra = *dec.Extra - if dec.MixDigest == nil { - return errors.New("missing required field 'mixHash' for Header") + if dec.MixDigest != nil { + h.MixDigest = *dec.MixDigest } - h.MixDigest = *dec.MixDigest - if dec.Nonce == nil { - return errors.New("missing required field 'nonce' for Header") + if dec.Nonce != nil { + h.Nonce = *dec.Nonce } - h.Nonce = *dec.Nonce return nil } -- 2.45.2 From f0515800e6e92044466b095bbb2374465c53b6f1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Felf=C3=B6ldi=20Zsolt?= Date: Mon, 26 Nov 2018 12:34:33 +0100 Subject: [PATCH 14/48] les: fix fetcher syncing logic (#18072) --- les/fetcher.go | 56 ++++++++++++++++++++++++++------------------------ 1 file changed, 29 insertions(+), 27 deletions(-) diff --git a/les/fetcher.go b/les/fetcher.go index f0d3b188d..2615f69df 100644 --- a/les/fetcher.go +++ b/les/fetcher.go @@ -141,36 +141,39 @@ func (f *lightFetcher) syncLoop() { s := requesting requesting = false var ( - rq *distReq - reqID uint64 + rq *distReq + reqID uint64 + syncing bool ) if !f.syncing && !(newAnnounce && s) { - rq, reqID = f.nextRequest() + rq, reqID, syncing = f.nextRequest() } - syncing := f.syncing f.lock.Unlock() if rq != nil { requesting = true - _, ok := <-f.pm.reqDist.queue(rq) - if !ok { + if _, ok := <-f.pm.reqDist.queue(rq); ok { + if syncing { + f.lock.Lock() + f.syncing = true + f.lock.Unlock() + } else { + go func() { + time.Sleep(softRequestTimeout) + f.reqMu.Lock() + req, ok := f.requested[reqID] + if ok { + req.timeout = true + f.requested[reqID] = req + } + f.reqMu.Unlock() + // keep starting new requests while possible + f.requestChn <- false + }() + } + } else { f.requestChn <- false } - - if !syncing { - go func() { - time.Sleep(softRequestTimeout) - f.reqMu.Lock() - req, ok := f.requested[reqID] - if ok { - req.timeout = true - f.requested[reqID] = req - } - f.reqMu.Unlock() - // keep starting new requests while possible - f.requestChn <- false - }() - } } case reqID := <-f.timeoutChn: f.reqMu.Lock() @@ -209,6 +212,7 @@ func (f *lightFetcher) syncLoop() { f.checkSyncedHeaders(p) f.syncing = false f.lock.Unlock() + f.requestChn <- false } } } @@ -405,7 +409,7 @@ func (f *lightFetcher) requestedID(reqID uint64) bool { // nextRequest selects the peer and announced head to be requested next, amount // to be downloaded starting from the head backwards is also returned -func (f *lightFetcher) nextRequest() (*distReq, uint64) { +func (f *lightFetcher) nextRequest() (*distReq, uint64, bool) { var ( bestHash common.Hash bestAmount uint64 @@ -427,14 +431,12 @@ func (f *lightFetcher) nextRequest() (*distReq, uint64) { } } if bestTd == f.maxConfirmedTd { - return nil, 0 + return nil, 0, false } - f.syncing = bestSyncing - var rq *distReq reqID := genReqID() - if f.syncing { + if bestSyncing { rq = &distReq{ getCost: func(dp distPeer) uint64 { return 0 @@ -500,7 +502,7 @@ func (f *lightFetcher) nextRequest() (*distReq, uint64) { }, } } - return rq, reqID + return rq, reqID, bestSyncing } // deliverHeaders delivers header download request responses for processing -- 2.45.2 From 93854bbad4fb819375463d04cf6b7e66b7f272f7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jano=C5=A1=20Gulja=C5=A1?= Date: Mon, 26 Nov 2018 12:39:38 +0100 Subject: [PATCH 15/48] swarm/network/simulation: fix New function for-loop scope (#18161) --- swarm/network/simulation/node_test.go | 35 +++++++++++++++++++++ swarm/network/simulation/simulation.go | 7 +++++ swarm/network/simulation/simulation_test.go | 13 ++++++++ swarm/network/stream/delivery.go | 2 +- 4 files changed, 56 insertions(+), 1 deletion(-) diff --git a/swarm/network/simulation/node_test.go b/swarm/network/simulation/node_test.go index 086ab606f..01346ef14 100644 --- a/swarm/network/simulation/node_test.go +++ b/swarm/network/simulation/node_test.go @@ -160,6 +160,41 @@ func TestAddNodeWithService(t *testing.T) { } } +func TestAddNodeMultipleServices(t *testing.T) { + sim := New(map[string]ServiceFunc{ + "noop1": noopServiceFunc, + "noop2": noopService2Func, + }) + defer sim.Close() + + id, err := sim.AddNode() + if err != nil { + t.Fatal(err) + } + + n := sim.Net.GetNode(id).Node.(*adapters.SimNode) + if n.Service("noop1") == nil { + t.Error("service noop1 not found on node") + } + if n.Service("noop2") == nil { + t.Error("service noop2 not found on node") + } +} + +func TestAddNodeDuplicateServiceError(t *testing.T) { + sim := New(map[string]ServiceFunc{ + "noop1": noopServiceFunc, + "noop2": noopServiceFunc, + }) + defer sim.Close() + + wantErr := "duplicate service: *simulation.noopService" + _, err := sim.AddNode() + if err.Error() != wantErr { + t.Errorf("got error %q, want %q", err, wantErr) + } +} + func TestAddNodes(t *testing.T) { sim := New(noopServiceFuncMap) defer sim.Close() diff --git a/swarm/network/simulation/simulation.go b/swarm/network/simulation/simulation.go index f6d3ce229..e5435b9f0 100644 --- a/swarm/network/simulation/simulation.go +++ b/swarm/network/simulation/simulation.go @@ -68,6 +68,10 @@ type ServiceFunc func(ctx *adapters.ServiceContext, bucket *sync.Map) (s node.Se // New creates a new Simulation instance with new // simulations.Network initialized with provided services. +// Services map must have unique keys as service names and +// every ServiceFunc must return a node.Service of the unique type. +// This restriction is required by node.Node.Start() function +// which is used to start node.Service returned by ServiceFunc. func New(services map[string]ServiceFunc) (s *Simulation) { s = &Simulation{ buckets: make(map[enode.ID]*sync.Map), @@ -76,6 +80,9 @@ func New(services map[string]ServiceFunc) (s *Simulation) { adapterServices := make(map[string]adapters.ServiceFunc, len(services)) for name, serviceFunc := range services { + // Scope this variables correctly + // as they will be in the adapterServices[name] function accessed later. + name, serviceFunc := name, serviceFunc s.serviceNames = append(s.serviceNames, name) adapterServices[name] = func(ctx *adapters.ServiceContext) (node.Service, error) { b := new(sync.Map) diff --git a/swarm/network/simulation/simulation_test.go b/swarm/network/simulation/simulation_test.go index eed09bf50..ca8599d7c 100644 --- a/swarm/network/simulation/simulation_test.go +++ b/swarm/network/simulation/simulation_test.go @@ -205,3 +205,16 @@ func (t *noopService) Start(server *p2p.Server) error { func (t *noopService) Stop() error { return nil } + +// a helper function for most basic noop service +// of a different type then noopService to test +// multiple services on one node. +func noopService2Func(ctx *adapters.ServiceContext, b *sync.Map) (node.Service, func(), error) { + return new(noopService2), nil, nil +} + +// noopService2 is the service that does not do anything +// but implements node.Service interface. +type noopService2 struct { + noopService +} diff --git a/swarm/network/stream/delivery.go b/swarm/network/stream/delivery.go index 64d754336..a5c9fa7e8 100644 --- a/swarm/network/stream/delivery.go +++ b/swarm/network/stream/delivery.go @@ -255,7 +255,7 @@ func (d *Delivery) RequestFromPeers(ctx context.Context, req *network.Request) ( } sp = d.getPeer(id) if sp == nil { - log.Warn("Delivery.RequestFromPeers: peer not found", "id", id) + //log.Warn("Delivery.RequestFromPeers: peer not found", "id", id) return true } spID = &id -- 2.45.2 From f5e6634fd2fd2be5520c77edb7a5cd375eb28fd4 Mon Sep 17 00:00:00 2001 From: Elad Date: Mon, 26 Nov 2018 17:23:15 +0530 Subject: [PATCH 16/48] swarm/api: improve not found error msg (#18171) --- swarm/api/api.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/swarm/api/api.go b/swarm/api/api.go index 7bb631967..099a46939 100644 --- a/swarm/api/api.go +++ b/swarm/api/api.go @@ -472,7 +472,7 @@ func (a *API) Get(ctx context.Context, decrypt DecryptFunc, manifestAddr storage // no entry found status = http.StatusNotFound apiGetNotFound.Inc(1) - err = fmt.Errorf("manifest entry for '%s' not found", path) + err = fmt.Errorf("Not found: could not find resource '%s'", path) log.Trace("manifest entry not found", "key", contentAddr, "path", path) } return -- 2.45.2 From ca228569e4aedd5ebbe853ec61761561d86579a6 Mon Sep 17 00:00:00 2001 From: Sheldon <11510383@mail.sustc.edu.cn> Date: Mon, 26 Nov 2018 20:27:49 +0800 Subject: [PATCH 17/48] light: odrTrie tryUpdate should use update (#18107) TryUpdate does not call t.trie.TryUpdate(key, value) and calls t.trie.TryDelete instead. The update operation simply deletes the corresponding entry, though it could retrieve later by odr. However, it adds further network overhead. --- light/trie.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/light/trie.go b/light/trie.go index c07e99461..ab4e18b43 100644 --- a/light/trie.go +++ b/light/trie.go @@ -108,7 +108,7 @@ func (t *odrTrie) TryGet(key []byte) ([]byte, error) { func (t *odrTrie) TryUpdate(key, value []byte) error { key = crypto.Keccak256(key) return t.do(key, func() error { - return t.trie.TryDelete(key) + return t.trie.TryUpdate(key, value) }) } -- 2.45.2 From 197d609b9a3a9b5436a9902dcc8db38bb3e7543b Mon Sep 17 00:00:00 2001 From: lash Date: Mon, 26 Nov 2018 13:52:04 +0100 Subject: [PATCH 18/48] swarm/pss: Message handler refactor (#18169) --- swarm/network/kademlia.go | 26 +- swarm/pss/api.go | 12 +- swarm/pss/client/client.go | 2 +- swarm/pss/handshake.go | 2 +- swarm/pss/notify/notify.go | 4 +- swarm/pss/notify/notify_test.go | 4 +- swarm/pss/protocol_test.go | 5 +- swarm/pss/pss.go | 171 ++++++++--- swarm/pss/pss_test.go | 493 +++++++++++++++++++++++++++++--- swarm/pss/types.go | 34 ++- 10 files changed, 644 insertions(+), 109 deletions(-) diff --git a/swarm/network/kademlia.go b/swarm/network/kademlia.go index cd94741be..5fda51e3e 100644 --- a/swarm/network/kademlia.go +++ b/swarm/network/kademlia.go @@ -81,14 +81,15 @@ func NewKadParams() *KadParams { // Kademlia is a table of live peers and a db of known peers (node records) type Kademlia struct { lock sync.RWMutex - *KadParams // Kademlia configuration parameters - base []byte // immutable baseaddress of the table - addrs *pot.Pot // pots container for known peer addresses - conns *pot.Pot // pots container for live peer connections - depth uint8 // stores the last current depth of saturation - nDepth int // stores the last neighbourhood depth - nDepthC chan int // returned by DepthC function to signal neighbourhood depth change - addrCountC chan int // returned by AddrCountC function to signal peer count change + *KadParams // Kademlia configuration parameters + base []byte // immutable baseaddress of the table + addrs *pot.Pot // pots container for known peer addresses + conns *pot.Pot // pots container for live peer connections + depth uint8 // stores the last current depth of saturation + nDepth int // stores the last neighbourhood depth + nDepthC chan int // returned by DepthC function to signal neighbourhood depth change + addrCountC chan int // returned by AddrCountC function to signal peer count change + Pof func(pot.Val, pot.Val, int) (int, bool) // function for calculating kademlia routing distance between two addresses } // NewKademlia creates a Kademlia table for base address addr @@ -103,6 +104,7 @@ func NewKademlia(addr []byte, params *KadParams) *Kademlia { KadParams: params, addrs: pot.NewPot(nil, 0), conns: pot.NewPot(nil, 0), + Pof: pof, } } @@ -289,6 +291,7 @@ func (k *Kademlia) On(p *Peer) (uint8, bool) { // neighbourhood depth on each change. // Not receiving from the returned channel will block On function // when the neighbourhood depth is changed. +// TODO: Why is this exported, and if it should be; why can't we have more subscribers than one? func (k *Kademlia) NeighbourhoodDepthC() <-chan int { k.lock.Lock() defer k.lock.Unlock() @@ -429,7 +432,12 @@ func (k *Kademlia) eachAddr(base []byte, o int, f func(*BzzAddr, int, bool) bool // neighbourhoodDepth returns the proximity order that defines the distance of // the nearest neighbour set with cardinality >= MinProxBinSize // if there is altogether less than MinProxBinSize peers it returns 0 -// caller must hold the lock +func (k *Kademlia) NeighbourhoodDepth() (depth int) { + k.lock.RLock() + defer k.lock.RUnlock() + return k.neighbourhoodDepth() +} + func (k *Kademlia) neighbourhoodDepth() (depth int) { if k.conns.Size() < k.MinProxBinSize { return 0 diff --git a/swarm/pss/api.go b/swarm/pss/api.go index eba7bb722..dd55b2a70 100644 --- a/swarm/pss/api.go +++ b/swarm/pss/api.go @@ -51,7 +51,7 @@ func NewAPI(ps *Pss) *API { // // All incoming messages to the node matching this topic will be encapsulated in the APIMsg // struct and sent to the subscriber -func (pssapi *API) Receive(ctx context.Context, topic Topic) (*rpc.Subscription, error) { +func (pssapi *API) Receive(ctx context.Context, topic Topic, raw bool, prox bool) (*rpc.Subscription, error) { notifier, supported := rpc.NotifierFromContext(ctx) if !supported { return nil, fmt.Errorf("Subscribe not supported") @@ -59,7 +59,7 @@ func (pssapi *API) Receive(ctx context.Context, topic Topic) (*rpc.Subscription, psssub := notifier.CreateSubscription() - handler := func(msg []byte, p *p2p.Peer, asymmetric bool, keyid string) error { + hndlr := NewHandler(func(msg []byte, p *p2p.Peer, asymmetric bool, keyid string) error { apimsg := &APIMsg{ Msg: hexutil.Bytes(msg), Asymmetric: asymmetric, @@ -69,9 +69,15 @@ func (pssapi *API) Receive(ctx context.Context, topic Topic) (*rpc.Subscription, log.Warn(fmt.Sprintf("notification on pss sub topic rpc (sub %v) msg %v failed!", psssub.ID, msg)) } return nil + }) + if raw { + hndlr.caps.raw = true + } + if prox { + hndlr.caps.prox = true } - deregf := pssapi.Register(&topic, handler) + deregf := pssapi.Register(&topic, hndlr) go func() { defer deregf() select { diff --git a/swarm/pss/client/client.go b/swarm/pss/client/client.go index d541081d3..5ee387aa7 100644 --- a/swarm/pss/client/client.go +++ b/swarm/pss/client/client.go @@ -236,7 +236,7 @@ func (c *Client) RunProtocol(ctx context.Context, proto *p2p.Protocol) error { topichex := topicobj.String() msgC := make(chan pss.APIMsg) c.peerPool[topicobj] = make(map[string]*pssRPCRW) - sub, err := c.rpc.Subscribe(ctx, "pss", msgC, "receive", topichex) + sub, err := c.rpc.Subscribe(ctx, "pss", msgC, "receive", topichex, false, false) if err != nil { return fmt.Errorf("pss event subscription failed: %v", err) } diff --git a/swarm/pss/handshake.go b/swarm/pss/handshake.go index e3ead77d0..5486abafa 100644 --- a/swarm/pss/handshake.go +++ b/swarm/pss/handshake.go @@ -486,7 +486,7 @@ func (api *HandshakeAPI) Handshake(pubkeyid string, topic Topic, sync bool, flus // Activate handshake functionality on a topic func (api *HandshakeAPI) AddHandshake(topic Topic) error { - api.ctrl.deregisterFuncs[topic] = api.ctrl.pss.Register(&topic, api.ctrl.handler) + api.ctrl.deregisterFuncs[topic] = api.ctrl.pss.Register(&topic, NewHandler(api.ctrl.handler)) return nil } diff --git a/swarm/pss/notify/notify.go b/swarm/pss/notify/notify.go index 3731fb9db..d3c89058b 100644 --- a/swarm/pss/notify/notify.go +++ b/swarm/pss/notify/notify.go @@ -113,7 +113,7 @@ func NewController(ps *pss.Pss) *Controller { notifiers: make(map[string]*notifier), subscriptions: make(map[string]*subscription), } - ctrl.pss.Register(&controlTopic, ctrl.Handler) + ctrl.pss.Register(&controlTopic, pss.NewHandler(ctrl.Handler)) return ctrl } @@ -336,7 +336,7 @@ func (c *Controller) handleNotifyWithKeyMsg(msg *Msg) error { // \TODO keep track of and add actual address updaterAddr := pss.PssAddress([]byte{}) c.pss.SetSymmetricKey(symkey, topic, &updaterAddr, true) - c.pss.Register(&topic, c.Handler) + c.pss.Register(&topic, pss.NewHandler(c.Handler)) return c.subscriptions[msg.namestring].handler(msg.namestring, msg.Payload[:len(msg.Payload)-symKeyLength]) } diff --git a/swarm/pss/notify/notify_test.go b/swarm/pss/notify/notify_test.go index d4d383a6b..6100195b0 100644 --- a/swarm/pss/notify/notify_test.go +++ b/swarm/pss/notify/notify_test.go @@ -121,7 +121,7 @@ func TestStart(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Second*2) defer cancel() rmsgC := make(chan *pss.APIMsg) - rightSub, err := rightRpc.Subscribe(ctx, "pss", rmsgC, "receive", controlTopic) + rightSub, err := rightRpc.Subscribe(ctx, "pss", rmsgC, "receive", controlTopic, false, false) if err != nil { t.Fatal(err) } @@ -174,7 +174,7 @@ func TestStart(t *testing.T) { t.Fatalf("expected payload length %d, have %d", len(updateMsg)+symKeyLength, len(dMsg.Payload)) } - rightSubUpdate, err := rightRpc.Subscribe(ctx, "pss", rmsgC, "receive", rsrcTopic) + rightSubUpdate, err := rightRpc.Subscribe(ctx, "pss", rmsgC, "receive", rsrcTopic, false, false) if err != nil { t.Fatal(err) } diff --git a/swarm/pss/protocol_test.go b/swarm/pss/protocol_test.go index 4ef3e90a0..520c48a20 100644 --- a/swarm/pss/protocol_test.go +++ b/swarm/pss/protocol_test.go @@ -92,7 +92,7 @@ func testProtocol(t *testing.T) { lmsgC := make(chan APIMsg) lctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() - lsub, err := clients[0].Subscribe(lctx, "pss", lmsgC, "receive", topic) + lsub, err := clients[0].Subscribe(lctx, "pss", lmsgC, "receive", topic, false, false) if err != nil { t.Fatal(err) } @@ -100,7 +100,7 @@ func testProtocol(t *testing.T) { rmsgC := make(chan APIMsg) rctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() - rsub, err := clients[1].Subscribe(rctx, "pss", rmsgC, "receive", topic) + rsub, err := clients[1].Subscribe(rctx, "pss", rmsgC, "receive", topic, false, false) if err != nil { t.Fatal(err) } @@ -130,6 +130,7 @@ func testProtocol(t *testing.T) { log.Debug("lnode ok") case cerr := <-lctx.Done(): t.Fatalf("test message timed out: %v", cerr) + return } select { case <-rmsgC: diff --git a/swarm/pss/pss.go b/swarm/pss/pss.go index e1e24e1f5..d0986d280 100644 --- a/swarm/pss/pss.go +++ b/swarm/pss/pss.go @@ -23,11 +23,13 @@ import ( "crypto/rand" "errors" "fmt" + "hash" "sync" "time" "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/crypto" + "github.com/ethereum/go-ethereum/crypto/sha3" "github.com/ethereum/go-ethereum/metrics" "github.com/ethereum/go-ethereum/p2p" "github.com/ethereum/go-ethereum/p2p/enode" @@ -136,10 +138,10 @@ type Pss struct { symKeyDecryptCacheCapacity int // max amount of symkeys to keep. // message handling - handlers map[Topic]map[*Handler]bool // topic and version based pss payload handlers. See pss.Handle() - handlersMu sync.RWMutex - allowRaw bool - hashPool sync.Pool + handlers map[Topic]map[*handler]bool // topic and version based pss payload handlers. See pss.Handle() + handlersMu sync.RWMutex + hashPool sync.Pool + topicHandlerCaps map[Topic]*handlerCaps // caches capabilities of each topic's handlers (see handlerCap* consts in types.go) // process quitC chan struct{} @@ -180,11 +182,12 @@ func NewPss(k *network.Kademlia, params *PssParams) (*Pss, error) { symKeyDecryptCache: make([]*string, params.SymKeyCacheCapacity), symKeyDecryptCacheCapacity: params.SymKeyCacheCapacity, - handlers: make(map[Topic]map[*Handler]bool), - allowRaw: params.AllowRaw, + handlers: make(map[Topic]map[*handler]bool), + topicHandlerCaps: make(map[Topic]*handlerCaps), + hashPool: sync.Pool{ New: func() interface{} { - return storage.MakeHashFunc(storage.DefaultHash)() + return sha3.NewKeccak256() }, }, } @@ -313,30 +316,54 @@ func (p *Pss) PublicKey() *ecdsa.PublicKey { // // Returns a deregister function which needs to be called to // deregister the handler, -func (p *Pss) Register(topic *Topic, handler Handler) func() { +func (p *Pss) Register(topic *Topic, hndlr *handler) func() { p.handlersMu.Lock() defer p.handlersMu.Unlock() handlers := p.handlers[*topic] if handlers == nil { - handlers = make(map[*Handler]bool) + handlers = make(map[*handler]bool) p.handlers[*topic] = handlers + log.Debug("registered handler", "caps", hndlr.caps) } - handlers[&handler] = true - return func() { p.deregister(topic, &handler) } + if hndlr.caps == nil { + hndlr.caps = &handlerCaps{} + } + handlers[hndlr] = true + if _, ok := p.topicHandlerCaps[*topic]; !ok { + p.topicHandlerCaps[*topic] = &handlerCaps{} + } + if hndlr.caps.raw { + p.topicHandlerCaps[*topic].raw = true + } + if hndlr.caps.prox { + p.topicHandlerCaps[*topic].prox = true + } + return func() { p.deregister(topic, hndlr) } } -func (p *Pss) deregister(topic *Topic, h *Handler) { +func (p *Pss) deregister(topic *Topic, hndlr *handler) { p.handlersMu.Lock() defer p.handlersMu.Unlock() handlers := p.handlers[*topic] - if len(handlers) == 1 { + if len(handlers) > 1 { delete(p.handlers, *topic) + // topic caps might have changed now that a handler is gone + caps := &handlerCaps{} + for h := range handlers { + if h.caps.raw { + caps.raw = true + } + if h.caps.prox { + caps.prox = true + } + } + p.topicHandlerCaps[*topic] = caps return } - delete(handlers, h) + delete(handlers, hndlr) } // get all registered handlers for respective topics -func (p *Pss) getHandlers(topic Topic) map[*Handler]bool { +func (p *Pss) getHandlers(topic Topic) map[*handler]bool { p.handlersMu.RLock() defer p.handlersMu.RUnlock() return p.handlers[topic] @@ -348,12 +375,11 @@ func (p *Pss) getHandlers(topic Topic) map[*Handler]bool { // Only passes error to pss protocol handler if payload is not valid pssmsg func (p *Pss) handlePssMsg(ctx context.Context, msg interface{}) error { metrics.GetOrRegisterCounter("pss.handlepssmsg", nil).Inc(1) - pssmsg, ok := msg.(*PssMsg) - if !ok { return fmt.Errorf("invalid message type. Expected *PssMsg, got %T ", msg) } + log.Trace("handler", "self", label(p.Kademlia.BaseAddr()), "topic", label(pssmsg.Payload.Topic[:])) if int64(pssmsg.Expire) < time.Now().Unix() { metrics.GetOrRegisterCounter("pss.expire", nil).Inc(1) log.Warn("pss filtered expired message", "from", common.ToHex(p.Kademlia.BaseAddr()), "to", common.ToHex(pssmsg.To)) @@ -365,13 +391,34 @@ func (p *Pss) handlePssMsg(ctx context.Context, msg interface{}) error { } p.addFwdCache(pssmsg) - if !p.isSelfPossibleRecipient(pssmsg) { - log.Trace("pss was for someone else :'( ... forwarding", "pss", common.ToHex(p.BaseAddr())) + psstopic := Topic(pssmsg.Payload.Topic) + + // raw is simplest handler contingency to check, so check that first + var isRaw bool + if pssmsg.isRaw() { + if !p.topicHandlerCaps[psstopic].raw { + log.Debug("No handler for raw message", "topic", psstopic) + return nil + } + isRaw = true + } + + // check if we can be recipient: + // - no prox handler on message and partial address matches + // - prox handler on message and we are in prox regardless of partial address match + // store this result so we don't calculate again on every handler + var isProx bool + if _, ok := p.topicHandlerCaps[psstopic]; ok { + isProx = p.topicHandlerCaps[psstopic].prox + } + isRecipient := p.isSelfPossibleRecipient(pssmsg, isProx) + if !isRecipient { + log.Trace("pss was for someone else :'( ... forwarding", "pss", common.ToHex(p.BaseAddr()), "prox", isProx) return p.enqueue(pssmsg) } - log.Trace("pss for us, yay! ... let's process!", "pss", common.ToHex(p.BaseAddr())) - if err := p.process(pssmsg); err != nil { + log.Trace("pss for us, yay! ... let's process!", "pss", common.ToHex(p.BaseAddr()), "prox", isProx, "raw", isRaw, "topic", label(pssmsg.Payload.Topic[:])) + if err := p.process(pssmsg, isRaw, isProx); err != nil { qerr := p.enqueue(pssmsg) if qerr != nil { return fmt.Errorf("process fail: processerr %v, queueerr: %v", err, qerr) @@ -384,7 +431,7 @@ func (p *Pss) handlePssMsg(ctx context.Context, msg interface{}) error { // Entry point to processing a message for which the current node can be the intended recipient. // Attempts symmetric and asymmetric decryption with stored keys. // Dispatches message to all handlers matching the message topic -func (p *Pss) process(pssmsg *PssMsg) error { +func (p *Pss) process(pssmsg *PssMsg, raw bool, prox bool) error { metrics.GetOrRegisterCounter("pss.process", nil).Inc(1) var err error @@ -397,10 +444,8 @@ func (p *Pss) process(pssmsg *PssMsg) error { envelope := pssmsg.Payload psstopic := Topic(envelope.Topic) - if pssmsg.isRaw() { - if !p.allowRaw { - return errors.New("raw message support disabled") - } + + if raw { payload = pssmsg.Payload.Data } else { if pssmsg.isSym() { @@ -422,19 +467,27 @@ func (p *Pss) process(pssmsg *PssMsg) error { return err } } - p.executeHandlers(psstopic, payload, from, asymmetric, keyid) + p.executeHandlers(psstopic, payload, from, raw, prox, asymmetric, keyid) return nil } -func (p *Pss) executeHandlers(topic Topic, payload []byte, from *PssAddress, asymmetric bool, keyid string) { +func (p *Pss) executeHandlers(topic Topic, payload []byte, from *PssAddress, raw bool, prox bool, asymmetric bool, keyid string) { handlers := p.getHandlers(topic) peer := p2p.NewPeer(enode.ID{}, fmt.Sprintf("%x", from), []p2p.Cap{}) - for f := range handlers { - err := (*f)(payload, peer, asymmetric, keyid) + for h := range handlers { + if !h.caps.raw && raw { + log.Warn("norawhandler") + continue + } + if !h.caps.prox && prox { + log.Warn("noproxhandler") + continue + } + err := (h.f)(payload, peer, asymmetric, keyid) if err != nil { - log.Warn("Pss handler %p failed: %v", f, err) + log.Warn("Pss handler failed", "err", err) } } } @@ -445,9 +498,23 @@ func (p *Pss) isSelfRecipient(msg *PssMsg) bool { } // test match of leftmost bytes in given message to node's Kademlia address -func (p *Pss) isSelfPossibleRecipient(msg *PssMsg) bool { +func (p *Pss) isSelfPossibleRecipient(msg *PssMsg, prox bool) bool { local := p.Kademlia.BaseAddr() - return bytes.Equal(msg.To, local[:len(msg.To)]) + + // if a partial address matches we are possible recipient regardless of prox + // if not and prox is not set, we are surely not + if bytes.Equal(msg.To, local[:len(msg.To)]) { + + return true + } else if !prox { + return false + } + + depth := p.Kademlia.NeighbourhoodDepth() + po, _ := p.Kademlia.Pof(p.Kademlia.BaseAddr(), msg.To, 0) + log.Trace("selfpossible", "po", po, "depth", depth) + + return depth <= po } ///////////////////////////////////////////////////////////////////// @@ -684,9 +751,6 @@ func (p *Pss) enqueue(msg *PssMsg) error { // // Will fail if raw messages are disallowed func (p *Pss) SendRaw(address PssAddress, topic Topic, msg []byte) error { - if !p.allowRaw { - return errors.New("Raw messages not enabled") - } pssMsgParams := &msgParams{ raw: true, } @@ -699,7 +763,17 @@ func (p *Pss) SendRaw(address PssAddress, topic Topic, msg []byte) error { pssMsg.Expire = uint32(time.Now().Add(p.msgTTL).Unix()) pssMsg.Payload = payload p.addFwdCache(pssMsg) - return p.enqueue(pssMsg) + err := p.enqueue(pssMsg) + if err != nil { + return err + } + + // if we have a proxhandler on this topic + // also deliver message to ourselves + if p.isSelfPossibleRecipient(pssMsg, true) && p.topicHandlerCaps[topic].prox { + return p.process(pssMsg, true, true) + } + return nil } // Send a message using symmetric encryption @@ -800,7 +874,16 @@ func (p *Pss) send(to []byte, topic Topic, msg []byte, asymmetric bool, key []by pssMsg.To = to pssMsg.Expire = uint32(time.Now().Add(p.msgTTL).Unix()) pssMsg.Payload = envelope - return p.enqueue(pssMsg) + err = p.enqueue(pssMsg) + if err != nil { + return err + } + if _, ok := p.topicHandlerCaps[topic]; ok { + if p.isSelfPossibleRecipient(pssMsg, true) && p.topicHandlerCaps[topic].prox { + return p.process(pssMsg, true, true) + } + } + return nil } // Forwards a pss message to the peer(s) closest to the to recipient address in the PssMsg struct @@ -895,6 +978,10 @@ func (p *Pss) cleanFwdCache() { } } +func label(b []byte) string { + return fmt.Sprintf("%04x", b[:2]) +} + // add a message to the cache func (p *Pss) addFwdCache(msg *PssMsg) error { metrics.GetOrRegisterCounter("pss.addfwdcache", nil).Inc(1) @@ -934,10 +1021,14 @@ func (p *Pss) checkFwdCache(msg *PssMsg) bool { // Digest of message func (p *Pss) digest(msg *PssMsg) pssDigest { - hasher := p.hashPool.Get().(storage.SwarmHash) + return p.digestBytes(msg.serialize()) +} + +func (p *Pss) digestBytes(msg []byte) pssDigest { + hasher := p.hashPool.Get().(hash.Hash) defer p.hashPool.Put(hasher) hasher.Reset() - hasher.Write(msg.serialize()) + hasher.Write(msg) digest := pssDigest{} key := hasher.Sum(nil) copy(digest[:], key[:digestLength]) diff --git a/swarm/pss/pss_test.go b/swarm/pss/pss_test.go index 66a90be62..32404aaaf 100644 --- a/swarm/pss/pss_test.go +++ b/swarm/pss/pss_test.go @@ -48,20 +48,23 @@ import ( "github.com/ethereum/go-ethereum/p2p/simulations/adapters" "github.com/ethereum/go-ethereum/rpc" "github.com/ethereum/go-ethereum/swarm/network" + "github.com/ethereum/go-ethereum/swarm/pot" "github.com/ethereum/go-ethereum/swarm/state" whisper "github.com/ethereum/go-ethereum/whisper/whisperv5" ) var ( - initOnce = sync.Once{} - debugdebugflag = flag.Bool("vv", false, "veryverbose") - debugflag = flag.Bool("v", false, "verbose") - longrunning = flag.Bool("longrunning", false, "do run long-running tests") - w *whisper.Whisper - wapi *whisper.PublicWhisperAPI - psslogmain log.Logger - pssprotocols map[string]*protoCtrl - useHandshake bool + initOnce = sync.Once{} + loglevel = flag.Int("loglevel", 2, "logging verbosity") + longrunning = flag.Bool("longrunning", false, "do run long-running tests") + w *whisper.Whisper + wapi *whisper.PublicWhisperAPI + psslogmain log.Logger + pssprotocols map[string]*protoCtrl + useHandshake bool + noopHandlerFunc = func(msg []byte, p *p2p.Peer, asymmetric bool, keyid string) error { + return nil + } ) func init() { @@ -75,16 +78,9 @@ func init() { func initTest() { initOnce.Do( func() { - loglevel := log.LvlInfo - if *debugflag { - loglevel = log.LvlDebug - } else if *debugdebugflag { - loglevel = log.LvlTrace - } - psslogmain = log.New("psslog", "*") hs := log.StreamHandler(os.Stderr, log.TerminalFormat(true)) - hf := log.LvlFilterHandler(loglevel, hs) + hf := log.LvlFilterHandler(log.Lvl(*loglevel), hs) h := log.CallerFileHandler(hf) log.Root().SetHandler(h) @@ -280,15 +276,14 @@ func TestAddressMatch(t *testing.T) { } pssmsg := &PssMsg{ - To: remoteaddr, - Payload: &whisper.Envelope{}, + To: remoteaddr, } // differ from first byte if ps.isSelfRecipient(pssmsg) { t.Fatalf("isSelfRecipient true but %x != %x", remoteaddr, localaddr) } - if ps.isSelfPossibleRecipient(pssmsg) { + if ps.isSelfPossibleRecipient(pssmsg, false) { t.Fatalf("isSelfPossibleRecipient true but %x != %x", remoteaddr[:8], localaddr[:8]) } @@ -297,7 +292,7 @@ func TestAddressMatch(t *testing.T) { if ps.isSelfRecipient(pssmsg) { t.Fatalf("isSelfRecipient true but %x != %x", remoteaddr, localaddr) } - if !ps.isSelfPossibleRecipient(pssmsg) { + if !ps.isSelfPossibleRecipient(pssmsg, false) { t.Fatalf("isSelfPossibleRecipient false but %x == %x", remoteaddr[:8], localaddr[:8]) } @@ -306,13 +301,342 @@ func TestAddressMatch(t *testing.T) { if !ps.isSelfRecipient(pssmsg) { t.Fatalf("isSelfRecipient false but %x == %x", remoteaddr, localaddr) } - if !ps.isSelfPossibleRecipient(pssmsg) { + if !ps.isSelfPossibleRecipient(pssmsg, false) { t.Fatalf("isSelfPossibleRecipient false but %x == %x", remoteaddr[:8], localaddr[:8]) } + +} + +// test that message is handled by sender if a prox handler exists and sender is in prox of message +func TestProxShortCircuit(t *testing.T) { + + // sender node address + localAddr := network.RandomAddr().Over() + localPotAddr := pot.NewAddressFromBytes(localAddr) + + // set up kademlia + kadParams := network.NewKadParams() + kad := network.NewKademlia(localAddr, kadParams) + peerCount := kad.MinBinSize + 1 + + // set up pss + privKey, err := crypto.GenerateKey() + pssp := NewPssParams().WithPrivateKey(privKey) + ps, err := NewPss(kad, pssp) + if err != nil { + t.Fatal(err.Error()) + } + + // create kademlia peers, so we have peers both inside and outside minproxlimit + var peers []*network.Peer + proxMessageAddress := pot.RandomAddressAt(localPotAddr, peerCount).Bytes() + distantMessageAddress := pot.RandomAddressAt(localPotAddr, 0).Bytes() + + for i := 0; i < peerCount; i++ { + rw := &p2p.MsgPipeRW{} + ptpPeer := p2p.NewPeer(enode.ID{}, "wanna be with me? [ ] yes [ ] no", []p2p.Cap{}) + protoPeer := protocols.NewPeer(ptpPeer, rw, &protocols.Spec{}) + peerAddr := pot.RandomAddressAt(localPotAddr, i) + bzzPeer := &network.BzzPeer{ + Peer: protoPeer, + BzzAddr: &network.BzzAddr{ + OAddr: peerAddr.Bytes(), + UAddr: []byte(fmt.Sprintf("%x", peerAddr[:])), + }, + } + peer := network.NewPeer(bzzPeer, kad) + kad.On(peer) + peers = append(peers, peer) + } + + // register it marking prox capability + delivered := make(chan struct{}) + rawHandlerFunc := func(msg []byte, p *p2p.Peer, asymmetric bool, keyid string) error { + log.Trace("in allowraw handler") + delivered <- struct{}{} + return nil + } + topic := BytesToTopic([]byte{0x2a}) + hndlrProxDereg := ps.Register(&topic, &handler{ + f: rawHandlerFunc, + caps: &handlerCaps{ + raw: true, + prox: true, + }, + }) + defer hndlrProxDereg() + + // send message too far away for sender to be in prox + // reception of this message should time out + errC := make(chan error) + go func() { + err := ps.SendRaw(distantMessageAddress, topic, []byte("foo")) + if err != nil { + errC <- err + } + }() + + ctx, cancel := context.WithTimeout(context.TODO(), time.Second) + defer cancel() + select { + case <-delivered: + t.Fatal("raw distant message delivered") + case err := <-errC: + t.Fatal(err) + case <-ctx.Done(): + } + + // send message that should be within sender prox + // this message should be delivered + go func() { + err := ps.SendRaw(proxMessageAddress, topic, []byte("bar")) + if err != nil { + errC <- err + } + }() + + ctx, cancel = context.WithTimeout(context.TODO(), time.Second) + defer cancel() + select { + case <-delivered: + case err := <-errC: + t.Fatal(err) + case <-ctx.Done(): + t.Fatal("raw timeout") + } + + // try the same prox message with sym and asym send + proxAddrPss := PssAddress(proxMessageAddress) + symKeyId, err := ps.GenerateSymmetricKey(topic, &proxAddrPss, true) + go func() { + err := ps.SendSym(symKeyId, topic, []byte("baz")) + if err != nil { + errC <- err + } + }() + ctx, cancel = context.WithTimeout(context.TODO(), time.Second) + defer cancel() + select { + case <-delivered: + case err := <-errC: + t.Fatal(err) + case <-ctx.Done(): + t.Fatal("sym timeout") + } + + err = ps.SetPeerPublicKey(&privKey.PublicKey, topic, &proxAddrPss) + if err != nil { + t.Fatal(err) + } + pubKeyId := hexutil.Encode(crypto.FromECDSAPub(&privKey.PublicKey)) + go func() { + err := ps.SendAsym(pubKeyId, topic, []byte("xyzzy")) + if err != nil { + errC <- err + } + }() + ctx, cancel = context.WithTimeout(context.TODO(), time.Second) + defer cancel() + select { + case <-delivered: + case err := <-errC: + t.Fatal(err) + case <-ctx.Done(): + t.Fatal("asym timeout") + } } -// -func TestHandlerConditions(t *testing.T) { +// verify that node can be set as recipient regardless of explicit message address match if minimum one handler of a topic is explicitly set to allow it +// note that in these tests we use the raw capability on handlers for convenience +func TestAddressMatchProx(t *testing.T) { + + // recipient node address + localAddr := network.RandomAddr().Over() + localPotAddr := pot.NewAddressFromBytes(localAddr) + + // set up kademlia + kadparams := network.NewKadParams() + kad := network.NewKademlia(localAddr, kadparams) + nnPeerCount := kad.MinBinSize + peerCount := nnPeerCount + 2 + + // set up pss + privKey, err := crypto.GenerateKey() + pssp := NewPssParams().WithPrivateKey(privKey) + ps, err := NewPss(kad, pssp) + if err != nil { + t.Fatal(err.Error()) + } + + // create kademlia peers, so we have peers both inside and outside minproxlimit + var peers []*network.Peer + for i := 0; i < peerCount; i++ { + rw := &p2p.MsgPipeRW{} + ptpPeer := p2p.NewPeer(enode.ID{}, "362436 call me anytime", []p2p.Cap{}) + protoPeer := protocols.NewPeer(ptpPeer, rw, &protocols.Spec{}) + peerAddr := pot.RandomAddressAt(localPotAddr, i) + bzzPeer := &network.BzzPeer{ + Peer: protoPeer, + BzzAddr: &network.BzzAddr{ + OAddr: peerAddr.Bytes(), + UAddr: []byte(fmt.Sprintf("%x", peerAddr[:])), + }, + } + peer := network.NewPeer(bzzPeer, kad) + kad.On(peer) + peers = append(peers, peer) + } + + // TODO: create a test in the network package to make a table with n peers where n-m are proxpeers + // meanwhile test regression for kademlia since we are compiling the test parameters from different packages + var proxes int + var conns int + kad.EachConn(nil, peerCount, func(p *network.Peer, po int, prox bool) bool { + conns++ + if prox { + proxes++ + } + log.Trace("kadconn", "po", po, "peer", p, "prox", prox) + return true + }) + if proxes != nnPeerCount { + t.Fatalf("expected %d proxpeers, have %d", nnPeerCount, proxes) + } else if conns != peerCount { + t.Fatalf("expected %d peers total, have %d", peerCount, proxes) + } + + // remote address distances from localAddr to try and the expected outcomes if we use prox handler + remoteDistances := []int{ + 255, + nnPeerCount + 1, + nnPeerCount, + nnPeerCount - 1, + 0, + } + expects := []bool{ + true, + true, + true, + false, + false, + } + + // first the unit test on the method that calculates possible receipient using prox + for i, distance := range remoteDistances { + pssMsg := newPssMsg(&msgParams{}) + pssMsg.To = make([]byte, len(localAddr)) + copy(pssMsg.To, localAddr) + var byteIdx = distance / 8 + pssMsg.To[byteIdx] ^= 1 << uint(7-(distance%8)) + log.Trace(fmt.Sprintf("addrmatch %v", bytes.Equal(pssMsg.To, localAddr))) + if ps.isSelfPossibleRecipient(pssMsg, true) != expects[i] { + t.Fatalf("expected distance %d to be %v", distance, expects[i]) + } + } + + // we move up to higher level and test the actual message handler + // for each distance check if we are possible recipient when prox variant is used is set + + // this handler will increment a counter for every message that gets passed to the handler + var receives int + rawHandlerFunc := func(msg []byte, p *p2p.Peer, asymmetric bool, keyid string) error { + log.Trace("in allowraw handler") + receives++ + return nil + } + + // register it marking prox capability + topic := BytesToTopic([]byte{0x2a}) + hndlrProxDereg := ps.Register(&topic, &handler{ + f: rawHandlerFunc, + caps: &handlerCaps{ + raw: true, + prox: true, + }, + }) + + // test the distances + var prevReceive int + for i, distance := range remoteDistances { + remotePotAddr := pot.RandomAddressAt(localPotAddr, distance) + remoteAddr := remotePotAddr.Bytes() + + var data [32]byte + rand.Read(data[:]) + pssMsg := newPssMsg(&msgParams{raw: true}) + pssMsg.To = remoteAddr + pssMsg.Expire = uint32(time.Now().Unix() + 4200) + pssMsg.Payload = &whisper.Envelope{ + Topic: whisper.TopicType(topic), + Data: data[:], + } + + log.Trace("withprox addrs", "local", localAddr, "remote", remoteAddr) + ps.handlePssMsg(context.TODO(), pssMsg) + if (!expects[i] && prevReceive != receives) || (expects[i] && prevReceive == receives) { + t.Fatalf("expected distance %d recipient %v when prox is set for handler", distance, expects[i]) + } + prevReceive = receives + } + + // now add a non prox-capable handler and test + ps.Register(&topic, &handler{ + f: rawHandlerFunc, + caps: &handlerCaps{ + raw: true, + }, + }) + receives = 0 + prevReceive = 0 + for i, distance := range remoteDistances { + remotePotAddr := pot.RandomAddressAt(localPotAddr, distance) + remoteAddr := remotePotAddr.Bytes() + + var data [32]byte + rand.Read(data[:]) + pssMsg := newPssMsg(&msgParams{raw: true}) + pssMsg.To = remoteAddr + pssMsg.Expire = uint32(time.Now().Unix() + 4200) + pssMsg.Payload = &whisper.Envelope{ + Topic: whisper.TopicType(topic), + Data: data[:], + } + + log.Trace("withprox addrs", "local", localAddr, "remote", remoteAddr) + ps.handlePssMsg(context.TODO(), pssMsg) + if (!expects[i] && prevReceive != receives) || (expects[i] && prevReceive == receives) { + t.Fatalf("expected distance %d recipient %v when prox is set for handler", distance, expects[i]) + } + prevReceive = receives + } + + // now deregister the prox capable handler, now none of the messages will be handled + hndlrProxDereg() + receives = 0 + + for _, distance := range remoteDistances { + remotePotAddr := pot.RandomAddressAt(localPotAddr, distance) + remoteAddr := remotePotAddr.Bytes() + + pssMsg := newPssMsg(&msgParams{raw: true}) + pssMsg.To = remoteAddr + pssMsg.Expire = uint32(time.Now().Unix() + 4200) + pssMsg.Payload = &whisper.Envelope{ + Topic: whisper.TopicType(topic), + Data: []byte(remotePotAddr.String()), + } + + log.Trace("noprox addrs", "local", localAddr, "remote", remoteAddr) + ps.handlePssMsg(context.TODO(), pssMsg) + if receives != 0 { + t.Fatalf("expected distance %d to not be recipient when prox is not set for handler", distance) + } + + } +} + +// verify that message queueing happens when it should, and that expired and corrupt messages are dropped +func TestMessageProcessing(t *testing.T) { t.Skip("Disabled due to probable faulty logic for outbox expectations") // setup @@ -326,13 +650,12 @@ func TestHandlerConditions(t *testing.T) { ps := newTestPss(privkey, network.NewKademlia(addr, network.NewKadParams()), NewPssParams()) // message should pass - msg := &PssMsg{ - To: addr, - Expire: uint32(time.Now().Add(time.Second * 60).Unix()), - Payload: &whisper.Envelope{ - Topic: [4]byte{}, - Data: []byte{0x66, 0x6f, 0x6f}, - }, + msg := newPssMsg(&msgParams{}) + msg.To = addr + msg.Expire = uint32(time.Now().Add(time.Second * 60).Unix()) + msg.Payload = &whisper.Envelope{ + Topic: [4]byte{}, + Data: []byte{0x66, 0x6f, 0x6f}, } if err := ps.handlePssMsg(context.TODO(), msg); err != nil { t.Fatal(err.Error()) @@ -498,6 +821,7 @@ func TestKeys(t *testing.T) { } } +// check that we can retrieve previously added public key entires per topic and peer func TestGetPublickeyEntries(t *testing.T) { privkey, err := crypto.GenerateKey() @@ -557,7 +881,7 @@ OUTER: } // forwarding should skip peers that do not have matching pss capabilities -func TestMismatch(t *testing.T) { +func TestPeerCapabilityMismatch(t *testing.T) { // create privkey for forwarder node privkey, err := crypto.GenerateKey() @@ -615,6 +939,76 @@ func TestMismatch(t *testing.T) { } +// verifies that message handlers for raw messages only are invoked when minimum one handler for the topic exists in which raw messages are explicitly allowed +func TestRawAllow(t *testing.T) { + + // set up pss like so many times before + privKey, err := crypto.GenerateKey() + if err != nil { + t.Fatal(err) + } + baseAddr := network.RandomAddr() + kad := network.NewKademlia((baseAddr).Over(), network.NewKadParams()) + ps := newTestPss(privKey, kad, nil) + topic := BytesToTopic([]byte{0x2a}) + + // create handler innards that increments every time a message hits it + var receives int + rawHandlerFunc := func(msg []byte, p *p2p.Peer, asymmetric bool, keyid string) error { + log.Trace("in allowraw handler") + receives++ + return nil + } + + // wrap this handler function with a handler without raw capability and register it + hndlrNoRaw := &handler{ + f: rawHandlerFunc, + } + ps.Register(&topic, hndlrNoRaw) + + // test it with a raw message, should be poo-poo + pssMsg := newPssMsg(&msgParams{ + raw: true, + }) + pssMsg.To = baseAddr.OAddr + pssMsg.Expire = uint32(time.Now().Unix() + 4200) + pssMsg.Payload = &whisper.Envelope{ + Topic: whisper.TopicType(topic), + } + ps.handlePssMsg(context.TODO(), pssMsg) + if receives > 0 { + t.Fatalf("Expected handler not to be executed with raw cap off") + } + + // now wrap the same handler function with raw capabilities and register it + hndlrRaw := &handler{ + f: rawHandlerFunc, + caps: &handlerCaps{ + raw: true, + }, + } + deregRawHandler := ps.Register(&topic, hndlrRaw) + + // should work now + pssMsg.Payload.Data = []byte("Raw Deal") + ps.handlePssMsg(context.TODO(), pssMsg) + if receives == 0 { + t.Fatalf("Expected handler to be executed with raw cap on") + } + + // now deregister the raw capable handler + prevReceives := receives + deregRawHandler() + + // check that raw messages fail again + pssMsg.Payload.Data = []byte("Raw Trump") + ps.handlePssMsg(context.TODO(), pssMsg) + if receives != prevReceives { + t.Fatalf("Expected handler not to be executed when raw handler is retracted") + } +} + +// verifies that nodes can send and receive raw (verbatim) messages func TestSendRaw(t *testing.T) { t.Run("32", testSendRaw) t.Run("8", testSendRaw) @@ -658,13 +1052,13 @@ func testSendRaw(t *testing.T) { lmsgC := make(chan APIMsg) lctx, lcancel := context.WithTimeout(context.Background(), time.Second*10) defer lcancel() - lsub, err := clients[0].Subscribe(lctx, "pss", lmsgC, "receive", topic) + lsub, err := clients[0].Subscribe(lctx, "pss", lmsgC, "receive", topic, true, false) log.Trace("lsub", "id", lsub) defer lsub.Unsubscribe() rmsgC := make(chan APIMsg) rctx, rcancel := context.WithTimeout(context.Background(), time.Second*10) defer rcancel() - rsub, err := clients[1].Subscribe(rctx, "pss", rmsgC, "receive", topic) + rsub, err := clients[1].Subscribe(rctx, "pss", rmsgC, "receive", topic, true, false) log.Trace("rsub", "id", rsub) defer rsub.Unsubscribe() @@ -757,13 +1151,13 @@ func testSendSym(t *testing.T) { lmsgC := make(chan APIMsg) lctx, lcancel := context.WithTimeout(context.Background(), time.Second*10) defer lcancel() - lsub, err := clients[0].Subscribe(lctx, "pss", lmsgC, "receive", topic) + lsub, err := clients[0].Subscribe(lctx, "pss", lmsgC, "receive", topic, false, false) log.Trace("lsub", "id", lsub) defer lsub.Unsubscribe() rmsgC := make(chan APIMsg) rctx, rcancel := context.WithTimeout(context.Background(), time.Second*10) defer rcancel() - rsub, err := clients[1].Subscribe(rctx, "pss", rmsgC, "receive", topic) + rsub, err := clients[1].Subscribe(rctx, "pss", rmsgC, "receive", topic, false, false) log.Trace("rsub", "id", rsub) defer rsub.Unsubscribe() @@ -872,13 +1266,13 @@ func testSendAsym(t *testing.T) { lmsgC := make(chan APIMsg) lctx, lcancel := context.WithTimeout(context.Background(), time.Second*10) defer lcancel() - lsub, err := clients[0].Subscribe(lctx, "pss", lmsgC, "receive", topic) + lsub, err := clients[0].Subscribe(lctx, "pss", lmsgC, "receive", topic, false, false) log.Trace("lsub", "id", lsub) defer lsub.Unsubscribe() rmsgC := make(chan APIMsg) rctx, rcancel := context.WithTimeout(context.Background(), time.Second*10) defer rcancel() - rsub, err := clients[1].Subscribe(rctx, "pss", rmsgC, "receive", topic) + rsub, err := clients[1].Subscribe(rctx, "pss", rmsgC, "receive", topic, false, false) log.Trace("rsub", "id", rsub) defer rsub.Unsubscribe() @@ -1037,7 +1431,7 @@ func testNetwork(t *testing.T) { msgC := make(chan APIMsg) ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) defer cancel() - sub, err := rpcclient.Subscribe(ctx, "pss", msgC, "receive", topic) + sub, err := rpcclient.Subscribe(ctx, "pss", msgC, "receive", topic, false, false) if err != nil { t.Fatal(err) } @@ -1209,7 +1603,7 @@ func TestDeduplication(t *testing.T) { rmsgC := make(chan APIMsg) rctx, cancel := context.WithTimeout(context.Background(), time.Second*1) defer cancel() - rsub, err := clients[1].Subscribe(rctx, "pss", rmsgC, "receive", topic) + rsub, err := clients[1].Subscribe(rctx, "pss", rmsgC, "receive", topic, false, false) log.Trace("rsub", "id", rsub) defer rsub.Unsubscribe() @@ -1392,8 +1786,8 @@ func benchmarkSymkeyBruteforceChangeaddr(b *testing.B) { if err != nil { b.Fatalf("could not generate whisper envelope: %v", err) } - ps.Register(&topic, func(msg []byte, p *p2p.Peer, asymmetric bool, keyid string) error { - return nil + ps.Register(&topic, &handler{ + f: noopHandlerFunc, }) pssmsgs = append(pssmsgs, &PssMsg{ To: to, @@ -1402,7 +1796,7 @@ func benchmarkSymkeyBruteforceChangeaddr(b *testing.B) { } b.ResetTimer() for i := 0; i < b.N; i++ { - if err := ps.process(pssmsgs[len(pssmsgs)-(i%len(pssmsgs))-1]); err != nil { + if err := ps.process(pssmsgs[len(pssmsgs)-(i%len(pssmsgs))-1], false, false); err != nil { b.Fatalf("pss processing failed: %v", err) } } @@ -1476,15 +1870,15 @@ func benchmarkSymkeyBruteforceSameaddr(b *testing.B) { if err != nil { b.Fatalf("could not generate whisper envelope: %v", err) } - ps.Register(&topic, func(msg []byte, p *p2p.Peer, asymmetric bool, keyid string) error { - return nil + ps.Register(&topic, &handler{ + f: noopHandlerFunc, }) pssmsg := &PssMsg{ To: addr[len(addr)-1][:], Payload: env, } for i := 0; i < b.N; i++ { - if err := ps.process(pssmsg); err != nil { + if err := ps.process(pssmsg, false, false); err != nil { b.Fatalf("pss processing failed: %v", err) } } @@ -1581,7 +1975,12 @@ func newServices(allowRaw bool) adapters.Services { if useHandshake { SetHandshakeController(ps, NewHandshakeParams()) } - ps.Register(&PingTopic, pp.Handle) + ps.Register(&PingTopic, &handler{ + f: pp.Handle, + caps: &handlerCaps{ + raw: true, + }, + }) ps.addAPI(rpc.API{ Namespace: "psstest", Version: "0.3", diff --git a/swarm/pss/types.go b/swarm/pss/types.go index 56c2c51dc..ba963067c 100644 --- a/swarm/pss/types.go +++ b/swarm/pss/types.go @@ -159,9 +159,39 @@ func (msg *PssMsg) String() string { } // Signature for a message handler function for a PssMsg -// // Implementations of this type are passed to Pss.Register together with a topic, -type Handler func(msg []byte, p *p2p.Peer, asymmetric bool, keyid string) error +type HandlerFunc func(msg []byte, p *p2p.Peer, asymmetric bool, keyid string) error + +type handlerCaps struct { + raw bool + prox bool +} + +// Handler defines code to be executed upon reception of content. +type handler struct { + f HandlerFunc + caps *handlerCaps +} + +// NewHandler returns a new message handler +func NewHandler(f HandlerFunc) *handler { + return &handler{ + f: f, + caps: &handlerCaps{}, + } +} + +// WithRaw is a chainable method that allows raw messages to be handled. +func (h *handler) WithRaw() *handler { + h.caps.raw = true + return h +} + +// WithProxBin is a chainable method that allows sending messages with full addresses to neighbourhoods using the kademlia depth as reference +func (h *handler) WithProxBin() *handler { + h.caps.prox = true + return h +} // the stateStore handles saving and loading PSS peers and their corresponding keys // it is currently unimplemented -- 2.45.2 From 0699287440527dedbc7d6f881fc466057b7822bc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Pawe=C5=82=20Bylica?= Date: Mon, 26 Nov 2018 16:09:32 +0100 Subject: [PATCH 19/48] tests: Add flag to use EVMC for state tests (#18084) --- tests/state_test.go | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/tests/state_test.go b/tests/state_test.go index ad77e4f33..964405382 100644 --- a/tests/state_test.go +++ b/tests/state_test.go @@ -18,10 +18,12 @@ package tests import ( "bytes" + "flag" "fmt" "reflect" "testing" + "github.com/ethereum/go-ethereum/cmd/utils" "github.com/ethereum/go-ethereum/core/vm" ) @@ -65,8 +67,17 @@ func TestState(t *testing.T) { // Transactions with gasLimit above this value will not get a VM trace on failure. const traceErrorLimit = 400000 +// The VM config for state tests that accepts --vm.* command line arguments. +var testVMConfig = func() vm.Config { + vmconfig := vm.Config{} + flag.StringVar(&vmconfig.EVMInterpreter, utils.EVMInterpreterFlag.Name, utils.EVMInterpreterFlag.Value, utils.EVMInterpreterFlag.Usage) + flag.StringVar(&vmconfig.EWASMInterpreter, utils.EWASMInterpreterFlag.Name, utils.EWASMInterpreterFlag.Value, utils.EWASMInterpreterFlag.Usage) + flag.Parse() + return vmconfig +}() + func withTrace(t *testing.T, gasLimit uint64, test func(vm.Config) error) { - err := test(vm.Config{}) + err := test(testVMConfig) if err == nil { return } -- 2.45.2 From 2714e8f091117b4f110198008348bfc19233ed60 Mon Sep 17 00:00:00 2001 From: Javier Peletier Date: Mon, 26 Nov 2018 16:10:22 +0100 Subject: [PATCH 20/48] Remove multihash from Swarm bzz:// for Feeds (#18175) --- cmd/swarm/swarm-smoke/feed_upload_and_sync.go | 6 +- swarm/OWNERS | 1 - swarm/api/api.go | 25 ++-- swarm/api/client/client_test.go | 112 +++++++++++++----- swarm/api/http/server_test.go | 104 ++++++++++------ swarm/multihash/multihash.go | 92 -------------- swarm/multihash/multihash_test.go | 53 --------- 7 files changed, 167 insertions(+), 226 deletions(-) delete mode 100644 swarm/multihash/multihash.go delete mode 100644 swarm/multihash/multihash_test.go diff --git a/cmd/swarm/swarm-smoke/feed_upload_and_sync.go b/cmd/swarm/swarm-smoke/feed_upload_and_sync.go index 1371d6654..0328d656f 100644 --- a/cmd/swarm/swarm-smoke/feed_upload_and_sync.go +++ b/cmd/swarm/swarm-smoke/feed_upload_and_sync.go @@ -16,7 +16,6 @@ import ( "github.com/ethereum/go-ethereum/common/hexutil" "github.com/ethereum/go-ethereum/crypto" "github.com/ethereum/go-ethereum/log" - "github.com/ethereum/go-ethereum/swarm/multihash" "github.com/ethereum/go-ethereum/swarm/storage/feed" colorable "github.com/mattn/go-colorable" "github.com/pborman/uuid" @@ -36,7 +35,7 @@ func cliFeedUploadAndSync(c *cli.Context) error { generateEndpoints(scheme, cluster, from, to) - log.Info("generating and uploading MRUs to " + endpoints[0] + " and syncing") + log.Info("generating and uploading feeds to " + endpoints[0] + " and syncing") // create a random private key to sign updates with and derive the address pkFile, err := ioutil.TempFile("", "swarm-feed-smoke-test") @@ -218,8 +217,7 @@ func cliFeedUploadAndSync(c *cli.Context) error { if err != nil { return err } - multihashHex := hexutil.Encode(multihash.ToMultihash(hashBytes)) - + multihashHex := hexutil.Encode(hashBytes) fileHash, err := digest(f) if err != nil { return err diff --git a/swarm/OWNERS b/swarm/OWNERS index d4204e08c..4b9ca96eb 100644 --- a/swarm/OWNERS +++ b/swarm/OWNERS @@ -7,7 +7,6 @@ swarm ├── fuse ────────────────── @jmozah, @holisticode ├── grafana_dashboards ──── @nonsense ├── metrics ─────────────── @nonsense, @holisticode -├── multihash ───────────── @nolash ├── network ─────────────── ethersphere │ ├── bitvector ───────── @zelig, @janos, @gbalint │ ├── priorityqueue ───── @zelig, @janos, @gbalint diff --git a/swarm/api/api.go b/swarm/api/api.go index 099a46939..33a8e3539 100644 --- a/swarm/api/api.go +++ b/swarm/api/api.go @@ -42,7 +42,6 @@ import ( "github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/metrics" "github.com/ethereum/go-ethereum/swarm/log" - "github.com/ethereum/go-ethereum/swarm/multihash" "github.com/ethereum/go-ethereum/swarm/spancontext" "github.com/ethereum/go-ethereum/swarm/storage" "github.com/ethereum/go-ethereum/swarm/storage/feed" @@ -417,7 +416,7 @@ func (a *API) Get(ctx context.Context, decrypt DecryptFunc, manifestAddr storage return reader, mimeType, status, nil, err } // get the data of the update - _, rsrcData, err := a.feed.GetContent(entry.Feed) + _, contentAddr, err := a.feed.GetContent(entry.Feed) if err != nil { apiGetNotFound.Inc(1) status = http.StatusNotFound @@ -425,23 +424,23 @@ func (a *API) Get(ctx context.Context, decrypt DecryptFunc, manifestAddr storage return reader, mimeType, status, nil, err } - // extract multihash - decodedMultihash, err := multihash.FromMultihash(rsrcData) - if err != nil { + // extract content hash + if len(contentAddr) != storage.AddressLength { apiGetInvalid.Inc(1) status = http.StatusUnprocessableEntity - log.Warn("invalid multihash in feed update", "err", err) - return reader, mimeType, status, nil, err + errorMessage := fmt.Sprintf("invalid swarm hash in feed update. Expected %d bytes. Got %d", storage.AddressLength, len(contentAddr)) + log.Warn(errorMessage) + return reader, mimeType, status, nil, errors.New(errorMessage) } - manifestAddr = storage.Address(decodedMultihash) - log.Trace("feed update contains multihash", "key", manifestAddr) + manifestAddr = storage.Address(contentAddr) + log.Trace("feed update contains swarm hash", "key", manifestAddr) - // get the manifest the multihash digest points to + // get the manifest the swarm hash points to trie, err := loadManifest(ctx, a.fileStore, manifestAddr, nil, NOOPDecrypt) if err != nil { apiGetNotFound.Inc(1) status = http.StatusNotFound - log.Warn(fmt.Sprintf("loadManifestTrie (feed update multihash) error: %v", err)) + log.Warn(fmt.Sprintf("loadManifestTrie (feed update) error: %v", err)) return reader, mimeType, status, nil, err } @@ -451,8 +450,8 @@ func (a *API) Get(ctx context.Context, decrypt DecryptFunc, manifestAddr storage if entry == nil { status = http.StatusNotFound apiGetNotFound.Inc(1) - err = fmt.Errorf("manifest (feed update multihash) entry for '%s' not found", path) - log.Trace("manifest (feed update multihash) entry not found", "key", manifestAddr, "path", path) + err = fmt.Errorf("manifest (feed update) entry for '%s' not found", path) + log.Trace("manifest (feed update) entry not found", "key", manifestAddr, "path", path) return reader, mimeType, status, nil, err } } diff --git a/swarm/api/client/client_test.go b/swarm/api/client/client_test.go index 76b349397..39f6e4797 100644 --- a/swarm/api/client/client_test.go +++ b/swarm/api/client/client_test.go @@ -25,13 +25,13 @@ import ( "sort" "testing" + "github.com/ethereum/go-ethereum/swarm/storage" "github.com/ethereum/go-ethereum/swarm/storage/feed/lookup" "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/crypto" "github.com/ethereum/go-ethereum/swarm/api" swarmhttp "github.com/ethereum/go-ethereum/swarm/api/http" - "github.com/ethereum/go-ethereum/swarm/multihash" "github.com/ethereum/go-ethereum/swarm/storage/feed" ) @@ -368,58 +368,99 @@ func newTestSigner() (*feed.GenericSigner, error) { return feed.NewGenericSigner(privKey), nil } -// test the transparent resolving of multihash feed updates with bzz:// scheme +// Test the transparent resolving of feed updates with bzz:// scheme // -// first upload data, and store the multihash to the resulting manifest in a feed update -// retrieving the update with the multihash should return the manifest pointing directly to the data +// First upload data to bzz:, and store the Swarm hash to the resulting manifest in a feed update. +// This effectively uses a feed to store a pointer to content rather than the content itself +// Retrieving the update with the Swarm hash should return the manifest pointing directly to the data // and raw retrieve of that hash should return the data -func TestClientCreateFeedMultihash(t *testing.T) { +func TestClientBzzWithFeed(t *testing.T) { signer, _ := newTestSigner() + // Initialize a Swarm test server srv := swarmhttp.NewTestSwarmServer(t, serverFunc, nil) - client := NewClient(srv.URL) + swarmClient := NewClient(srv.URL) defer srv.Close() - // add the data our multihash aliased manifest will point to - databytes := []byte("bar") + // put together some data for our test: + dataBytes := []byte(` + // + // Create some data our manifest will point to. Data that could be very big and wouldn't fit in a feed update. + // So what we are going to do is upload it to Swarm bzz:// and obtain a **manifest hash** pointing to it: + // + // MANIFEST HASH --> DATA + // + // Then, we store that **manifest hash** into a Swarm Feed update. Once we have done this, + // we can use the **feed manifest hash** in bzz:// instead, this way: bzz://feed-manifest-hash. + // + // FEED MANIFEST HASH --> MANIFEST HASH --> DATA + // + // Given that we can update the feed at any time with a new **manifest hash** but the **feed manifest hash** + // stays constant, we have effectively created a fixed address to changing content. (Applause) + // + // FEED MANIFEST HASH (the same) --> MANIFEST HASH(2) --> DATA(2) + // + `) - swarmHash, err := client.UploadRaw(bytes.NewReader(databytes), int64(len(databytes)), false) - if err != nil { - t.Fatalf("Error uploading raw test data: %s", err) + // Create a virtual File out of memory containing the above data + f := &File{ + ReadCloser: ioutil.NopCloser(bytes.NewReader(dataBytes)), + ManifestEntry: api.ManifestEntry{ + ContentType: "text/plain", + Mode: 0660, + Size: int64(len(dataBytes)), + }, } - s := common.FromHex(swarmHash) - mh := multihash.ToMultihash(s) + // upload data to bzz:// and retrieve the content-addressed manifest hash, hex-encoded. + manifestAddressHex, err := swarmClient.Upload(f, "", false) + if err != nil { + t.Fatalf("Error creating manifest: %s", err) + } - // our feed topic - topic, _ := feed.NewTopic("foo.eth", nil) + // convert the hex-encoded manifest hash to a 32-byte slice + manifestAddress := common.FromHex(manifestAddressHex) - createRequest := feed.NewFirstRequest(topic) + if len(manifestAddress) != storage.AddressLength { + t.Fatalf("Something went wrong. Got a hash of an unexpected length. Expected %d bytes. Got %d", storage.AddressLength, len(manifestAddress)) + } - createRequest.SetData(mh) - if err := createRequest.Sign(signer); err != nil { + // Now create a **feed manifest**. For that, we need a topic: + topic, _ := feed.NewTopic("interesting topic indeed", nil) + + // Build a feed request to update data + request := feed.NewFirstRequest(topic) + + // Put the 32-byte address of the manifest into the feed update + request.SetData(manifestAddress) + + // Sign the update + if err := request.Sign(signer); err != nil { t.Fatalf("Error signing update: %s", err) } - feedManifestHash, err := client.CreateFeedWithManifest(createRequest) - + // Publish the update and at the same time request a **feed manifest** to be created + feedManifestAddressHex, err := swarmClient.CreateFeedWithManifest(request) if err != nil { t.Fatalf("Error creating feed manifest: %s", err) } - correctManifestAddrHex := "bb056a5264c295c2b0f613c8409b9c87ce9d71576ace02458160df4cc894210b" - if feedManifestHash != correctManifestAddrHex { - t.Fatalf("Response feed manifest mismatch, expected '%s', got '%s'", correctManifestAddrHex, feedManifestHash) + // Check we have received the exact **feed manifest** to be expected + // given the topic and user signing the updates: + correctFeedManifestAddrHex := "747c402e5b9dc715a25a4393147512167bab018a007fad7cdcd9adc7fce1ced2" + if feedManifestAddressHex != correctFeedManifestAddrHex { + t.Fatalf("Response feed manifest mismatch, expected '%s', got '%s'", correctFeedManifestAddrHex, feedManifestAddressHex) } // Check we get a not found error when trying to get feed updates with a made-up manifest - _, err = client.QueryFeed(nil, "bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb") + _, err = swarmClient.QueryFeed(nil, "bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb") if err != ErrNoFeedUpdatesFound { t.Fatalf("Expected to receive ErrNoFeedUpdatesFound error. Got: %s", err) } - reader, err := client.QueryFeed(nil, correctManifestAddrHex) + // If we query the feed directly we should get **manifest hash** back: + reader, err := swarmClient.QueryFeed(nil, correctFeedManifestAddrHex) if err != nil { t.Fatalf("Error retrieving feed updates: %s", err) } @@ -428,10 +469,27 @@ func TestClientCreateFeedMultihash(t *testing.T) { if err != nil { t.Fatal(err) } - if !bytes.Equal(mh, gotData) { - t.Fatalf("Expected: %v, got %v", mh, gotData) + + //Check that indeed the **manifest hash** is retrieved + if !bytes.Equal(manifestAddress, gotData) { + t.Fatalf("Expected: %v, got %v", manifestAddress, gotData) } + // Now the final test we were looking for: Use bzz:// and that should resolve all manifests + // and return the original data directly: + f, err = swarmClient.Download(feedManifestAddressHex, "") + if err != nil { + t.Fatal(err) + } + gotData, err = ioutil.ReadAll(f) + if err != nil { + t.Fatal(err) + } + + // Check that we get back the original data: + if !bytes.Equal(dataBytes, gotData) { + t.Fatalf("Expected: %v, got %v", manifestAddress, gotData) + } } // TestClientCreateUpdateFeed will check that feeds can be created and updated via the HTTP client. diff --git a/swarm/api/http/server_test.go b/swarm/api/http/server_test.go index 1ef3deece..e82762ce0 100644 --- a/swarm/api/http/server_test.go +++ b/swarm/api/http/server_test.go @@ -45,7 +45,6 @@ import ( "github.com/ethereum/go-ethereum/log" "github.com/ethereum/go-ethereum/swarm/api" swarm "github.com/ethereum/go-ethereum/swarm/api/client" - "github.com/ethereum/go-ethereum/swarm/multihash" "github.com/ethereum/go-ethereum/swarm/storage" "github.com/ethereum/go-ethereum/swarm/storage/feed" "github.com/ethereum/go-ethereum/swarm/testutil" @@ -69,60 +68,91 @@ func newTestSigner() (*feed.GenericSigner, error) { return feed.NewGenericSigner(privKey), nil } -// test the transparent resolving of multihash-containing feed updates with bzz:// scheme +// Test the transparent resolving of feed updates with bzz:// scheme // -// first upload data, and store the multihash to the resulting manifest in a feed update -// retrieving the update with the multihash should return the manifest pointing directly to the data +// First upload data to bzz:, and store the Swarm hash to the resulting manifest in a feed update. +// This effectively uses a feed to store a pointer to content rather than the content itself +// Retrieving the update with the Swarm hash should return the manifest pointing directly to the data // and raw retrieve of that hash should return the data -func TestBzzFeedMultihash(t *testing.T) { +func TestBzzWithFeed(t *testing.T) { signer, _ := newTestSigner() + // Initialize Swarm test server srv := NewTestSwarmServer(t, serverFunc, nil) defer srv.Close() - // add the data our multihash aliased manifest will point to - databytes := "bar" - testBzzUrl := fmt.Sprintf("%s/bzz:/", srv.URL) - resp, err := http.Post(testBzzUrl, "text/plain", bytes.NewReader([]byte(databytes))) + // put together some data for our test: + dataBytes := []byte(` + // + // Create some data our manifest will point to. Data that could be very big and wouldn't fit in a feed update. + // So what we are going to do is upload it to Swarm bzz:// and obtain a **manifest hash** pointing to it: + // + // MANIFEST HASH --> DATA + // + // Then, we store that **manifest hash** into a Swarm Feed update. Once we have done this, + // we can use the **feed manifest hash** in bzz:// instead, this way: bzz://feed-manifest-hash. + // + // FEED MANIFEST HASH --> MANIFEST HASH --> DATA + // + // Given that we can update the feed at any time with a new **manifest hash** but the **feed manifest hash** + // stays constant, we have effectively created a fixed address to changing content. (Applause) + // + // FEED MANIFEST HASH (the same) --> MANIFEST HASH(2) --> DATA(2) ... + // + `) + + // POST data to bzz and get back a content-addressed **manifest hash** pointing to it. + resp, err := http.Post(fmt.Sprintf("%s/bzz:/", srv.URL), "text/plain", bytes.NewReader([]byte(dataBytes))) if err != nil { t.Fatal(err) } + defer resp.Body.Close() if resp.StatusCode != http.StatusOK { t.Fatalf("err %s", resp.Status) } - b, err := ioutil.ReadAll(resp.Body) + manifestAddressHex, err := ioutil.ReadAll(resp.Body) if err != nil { t.Fatal(err) } - s := common.FromHex(string(b)) - mh := multihash.ToMultihash(s) - log.Info("added data", "manifest", string(b), "data", common.ToHex(mh)) + manifestAddress := common.FromHex(string(manifestAddressHex)) - topic, _ := feed.NewTopic("foo.eth", nil) + log.Info("added data", "manifest", string(manifestAddressHex)) + + // At this point we have uploaded the data and have a manifest pointing to it + // Now store that manifest address in a feed update. + // We also want a feed manifest, so we can use it to refer to the feed. + + // First, create a topic for our feed: + topic, _ := feed.NewTopic("interesting topic indeed", nil) + + // Create a feed update request: updateRequest := feed.NewFirstRequest(topic) - updateRequest.SetData(mh) + // Store the **manifest address** as data into the feed update. + updateRequest.SetData(manifestAddress) + // Sign the update if err := updateRequest.Sign(signer); err != nil { t.Fatal(err) } - log.Info("added data", "manifest", string(b), "data", common.ToHex(mh)) + log.Info("added data", "data", common.ToHex(manifestAddress)) - testUrl, err := url.Parse(fmt.Sprintf("%s/bzz-feed:/", srv.URL)) + // Build the feed update http request: + feedUpdateURL, err := url.Parse(fmt.Sprintf("%s/bzz-feed:/", srv.URL)) if err != nil { t.Fatal(err) } - query := testUrl.Query() + query := feedUpdateURL.Query() body := updateRequest.AppendValues(query) // this adds all query parameters and returns the data to be posted - query.Set("manifest", "1") // indicate we want a manifest back - testUrl.RawQuery = query.Encode() + query.Set("manifest", "1") // indicate we want a feed manifest back + feedUpdateURL.RawQuery = query.Encode() - // create the multihash update - resp, err = http.Post(testUrl.String(), "application/octet-stream", bytes.NewReader(body)) + // submit the feed update request to Swarm + resp, err = http.Post(feedUpdateURL.String(), "application/octet-stream", bytes.NewReader(body)) if err != nil { t.Fatal(err) } @@ -130,24 +160,25 @@ func TestBzzFeedMultihash(t *testing.T) { if resp.StatusCode != http.StatusOK { t.Fatalf("err %s", resp.Status) } - b, err = ioutil.ReadAll(resp.Body) + + feedManifestAddressHex, err := ioutil.ReadAll(resp.Body) if err != nil { t.Fatal(err) } - rsrcResp := &storage.Address{} - err = json.Unmarshal(b, rsrcResp) + feedManifestAddress := &storage.Address{} + err = json.Unmarshal(feedManifestAddressHex, feedManifestAddress) if err != nil { - t.Fatalf("data %s could not be unmarshaled: %v", b, err) + t.Fatalf("data %s could not be unmarshaled: %v", feedManifestAddressHex, err) } - correctManifestAddrHex := "bb056a5264c295c2b0f613c8409b9c87ce9d71576ace02458160df4cc894210b" - if rsrcResp.Hex() != correctManifestAddrHex { - t.Fatalf("Response feed manifest address mismatch, expected '%s', got '%s'", correctManifestAddrHex, rsrcResp.Hex()) + correctManifestAddrHex := "747c402e5b9dc715a25a4393147512167bab018a007fad7cdcd9adc7fce1ced2" + if feedManifestAddress.Hex() != correctManifestAddrHex { + t.Fatalf("Response feed manifest address mismatch, expected '%s', got '%s'", correctManifestAddrHex, feedManifestAddress.Hex()) } // get bzz manifest transparent feed update resolve - testBzzUrl = fmt.Sprintf("%s/bzz:/%s", srv.URL, rsrcResp) - resp, err = http.Get(testBzzUrl) + getBzzURL := fmt.Sprintf("%s/bzz:/%s", srv.URL, feedManifestAddress) + resp, err = http.Get(getBzzURL) if err != nil { t.Fatal(err) } @@ -155,12 +186,12 @@ func TestBzzFeedMultihash(t *testing.T) { if resp.StatusCode != http.StatusOK { t.Fatalf("err %s", resp.Status) } - b, err = ioutil.ReadAll(resp.Body) + retrievedData, err := ioutil.ReadAll(resp.Body) if err != nil { t.Fatal(err) } - if !bytes.Equal(b, []byte(databytes)) { - t.Fatalf("retrieved data mismatch, expected %x, got %x", databytes, b) + if !bytes.Equal(retrievedData, []byte(dataBytes)) { + t.Fatalf("retrieved data mismatch, expected %x, got %x", dataBytes, retrievedData) } } @@ -245,7 +276,8 @@ func TestBzzFeed(t *testing.T) { t.Fatalf("Expected manifest Feed '%s', got '%s'", correctFeedHex, manifest.Entries[0].Feed.Hex()) } - // get bzz manifest transparent feed update resolve + // take the chance to have bzz: crash on resolving a feed update that does not contain + // a swarm hash: testBzzUrl := fmt.Sprintf("%s/bzz:/%s", srv.URL, rsrcResp) resp, err = http.Get(testBzzUrl) if err != nil { @@ -253,7 +285,7 @@ func TestBzzFeed(t *testing.T) { } defer resp.Body.Close() if resp.StatusCode == http.StatusOK { - t.Fatal("Expected error status since feed update does not contain multihash. Received 200 OK") + t.Fatal("Expected error status since feed update does not contain a Swarm hash. Received 200 OK") } _, err = ioutil.ReadAll(resp.Body) if err != nil { diff --git a/swarm/multihash/multihash.go b/swarm/multihash/multihash.go deleted file mode 100644 index 3306e3a6d..000000000 --- a/swarm/multihash/multihash.go +++ /dev/null @@ -1,92 +0,0 @@ -// Copyright 2018 The go-ethereum Authors -// This file is part of the go-ethereum library. -// -// The go-ethereum library is free software: you can redistribute it and/or modify -// it under the terms of the GNU Lesser General Public License as published by -// the Free Software Foundation, either version 3 of the License, or -// (at your option) any later version. -// -// The go-ethereum library is distributed in the hope that it will be useful, -// but WITHOUT ANY WARRANTY; without even the implied warranty of -// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the -// GNU Lesser General Public License for more details. -// -// You should have received a copy of the GNU Lesser General Public License -// along with the go-ethereum library. If not, see . - -package multihash - -import ( - "bytes" - "encoding/binary" - "errors" - "fmt" -) - -const ( - defaultMultihashLength = 32 - defaultMultihashTypeCode = 0x1b -) - -var ( - multihashTypeCode uint8 - MultihashLength = defaultMultihashLength -) - -func init() { - multihashTypeCode = defaultMultihashTypeCode - MultihashLength = defaultMultihashLength -} - -// check if valid swarm multihash -func isSwarmMultihashType(code uint8) bool { - return code == multihashTypeCode -} - -// GetMultihashLength returns the digest length of the provided multihash -// It will fail if the multihash is not a valid swarm mulithash -func GetMultihashLength(data []byte) (int, int, error) { - cursor := 0 - typ, c := binary.Uvarint(data) - if c <= 0 { - return 0, 0, errors.New("unreadable hashtype field") - } - if !isSwarmMultihashType(uint8(typ)) { - return 0, 0, fmt.Errorf("hash code %x is not a swarm hashtype", typ) - } - cursor += c - hashlength, c := binary.Uvarint(data[cursor:]) - if c <= 0 { - return 0, 0, errors.New("unreadable length field") - } - cursor += c - - // we cheekily assume hashlength < maxint - inthashlength := int(hashlength) - if len(data[c:]) < inthashlength { - return 0, 0, errors.New("length mismatch") - } - return inthashlength, cursor, nil -} - -// FromMulithash returns the digest portion of the multihash -// It will fail if the multihash is not a valid swarm multihash -func FromMultihash(data []byte) ([]byte, error) { - hashLength, _, err := GetMultihashLength(data) - if err != nil { - return nil, err - } - return data[len(data)-hashLength:], nil -} - -// ToMulithash wraps the provided digest data with a swarm mulithash header -func ToMultihash(hashData []byte) []byte { - buf := bytes.NewBuffer(nil) - b := make([]byte, 8) - c := binary.PutUvarint(b, uint64(multihashTypeCode)) - buf.Write(b[:c]) - c = binary.PutUvarint(b, uint64(len(hashData))) - buf.Write(b[:c]) - buf.Write(hashData) - return buf.Bytes() -} diff --git a/swarm/multihash/multihash_test.go b/swarm/multihash/multihash_test.go deleted file mode 100644 index 85df741dd..000000000 --- a/swarm/multihash/multihash_test.go +++ /dev/null @@ -1,53 +0,0 @@ -// Copyright 2018 The go-ethereum Authors -// This file is part of the go-ethereum library. -// -// The go-ethereum library is free software: you can redistribute it and/or modify -// it under the terms of the GNU Lesser General Public License as published by -// the Free Software Foundation, either version 3 of the License, or -// (at your option) any later version. -// -// The go-ethereum library is distributed in the hope that it will be useful, -// but WITHOUT ANY WARRANTY; without even the implied warranty of -// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the -// GNU Lesser General Public License for more details. -// -// You should have received a copy of the GNU Lesser General Public License -// along with the go-ethereum library. If not, see . - -package multihash - -import ( - "bytes" - "math/rand" - "testing" -) - -// parse multihash, and check that invalid multihashes fail -func TestCheckMultihash(t *testing.T) { - hashbytes := make([]byte, 32) - c, err := rand.Read(hashbytes) - if err != nil { - t.Fatal(err) - } else if c < 32 { - t.Fatal("short read") - } - - expected := ToMultihash(hashbytes) - - l, hl, _ := GetMultihashLength(expected) - if l != 32 { - t.Fatalf("expected length %d, got %d", 32, l) - } else if hl != 2 { - t.Fatalf("expected header length %d, got %d", 2, hl) - } - if _, _, err := GetMultihashLength(expected[1:]); err == nil { - t.Fatal("expected failure on corrupt header") - } - if _, _, err := GetMultihashLength(expected[:len(expected)-2]); err == nil { - t.Fatal("expected failure on short content") - } - dh, _ := FromMultihash(expected) - if !bytes.Equal(dh, hashbytes) { - t.Fatalf("expected content hash %x, got %x", hashbytes, dh) - } -} -- 2.45.2 From bba5fd81921ee700388b4db72db3240d32576b6c Mon Sep 17 00:00:00 2001 From: holisticode Date: Mon, 26 Nov 2018 11:05:18 -0500 Subject: [PATCH 21/48] Accounting metrics reporter (#18136) --- p2p/protocols/accounting.go | 43 ++++-- p2p/protocols/accounting_simulation_test.go | 10 ++ p2p/protocols/reporter.go | 147 ++++++++++++++++++++ p2p/protocols/reporter_test.go | 77 ++++++++++ swarm/swap/swap.go | 5 + swarm/swarm.go | 53 ++++--- 6 files changed, 305 insertions(+), 30 deletions(-) create mode 100644 p2p/protocols/reporter.go create mode 100644 p2p/protocols/reporter_test.go diff --git a/p2p/protocols/accounting.go b/p2p/protocols/accounting.go index 06a1a5845..770406a27 100644 --- a/p2p/protocols/accounting.go +++ b/p2p/protocols/accounting.go @@ -16,29 +16,32 @@ package protocols -import "github.com/ethereum/go-ethereum/metrics" +import ( + "time" + + "github.com/ethereum/go-ethereum/metrics" +) //define some metrics var ( - //NOTE: these metrics just define the interfaces and are currently *NOT persisted* over sessions //All metrics are cumulative //total amount of units credited - mBalanceCredit = metrics.NewRegisteredCounterForced("account.balance.credit", nil) + mBalanceCredit metrics.Counter //total amount of units debited - mBalanceDebit = metrics.NewRegisteredCounterForced("account.balance.debit", nil) + mBalanceDebit metrics.Counter //total amount of bytes credited - mBytesCredit = metrics.NewRegisteredCounterForced("account.bytes.credit", nil) + mBytesCredit metrics.Counter //total amount of bytes debited - mBytesDebit = metrics.NewRegisteredCounterForced("account.bytes.debit", nil) + mBytesDebit metrics.Counter //total amount of credited messages - mMsgCredit = metrics.NewRegisteredCounterForced("account.msg.credit", nil) + mMsgCredit metrics.Counter //total amount of debited messages - mMsgDebit = metrics.NewRegisteredCounterForced("account.msg.debit", nil) + mMsgDebit metrics.Counter //how many times local node had to drop remote peers - mPeerDrops = metrics.NewRegisteredCounterForced("account.peerdrops", nil) + mPeerDrops metrics.Counter //how many times local node overdrafted and dropped - mSelfDrops = metrics.NewRegisteredCounterForced("account.selfdrops", nil) + mSelfDrops metrics.Counter ) //Prices defines how prices are being passed on to the accounting instance @@ -105,6 +108,26 @@ func NewAccounting(balance Balance, po Prices) *Accounting { return ah } +//SetupAccountingMetrics creates a separate registry for p2p accounting metrics; +//this registry should be independent of any other metrics as it persists at different endpoints. +//It also instantiates the given metrics and starts the persisting go-routine which +//at the passed interval writes the metrics to a LevelDB +func SetupAccountingMetrics(reportInterval time.Duration, path string) *AccountingMetrics { + //create an empty registry + registry := metrics.NewRegistry() + //instantiate the metrics + mBalanceCredit = metrics.NewRegisteredCounterForced("account.balance.credit", registry) + mBalanceDebit = metrics.NewRegisteredCounterForced("account.balance.debit", registry) + mBytesCredit = metrics.NewRegisteredCounterForced("account.bytes.credit", registry) + mBytesDebit = metrics.NewRegisteredCounterForced("account.bytes.debit", registry) + mMsgCredit = metrics.NewRegisteredCounterForced("account.msg.credit", registry) + mMsgDebit = metrics.NewRegisteredCounterForced("account.msg.debit", registry) + mPeerDrops = metrics.NewRegisteredCounterForced("account.peerdrops", registry) + mSelfDrops = metrics.NewRegisteredCounterForced("account.selfdrops", registry) + //create the DB and start persisting + return NewAccountingMetrics(registry, reportInterval, path) +} + //Implement Hook.Send // Send takes a peer, a size and a msg and // - calculates the cost for the local node sending a msg of size to peer using the Prices interface diff --git a/p2p/protocols/accounting_simulation_test.go b/p2p/protocols/accounting_simulation_test.go index 65b737abe..e90a1d81d 100644 --- a/p2p/protocols/accounting_simulation_test.go +++ b/p2p/protocols/accounting_simulation_test.go @@ -20,7 +20,10 @@ import ( "context" "flag" "fmt" + "io/ioutil" "math/rand" + "os" + "path/filepath" "reflect" "sync" "testing" @@ -66,6 +69,13 @@ func init() { func TestAccountingSimulation(t *testing.T) { //setup the balances objects for every node bal := newBalances(*nodes) + //setup the metrics system or tests will fail trying to write metrics + dir, err := ioutil.TempDir("", "account-sim") + if err != nil { + t.Fatal(err) + } + defer os.RemoveAll(dir) + SetupAccountingMetrics(1*time.Second, filepath.Join(dir, "metrics.db")) //define the node.Service for this test services := adapters.Services{ "accounting": func(ctx *adapters.ServiceContext) (node.Service, error) { diff --git a/p2p/protocols/reporter.go b/p2p/protocols/reporter.go new file mode 100644 index 000000000..215d4fe31 --- /dev/null +++ b/p2p/protocols/reporter.go @@ -0,0 +1,147 @@ +// Copyright 2018 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +package protocols + +import ( + "encoding/binary" + "time" + + "github.com/ethereum/go-ethereum/log" + "github.com/ethereum/go-ethereum/metrics" + + "github.com/syndtr/goleveldb/leveldb" +) + +//AccountMetrics abstracts away the metrics DB and +//the reporter to persist metrics +type AccountingMetrics struct { + reporter *reporter +} + +//Close will be called when the node is being shutdown +//for a graceful cleanup +func (am *AccountingMetrics) Close() { + close(am.reporter.quit) + am.reporter.db.Close() +} + +//reporter is an internal structure used to write p2p accounting related +//metrics to a LevelDB. It will periodically write the accrued metrics to the DB. +type reporter struct { + reg metrics.Registry //the registry for these metrics (independent of other metrics) + interval time.Duration //duration at which the reporter will persist metrics + db *leveldb.DB //the actual DB + quit chan struct{} //quit the reporter loop +} + +//NewMetricsDB creates a new LevelDB instance used to persist metrics defined +//inside p2p/protocols/accounting.go +func NewAccountingMetrics(r metrics.Registry, d time.Duration, path string) *AccountingMetrics { + var val = make([]byte, 8) + var err error + + //Create the LevelDB + db, err := leveldb.OpenFile(path, nil) + if err != nil { + log.Error(err.Error()) + return nil + } + + //Check for all defined metrics that there is a value in the DB + //If there is, assign it to the metric. This means that the node + //has been running before and that metrics have been persisted. + metricsMap := map[string]metrics.Counter{ + "account.balance.credit": mBalanceCredit, + "account.balance.debit": mBalanceDebit, + "account.bytes.credit": mBytesCredit, + "account.bytes.debit": mBytesDebit, + "account.msg.credit": mMsgCredit, + "account.msg.debit": mMsgDebit, + "account.peerdrops": mPeerDrops, + "account.selfdrops": mSelfDrops, + } + //iterate the map and get the values + for key, metric := range metricsMap { + val, err = db.Get([]byte(key), nil) + //until the first time a value is being written, + //this will return an error. + //it could be beneficial though to log errors later, + //but that would require a different logic + if err == nil { + metric.Inc(int64(binary.BigEndian.Uint64(val))) + } + } + + //create the reporter + rep := &reporter{ + reg: r, + interval: d, + db: db, + quit: make(chan struct{}), + } + + //run the go routine + go rep.run() + + m := &AccountingMetrics{ + reporter: rep, + } + + return m +} + +//run is the goroutine which periodically sends the metrics to the configured LevelDB +func (r *reporter) run() { + intervalTicker := time.NewTicker(r.interval) + + for { + select { + case <-intervalTicker.C: + //at each tick send the metrics + if err := r.save(); err != nil { + log.Error("unable to send metrics to LevelDB", "err", err) + //If there is an error in writing, exit the routine; we assume here that the error is + //severe and don't attempt to write again. + //Also, this should prevent leaking when the node is stopped + return + } + case <-r.quit: + //graceful shutdown + return + } + } +} + +//send the metrics to the DB +func (r *reporter) save() error { + //create a LevelDB Batch + batch := leveldb.Batch{} + //for each metric in the registry (which is independent)... + r.reg.Each(func(name string, i interface{}) { + metric, ok := i.(metrics.Counter) + if ok { + //assuming every metric here to be a Counter (separate registry) + //...create a snapshot... + ms := metric.Snapshot() + byteVal := make([]byte, 8) + binary.BigEndian.PutUint64(byteVal, uint64(ms.Count())) + //...and save the value to the DB + batch.Put([]byte(name), byteVal) + } + }) + return r.db.Write(&batch, nil) +} diff --git a/p2p/protocols/reporter_test.go b/p2p/protocols/reporter_test.go new file mode 100644 index 000000000..b9f06e674 --- /dev/null +++ b/p2p/protocols/reporter_test.go @@ -0,0 +1,77 @@ +// Copyright 2018 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +package protocols + +import ( + "io/ioutil" + "os" + "path/filepath" + "testing" + "time" + + "github.com/ethereum/go-ethereum/log" +) + +//TestReporter tests that the metrics being collected for p2p accounting +//are being persisted and available after restart of a node. +//It simulates restarting by just recreating the DB as if the node had restarted. +func TestReporter(t *testing.T) { + //create a test directory + dir, err := ioutil.TempDir("", "reporter-test") + if err != nil { + t.Fatal(err) + } + defer os.RemoveAll(dir) + + //setup the metrics + log.Debug("Setting up metrics first time") + reportInterval := 5 * time.Millisecond + metrics := SetupAccountingMetrics(reportInterval, filepath.Join(dir, "test.db")) + log.Debug("Done.") + + //do some metrics + mBalanceCredit.Inc(12) + mBytesCredit.Inc(34) + mMsgDebit.Inc(9) + + //give the reporter time to write the metrics to DB + time.Sleep(20 * time.Millisecond) + + //set the metrics to nil - this effectively simulates the node having shut down... + mBalanceCredit = nil + mBytesCredit = nil + mMsgDebit = nil + //close the DB also, or we can't create a new one + metrics.Close() + + //setup the metrics again + log.Debug("Setting up metrics second time") + metrics = SetupAccountingMetrics(reportInterval, filepath.Join(dir, "test.db")) + defer metrics.Close() + log.Debug("Done.") + + //now check the metrics, they should have the same value as before "shutdown" + if mBalanceCredit.Count() != 12 { + t.Fatalf("Expected counter to be %d, but is %d", 12, mBalanceCredit.Count()) + } + if mBytesCredit.Count() != 34 { + t.Fatalf("Expected counter to be %d, but is %d", 23, mBytesCredit.Count()) + } + if mMsgDebit.Count() != 9 { + t.Fatalf("Expected counter to be %d, but is %d", 9, mMsgDebit.Count()) + } +} diff --git a/swarm/swap/swap.go b/swarm/swap/swap.go index 137eb141d..5d636dc20 100644 --- a/swarm/swap/swap.go +++ b/swarm/swap/swap.go @@ -91,3 +91,8 @@ func (s *Swap) loadState(peer *protocols.Peer) (err error) { } return } + +//Clean up Swap +func (swap *Swap) Close() { + swap.stateStore.Close() +} diff --git a/swarm/swarm.go b/swarm/swarm.go index dc3756d3a..a4ff94051 100644 --- a/swarm/swarm.go +++ b/swarm/swarm.go @@ -1,4 +1,4 @@ -// Copyright 2016 The go-ethereum Authors +// Copyright 2018 The go-ethereum Authors // This file is part of the go-ethereum library. // // The go-ethereum library is free software: you can redistribute it and/or modify @@ -66,20 +66,22 @@ var ( // the swarm stack type Swarm struct { - config *api.Config // swarm configuration - api *api.API // high level api layer (fs/manifest) - dns api.Resolver // DNS registrar - fileStore *storage.FileStore // distributed preimage archive, the local API to the storage with document level storage/retrieval support - streamer *stream.Registry - bzz *network.Bzz // the logistic manager - backend chequebook.Backend // simple blockchain Backend - privateKey *ecdsa.PrivateKey - corsString string - swapEnabled bool - netStore *storage.NetStore - sfs *fuse.SwarmFS // need this to cleanup all the active mounts on node exit - ps *pss.Pss - swap *swap.Swap + config *api.Config // swarm configuration + api *api.API // high level api layer (fs/manifest) + dns api.Resolver // DNS registrar + fileStore *storage.FileStore // distributed preimage archive, the local API to the storage with document level storage/retrieval support + streamer *stream.Registry + bzz *network.Bzz // the logistic manager + backend chequebook.Backend // simple blockchain Backend + privateKey *ecdsa.PrivateKey + corsString string + swapEnabled bool + netStore *storage.NetStore + sfs *fuse.SwarmFS // need this to cleanup all the active mounts on node exit + ps *pss.Pss + swap *swap.Swap + stateStore *state.DBStore + accountingMetrics *protocols.AccountingMetrics tracerClose io.Closer } @@ -134,7 +136,7 @@ func NewSwarm(config *api.Config, mockStore *mock.NodeStore) (self *Swarm, err e LightNode: config.LightNodeEnabled, } - stateStore, err := state.NewDBStore(filepath.Join(config.Path, "state-store.db")) + self.stateStore, err = state.NewDBStore(filepath.Join(config.Path, "state-store.db")) if err != nil { return } @@ -179,6 +181,7 @@ func NewSwarm(config *api.Config, mockStore *mock.NodeStore) (self *Swarm, err e return nil, err } self.swap = swap.New(balancesStore) + self.accountingMetrics = protocols.SetupAccountingMetrics(10*time.Second, filepath.Join(config.Path, "metrics.db")) } var nodeID enode.ID @@ -203,7 +206,7 @@ func NewSwarm(config *api.Config, mockStore *mock.NodeStore) (self *Swarm, err e SyncUpdateDelay: config.SyncUpdateDelay, MaxPeerServers: config.MaxStreamPeerServers, } - self.streamer = stream.NewRegistry(nodeID, delivery, self.netStore, stateStore, registryOptions, self.swap) + self.streamer = stream.NewRegistry(nodeID, delivery, self.netStore, self.stateStore, registryOptions, self.swap) // Swarm Hash Merklised Chunking for Arbitrary-length Document/File storage self.fileStore = storage.NewFileStore(self.netStore, self.config.FileStoreParams) @@ -226,7 +229,7 @@ func NewSwarm(config *api.Config, mockStore *mock.NodeStore) (self *Swarm, err e log.Debug("Setup local storage") - self.bzz = network.NewBzz(bzzconfig, to, stateStore, self.streamer.GetSpec(), self.streamer.Run) + self.bzz = network.NewBzz(bzzconfig, to, self.stateStore, self.streamer.GetSpec(), self.streamer.Run) // Pss = postal service over swarm (devp2p over bzz) self.ps, err = pss.NewPss(to, config.Pss) @@ -446,14 +449,24 @@ func (self *Swarm) Stop() error { ch.Stop() ch.Save() } - + if self.swap != nil { + self.swap.Close() + } + if self.accountingMetrics != nil { + self.accountingMetrics.Close() + } if self.netStore != nil { self.netStore.Close() } self.sfs.Stop() stopCounter.Inc(1) self.streamer.Stop() - return self.bzz.Stop() + + err := self.bzz.Stop() + if self.stateStore != nil { + self.stateStore.Close() + } + return err } // implements the node.Service interface -- 2.45.2 From 1cd007ecae437f5cc54a026e592edfad642a0b69 Mon Sep 17 00:00:00 2001 From: lash Date: Mon, 26 Nov 2018 17:13:59 +0100 Subject: [PATCH 22/48] swarm/network: Correct neighborhood depth (#18066) --- swarm/network/kademlia.go | 135 +++++++++++++----- swarm/network/kademlia_test.go | 89 +++++++++++- swarm/network/simulation/example_test.go | 4 + swarm/network/simulation/kademlia.go | 1 + swarm/network/simulation/kademlia_test.go | 1 + swarm/network/stream/delivery_test.go | 2 + swarm/network/stream/intervals_test.go | 2 + .../network/stream/snapshot_retrieval_test.go | 1 + swarm/network/stream/snapshot_sync_test.go | 4 + swarm/network/stream/syncer_test.go | 2 + .../visualized_snapshot_sync_sim_test.go | 3 + swarm/network_test.go | 2 + 12 files changed, 209 insertions(+), 37 deletions(-) diff --git a/swarm/network/kademlia.go b/swarm/network/kademlia.go index 5fda51e3e..a8ecaa4be 100644 --- a/swarm/network/kademlia.go +++ b/swarm/network/kademlia.go @@ -177,7 +177,7 @@ func (k *Kademlia) SuggestPeer() (a *BzzAddr, o int, want bool) { k.lock.Lock() defer k.lock.Unlock() minsize := k.MinBinSize - depth := k.neighbourhoodDepth() + depth := depthForPot(k.conns, k.MinProxBinSize, k.base) // if there is a callable neighbour within the current proxBin, connect // this makes sure nearest neighbour set is fully connected var ppo int @@ -308,7 +308,7 @@ func (k *Kademlia) sendNeighbourhoodDepthChange() { // It provides signaling of neighbourhood depth change. // This part of the code is sending new neighbourhood depth to nDepthC if that condition is met. if k.nDepthC != nil { - nDepth := k.neighbourhoodDepth() + nDepth := depthForPot(k.conns, k.MinProxBinSize, k.base) if nDepth != k.nDepth { k.nDepth = nDepth k.nDepthC <- nDepth @@ -364,7 +364,7 @@ func (k *Kademlia) EachBin(base []byte, pof pot.Pof, o int, eachBinFunc func(con var startPo int var endPo int - kadDepth := k.neighbourhoodDepth() + kadDepth := depthForPot(k.conns, k.MinProxBinSize, k.base) k.conns.EachBin(base, pof, o, func(po, size int, f func(func(val pot.Val, i int) bool) bool) bool { if startPo > 0 && endPo != k.MaxProxDisplay { @@ -398,7 +398,7 @@ func (k *Kademlia) eachConn(base []byte, o int, f func(*Peer, int, bool) bool) { if len(base) == 0 { base = k.base } - depth := k.neighbourhoodDepth() + depth := depthForPot(k.conns, k.MinProxBinSize, k.base) k.conns.EachNeighbour(base, pof, func(val pot.Val, po int) bool { if po > o { return true @@ -420,7 +420,7 @@ func (k *Kademlia) eachAddr(base []byte, o int, f func(*BzzAddr, int, bool) bool if len(base) == 0 { base = k.base } - depth := k.neighbourhoodDepth() + depth := depthForPot(k.conns, k.MinProxBinSize, k.base) k.addrs.EachNeighbour(base, pof, func(val pot.Val, po int) bool { if po > o { return true @@ -429,26 +429,72 @@ func (k *Kademlia) eachAddr(base []byte, o int, f func(*BzzAddr, int, bool) bool }) } -// neighbourhoodDepth returns the proximity order that defines the distance of -// the nearest neighbour set with cardinality >= MinProxBinSize -// if there is altogether less than MinProxBinSize peers it returns 0 func (k *Kademlia) NeighbourhoodDepth() (depth int) { k.lock.RLock() defer k.lock.RUnlock() - return k.neighbourhoodDepth() + return depthForPot(k.conns, k.MinProxBinSize, k.base) } -func (k *Kademlia) neighbourhoodDepth() (depth int) { - if k.conns.Size() < k.MinProxBinSize { +// depthForPot returns the proximity order that defines the distance of +// the nearest neighbour set with cardinality >= MinProxBinSize +// if there is altogether less than MinProxBinSize peers it returns 0 +// caller must hold the lock +func depthForPot(p *pot.Pot, minProxBinSize int, pivotAddr []byte) (depth int) { + if p.Size() <= minProxBinSize { return 0 } + + // total number of peers in iteration var size int + + // true if iteration has all prox peers + var b bool + + // last po recorded in iteration + var lastPo int + f := func(v pot.Val, i int) bool { + // po == 256 means that addr is the pivot address(self) + if i == 256 { + return true + } size++ - depth = i - return size < k.MinProxBinSize + + // this means we have all nn-peers. + // depth is by default set to the bin of the farthest nn-peer + if size == minProxBinSize { + b = true + depth = i + return true + } + + // if there are empty bins between farthest nn and current node, + // the depth should recalculated to be + // the farthest of those empty bins + // + // 0 abac ccde + // 1 2a2a + // 2 589f <--- nearest non-nn + // ============ DEPTH 3 =========== + // 3 <--- don't count as empty bins + // 4 <--- don't count as empty bins + // 5 cbcb cdcd <---- furthest nn + // 6 a1a2 b3c4 + if b && i < depth { + depth = i + 1 + lastPo = i + return false + } + lastPo = i + return true + } + p.EachNeighbour(pivotAddr, pof, f) + + // cover edge case where more than one farthest nn + // AND we only have nn-peers + if lastPo == depth { + depth = 0 } - k.conns.EachNeighbour(k.base, pof, f) return depth } @@ -508,7 +554,7 @@ func (k *Kademlia) string() string { liverows := make([]string, k.MaxProxDisplay) peersrows := make([]string, k.MaxProxDisplay) - depth := k.neighbourhoodDepth() + depth := depthForPot(k.conns, k.MinProxBinSize, k.base) rest := k.conns.Size() k.conns.EachBin(k.base, pof, 0, func(po, size int, f func(func(val pot.Val, i int) bool) bool) bool { var rowlen int @@ -578,6 +624,7 @@ type PeerPot struct { // as hexadecimal representations of the address. // used for testing only func NewPeerPotMap(kadMinProxSize int, addrs [][]byte) map[string]*PeerPot { + // create a table of all nodes for health check np := pot.NewPot(nil, 0) for _, addr := range addrs { @@ -586,34 +633,47 @@ func NewPeerPotMap(kadMinProxSize int, addrs [][]byte) map[string]*PeerPot { ppmap := make(map[string]*PeerPot) for i, a := range addrs { - pl := 256 - prev := 256 + + // actual kademlia depth + depth := depthForPot(np, kadMinProxSize, a) + + // upon entering a new iteration + // this will hold the value the po should be + // if it's one higher than the po in the last iteration + prevPo := 256 + + // all empty bins which are outside neighbourhood depth var emptyBins []int + + // all nn-peers var nns [][]byte - np.EachNeighbour(addrs[i], pof, func(val pot.Val, po int) bool { - a := val.([]byte) + + np.EachNeighbour(a, pof, func(val pot.Val, po int) bool { + addr := val.([]byte) + // po == 256 means that addr is the pivot address(self) if po == 256 { return true } - if pl == 256 || pl == po { - nns = append(nns, a) + + // iterate through the neighbours, going from the closest to the farthest + // we calculate the nearest neighbours that should be in the set + // depth in this case equates to: + // 1. Within all bins that are higher or equal than depth there are + // at least minProxBinSize peers connected + // 2. depth-1 bin is not empty + if po >= depth { + nns = append(nns, addr) + prevPo = depth - 1 + return true } - if pl == 256 && len(nns) >= kadMinProxSize { - pl = po - prev = po + for j := prevPo; j > po; j-- { + emptyBins = append(emptyBins, j) } - if prev < pl { - for j := prev; j > po; j-- { - emptyBins = append(emptyBins, j) - } - } - prev = po - 1 + prevPo = po - 1 return true }) - for j := prev; j >= 0; j-- { - emptyBins = append(emptyBins, j) - } - log.Trace(fmt.Sprintf("%x NNS: %s", addrs[i][:4], LogAddrs(nns))) + + log.Trace(fmt.Sprintf("%x NNS: %s, emptyBins: %s", addrs[i][:4], LogAddrs(nns), logEmptyBins(emptyBins))) ppmap[common.Bytes2Hex(a)] = &PeerPot{nns, emptyBins} } return ppmap @@ -628,7 +688,7 @@ func (k *Kademlia) saturation(n int) int { prev++ return prev == po && size >= n }) - depth := k.neighbourhoodDepth() + depth := depthForPot(k.conns, k.MinProxBinSize, k.base) if depth < prev { return depth } @@ -641,8 +701,11 @@ func (k *Kademlia) full(emptyBins []int) (full bool) { prev := 0 e := len(emptyBins) ok := true - depth := k.neighbourhoodDepth() + depth := depthForPot(k.conns, k.MinProxBinSize, k.base) k.conns.EachBin(k.base, pof, 0, func(po, _ int, _ func(func(val pot.Val, i int) bool) bool) bool { + if po >= depth { + return false + } if prev == depth+1 { return true } diff --git a/swarm/network/kademlia_test.go b/swarm/network/kademlia_test.go index d2e051f45..184a2d942 100644 --- a/swarm/network/kademlia_test.go +++ b/swarm/network/kademlia_test.go @@ -25,6 +25,9 @@ import ( "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/log" + "github.com/ethereum/go-ethereum/p2p" + "github.com/ethereum/go-ethereum/p2p/enode" + "github.com/ethereum/go-ethereum/p2p/protocols" "github.com/ethereum/go-ethereum/swarm/pot" ) @@ -73,6 +76,76 @@ func Register(k *Kademlia, regs ...string) { } } +// tests the validity of neighborhood depth calculations +// +// in particular, it tests that if there are one or more consecutive +// empty bins above the farthest "nearest neighbor-peer" then +// the depth should be set at the farthest of those empty bins +// +// TODO: Make test adapt to change in MinProxBinSize +func TestNeighbourhoodDepth(t *testing.T) { + baseAddressBytes := RandomAddr().OAddr + kad := NewKademlia(baseAddressBytes, NewKadParams()) + + baseAddress := pot.NewAddressFromBytes(baseAddressBytes) + + closerAddress := pot.RandomAddressAt(baseAddress, 7) + closerPeer := newTestDiscoveryPeer(closerAddress, kad) + kad.On(closerPeer) + depth := kad.NeighbourhoodDepth() + if depth != 0 { + t.Fatalf("expected depth 0, was %d", depth) + } + + sameAddress := pot.RandomAddressAt(baseAddress, 7) + samePeer := newTestDiscoveryPeer(sameAddress, kad) + kad.On(samePeer) + depth = kad.NeighbourhoodDepth() + if depth != 0 { + t.Fatalf("expected depth 0, was %d", depth) + } + + midAddress := pot.RandomAddressAt(baseAddress, 4) + midPeer := newTestDiscoveryPeer(midAddress, kad) + kad.On(midPeer) + depth = kad.NeighbourhoodDepth() + if depth != 5 { + t.Fatalf("expected depth 5, was %d", depth) + } + + kad.Off(midPeer) + depth = kad.NeighbourhoodDepth() + if depth != 0 { + t.Fatalf("expected depth 0, was %d", depth) + } + + fartherAddress := pot.RandomAddressAt(baseAddress, 1) + fartherPeer := newTestDiscoveryPeer(fartherAddress, kad) + kad.On(fartherPeer) + depth = kad.NeighbourhoodDepth() + if depth != 2 { + t.Fatalf("expected depth 2, was %d", depth) + } + + midSameAddress := pot.RandomAddressAt(baseAddress, 4) + midSamePeer := newTestDiscoveryPeer(midSameAddress, kad) + kad.Off(closerPeer) + kad.On(midPeer) + kad.On(midSamePeer) + depth = kad.NeighbourhoodDepth() + if depth != 2 { + t.Fatalf("expected depth 2, was %d", depth) + } + + kad.Off(fartherPeer) + log.Trace(kad.string()) + time.Sleep(time.Millisecond) + depth = kad.NeighbourhoodDepth() + if depth != 0 { + t.Fatalf("expected depth 0, was %d", depth) + } +} + func testSuggestPeer(k *Kademlia, expAddr string, expPo int, expWant bool) error { addr, o, want := k.SuggestPeer() if binStr(addr) != expAddr { @@ -376,7 +449,7 @@ func TestKademliaHiveString(t *testing.T) { Register(k, "10000000", "10000001") k.MaxProxDisplay = 8 h := k.String() - expH := "\n=========================================================================\nMon Feb 27 12:10:28 UTC 2017 KΛÐΞMLIΛ hive: queen's address: 000000\npopulation: 2 (4), MinProxBinSize: 2, MinBinSize: 1, MaxBinSize: 4\n000 0 | 2 8100 (0) 8000 (0)\n============ DEPTH: 1 ==========================================\n001 1 4000 | 1 4000 (0)\n002 1 2000 | 1 2000 (0)\n003 0 | 0\n004 0 | 0\n005 0 | 0\n006 0 | 0\n007 0 | 0\n=========================================================================" + expH := "\n=========================================================================\nMon Feb 27 12:10:28 UTC 2017 KΛÐΞMLIΛ hive: queen's address: 000000\npopulation: 2 (4), MinProxBinSize: 2, MinBinSize: 1, MaxBinSize: 4\n============ DEPTH: 0 ==========================================\n000 0 | 2 8100 (0) 8000 (0)\n001 1 4000 | 1 4000 (0)\n002 1 2000 | 1 2000 (0)\n003 0 | 0\n004 0 | 0\n005 0 | 0\n006 0 | 0\n007 0 | 0\n=========================================================================" if expH[104:] != h[104:] { t.Fatalf("incorrect hive output. expected %v, got %v", expH, h) } @@ -644,3 +717,17 @@ func TestKademliaCase5(t *testing.T) { "78fafa0809929a1279ece089a51d12457c2d8416dff859aeb2ccc24bb50df5ec", "1dd39b1257e745f147cbbc3cadd609ccd6207c41056dbc4254bba5d2527d3ee5", "5f61dd66d4d94aec8fcc3ce0e7885c7edf30c43143fa730e2841c5d28e3cd081", "8aa8b0472cb351d967e575ad05c4b9f393e76c4b01ef4b3a54aac5283b78abc9", "4502f385152a915b438a6726ce3ea9342e7a6db91a23c2f6bee83a885ed7eb82", "718677a504249db47525e959ef1784bed167e1c46f1e0275b9c7b588e28a3758", "7c54c6ed1f8376323896ed3a4e048866410de189e9599dd89bf312ca4adb96b5", "18e03bd3378126c09e799a497150da5c24c895aedc84b6f0dbae41fc4bac081a", "23db76ac9e6e58d9f5395ca78252513a7b4118b4155f8462d3d5eec62486cadc", "40ae0e8f065e96c7adb7fa39505136401f01780481e678d718b7f6dbb2c906ec", "c1539998b8bae19d339d6bbb691f4e9daeb0e86847545229e80fe0dffe716e92", "ed139d73a2699e205574c08722ca9f030ad2d866c662f1112a276b91421c3cb9", "5bdb19584b7a36d09ca689422ef7e6bb681b8f2558a6b2177a8f7c812f631022", "636c9de7fe234ffc15d67a504c69702c719f626c17461d3f2918e924cd9d69e2", "de4455413ff9335c440d52458c6544191bd58a16d85f700c1de53b62773064ea", "de1963310849527acabc7885b6e345a56406a8f23e35e436b6d9725e69a79a83", "a80a50a467f561210a114cba6c7fb1489ed43a14d61a9edd70e2eb15c31f074d", "7804f12b8d8e6e4b375b242058242068a3809385e05df0e64973cde805cf729c", "60f9aa320c02c6f2e6370aa740cf7cea38083fa95fca8c99552cda52935c1520", "d8da963602390f6c002c00ce62a84b514edfce9ebde035b277a957264bb54d21", "8463d93256e026fe436abad44697152b9a56ac8e06a0583d318e9571b83d073c", "9a3f78fcefb9a05e40a23de55f6153d7a8b9d973ede43a380bf46bb3b3847de1", "e3bb576f4b3760b9ca6bff59326f4ebfc4a669d263fb7d67ab9797adea54ed13", "4d5cdbd6dcca5bdf819a0fe8d175dc55cc96f088d37462acd5ea14bc6296bdbe", "5a0ed28de7b5258c727cb85447071c74c00a5fbba9e6bc0393bc51944d04ab2a", "61e4ddb479c283c638f4edec24353b6cc7a3a13b930824aad016b0996ca93c47", "7e3610868acf714836cafaaa7b8c009a9ac6e3a6d443e5586cf661530a204ee2", "d74b244d4345d2c86e30a097105e4fb133d53c578320285132a952cdaa64416e", "cfeed57d0f935bfab89e3f630a7c97e0b1605f0724d85a008bbfb92cb47863a8", "580837af95055670e20d494978f60c7f1458dc4b9e389fc7aa4982b2aca3bce3", "df55c0c49e6c8a83d82dfa1c307d3bf6a20e18721c80d8ec4f1f68dc0a137ced", "5f149c51ce581ba32a285439a806c063ced01ccd4211cd024e6a615b8f216f95", "1eb76b00aeb127b10dd1b7cd4c3edeb4d812b5a658f0feb13e85c4d2b7c6fe06", "7a56ba7c3fb7cbfb5561a46a75d95d7722096b45771ec16e6fa7bbfab0b35dfe", "4bae85ad88c28470f0015246d530adc0cd1778bdd5145c3c6b538ee50c4e04bd", "afd1892e2a7145c99ec0ebe9ded0d3fec21089b277a68d47f45961ec5e39e7e0", "953138885d7b36b0ef79e46030f8e61fd7037fbe5ce9e0a94d728e8c8d7eab86", "de761613ef305e4f628cb6bf97d7b7dc69a9d513dc233630792de97bcda777a6", "3f3087280063d09504c084bbf7fdf984347a72b50d097fd5b086ffabb5b3fb4c", "7d18a94bb1ebfdef4d3e454d2db8cb772f30ca57920dd1e402184a9e598581a0", "a7d6fbdc9126d9f10d10617f49fb9f5474ffe1b229f76b7dd27cebba30eccb5d", "fad0246303618353d1387ec10c09ee991eb6180697ed3470ed9a6b377695203d", "1cf66e09ea51ee5c23df26615a9e7420be2ac8063f28f60a3bc86020e94fe6f3", "8269cdaa153da7c358b0b940791af74d7c651cd4d3f5ed13acfe6d0f2c539e7f", "90d52eaaa60e74bf1c79106113f2599471a902d7b1c39ac1f55b20604f453c09", "9788fd0c09190a3f3d0541f68073a2f44c2fcc45bb97558a7c319f36c25a75b3", "10b68fc44157ecfdae238ee6c1ce0333f906ad04d1a4cb1505c8e35c3c87fbb0", "e5284117fdf3757920475c786e0004cb00ba0932163659a89b36651a01e57394", "403ad51d911e113dcd5f9ff58c94f6d278886a2a4da64c3ceca2083282c92de3", ) } + +func newTestDiscoveryPeer(addr pot.Address, kad *Kademlia) *Peer { + rw := &p2p.MsgPipeRW{} + p := p2p.NewPeer(enode.ID{}, "foo", []p2p.Cap{}) + pp := protocols.NewPeer(p, rw, &protocols.Spec{}) + bp := &BzzPeer{ + Peer: pp, + BzzAddr: &BzzAddr{ + OAddr: addr.Bytes(), + UAddr: []byte(fmt.Sprintf("%x", addr[:])), + }, + } + return NewPeer(bp, kad) +} diff --git a/swarm/network/simulation/example_test.go b/swarm/network/simulation/example_test.go index bacc64d53..7b6204617 100644 --- a/swarm/network/simulation/example_test.go +++ b/swarm/network/simulation/example_test.go @@ -33,6 +33,10 @@ import ( // BucketKeyKademlia key. This allows to use WaitTillHealthy to block until // all nodes have the their Kadmlias healthy. func ExampleSimulation_WaitTillHealthy() { + + log.Error("temporarily disabled as simulations.WaitTillHealthy cannot be trusted") + return + sim := simulation.New(map[string]simulation.ServiceFunc{ "bzz": func(ctx *adapters.ServiceContext, b *sync.Map) (node.Service, func(), error) { addr := network.NewAddr(ctx.Config.Node()) diff --git a/swarm/network/simulation/kademlia.go b/swarm/network/simulation/kademlia.go index f895181d9..7982810ca 100644 --- a/swarm/network/simulation/kademlia.go +++ b/swarm/network/simulation/kademlia.go @@ -33,6 +33,7 @@ var BucketKeyKademlia BucketKey = "kademlia" // WaitTillHealthy is blocking until the health of all kademlias is true. // If error is not nil, a map of kademlia that was found not healthy is returned. +// TODO: Check correctness since change in kademlia depth calculation logic func (s *Simulation) WaitTillHealthy(ctx context.Context, kadMinProxSize int) (ill map[enode.ID]*network.Kademlia, err error) { // Prepare PeerPot map for checking Kademlia health var ppmap map[string]*network.PeerPot diff --git a/swarm/network/simulation/kademlia_test.go b/swarm/network/simulation/kademlia_test.go index 285644a0f..024830315 100644 --- a/swarm/network/simulation/kademlia_test.go +++ b/swarm/network/simulation/kademlia_test.go @@ -28,6 +28,7 @@ import ( ) func TestWaitTillHealthy(t *testing.T) { + sim := New(map[string]ServiceFunc{ "bzz": func(ctx *adapters.ServiceContext, b *sync.Map) (node.Service, func(), error) { addr := network.NewAddr(ctx.Config.Node()) diff --git a/swarm/network/stream/delivery_test.go b/swarm/network/stream/delivery_test.go index a6173a389..f69f80499 100644 --- a/swarm/network/stream/delivery_test.go +++ b/swarm/network/stream/delivery_test.go @@ -453,6 +453,8 @@ func TestDeliveryFromNodes(t *testing.T) { } func testDeliveryFromNodes(t *testing.T, nodes, conns, chunkCount int, skipCheck bool) { + + t.Skip("temporarily disabled as simulations.WaitTillHealthy cannot be trusted") sim := simulation.New(map[string]simulation.ServiceFunc{ "streamer": func(ctx *adapters.ServiceContext, bucket *sync.Map) (s node.Service, cleanup func(), err error) { node := ctx.Config.Node() diff --git a/swarm/network/stream/intervals_test.go b/swarm/network/stream/intervals_test.go index defb6df50..668cf586c 100644 --- a/swarm/network/stream/intervals_test.go +++ b/swarm/network/stream/intervals_test.go @@ -52,6 +52,8 @@ func TestIntervalsLiveAndHistory(t *testing.T) { } func testIntervals(t *testing.T, live bool, history *Range, skipCheck bool) { + + t.Skip("temporarily disabled as simulations.WaitTillHealthy cannot be trusted") nodes := 2 chunkCount := dataChunkCount externalStreamName := "externalStream" diff --git a/swarm/network/stream/snapshot_retrieval_test.go b/swarm/network/stream/snapshot_retrieval_test.go index 5ea0b1511..932e28b32 100644 --- a/swarm/network/stream/snapshot_retrieval_test.go +++ b/swarm/network/stream/snapshot_retrieval_test.go @@ -246,6 +246,7 @@ simulation's `action` function. The snapshot should have 'streamer' in its service list. */ func runRetrievalTest(chunkCount int, nodeCount int) error { + sim := simulation.New(retrievalSimServiceMap) defer sim.Close() diff --git a/swarm/network/stream/snapshot_sync_test.go b/swarm/network/stream/snapshot_sync_test.go index 4e56f71b5..4a632c8c9 100644 --- a/swarm/network/stream/snapshot_sync_test.go +++ b/swarm/network/stream/snapshot_sync_test.go @@ -182,6 +182,8 @@ func streamerFunc(ctx *adapters.ServiceContext, bucket *sync.Map) (s node.Servic } func testSyncingViaGlobalSync(t *testing.T, chunkCount int, nodeCount int) { + + t.Skip("temporarily disabled as simulations.WaitTillHealthy cannot be trusted") sim := simulation.New(simServiceMap) defer sim.Close() @@ -329,6 +331,8 @@ assuming that the snapshot file identifies a healthy kademlia network. The snapshot should have 'streamer' in its service list. */ func testSyncingViaDirectSubscribe(t *testing.T, chunkCount int, nodeCount int) error { + + t.Skip("temporarily disabled as simulations.WaitTillHealthy cannot be trusted") sim := simulation.New(map[string]simulation.ServiceFunc{ "streamer": func(ctx *adapters.ServiceContext, bucket *sync.Map) (s node.Service, cleanup func(), err error) { n := ctx.Config.Node() diff --git a/swarm/network/stream/syncer_test.go b/swarm/network/stream/syncer_test.go index 5764efc92..df1a98c92 100644 --- a/swarm/network/stream/syncer_test.go +++ b/swarm/network/stream/syncer_test.go @@ -68,6 +68,8 @@ func createMockStore(globalStore mock.GlobalStorer, id enode.ID, addr *network.B } func testSyncBetweenNodes(t *testing.T, nodes, conns, chunkCount int, skipCheck bool, po uint8) { + + t.Skip("temporarily disabled as simulations.WaitTillHealthy cannot be trusted") sim := simulation.New(map[string]simulation.ServiceFunc{ "streamer": func(ctx *adapters.ServiceContext, bucket *sync.Map) (s node.Service, cleanup func(), err error) { var store storage.ChunkStore diff --git a/swarm/network/stream/visualized_snapshot_sync_sim_test.go b/swarm/network/stream/visualized_snapshot_sync_sim_test.go index 437c17e5e..f6d618020 100644 --- a/swarm/network/stream/visualized_snapshot_sync_sim_test.go +++ b/swarm/network/stream/visualized_snapshot_sync_sim_test.go @@ -84,6 +84,8 @@ func watchSim(sim *simulation.Simulation) (context.Context, context.CancelFunc) //This test requests bogus hashes into the network func TestNonExistingHashesWithServer(t *testing.T) { + + t.Skip("temporarily disabled as simulations.WaitTillHealthy cannot be trusted") nodeCount, _, sim := setupSim(retrievalSimServiceMap) defer sim.Close() @@ -143,6 +145,7 @@ func sendSimTerminatedEvent(sim *simulation.Simulation) { //can visualize messages like SendOfferedMsg, WantedHashesMsg, DeliveryMsg func TestSnapshotSyncWithServer(t *testing.T) { + t.Skip("temporarily disabled as simulations.WaitTillHealthy cannot be trusted") nodeCount, chunkCount, sim := setupSim(simServiceMap) defer sim.Close() diff --git a/swarm/network_test.go b/swarm/network_test.go index d84f28147..41993dfc6 100644 --- a/swarm/network_test.go +++ b/swarm/network_test.go @@ -259,6 +259,8 @@ type testSwarmNetworkOptions struct { // - May wait for Kademlia on every node to be healthy. // - Checking if a file is retrievable from all nodes. func testSwarmNetwork(t *testing.T, o *testSwarmNetworkOptions, steps ...testSwarmNetworkStep) { + + t.Skip("temporarily disabled as simulations.WaitTillHealthy cannot be trusted") if o == nil { o = new(testSwarmNetworkOptions) } -- 2.45.2 From 4f0d978eaaebdd118af3bfe623782c70601daaed Mon Sep 17 00:00:00 2001 From: Javier Peletier Date: Mon, 26 Nov 2018 17:37:59 +0100 Subject: [PATCH 23/48] cmd/swarm: update should error on manifest mismatch (#18047) * cmd/swarm: fix ethersphere/go-ethereum#979: update should error on manifest mistmatch * cmd/swarm: fixed comments and remove sprintf from log.Info * cmd/swarm: remove unnecessary comment --- cmd/swarm/feeds.go | 6 +++++- cmd/swarm/feeds_test.go | 41 ++++++++++++++++++++++++++++++++++++----- 2 files changed, 41 insertions(+), 6 deletions(-) diff --git a/cmd/swarm/feeds.go b/cmd/swarm/feeds.go index f26a8cc7d..6cd971a92 100644 --- a/cmd/swarm/feeds.go +++ b/cmd/swarm/feeds.go @@ -169,7 +169,6 @@ func feedUpdate(ctx *cli.Context) { query = new(feed.Query) query.User = signer.Address() query.Topic = getTopic(ctx) - } // Retrieve a feed update request @@ -178,6 +177,11 @@ func feedUpdate(ctx *cli.Context) { utils.Fatalf("Error retrieving feed status: %s", err.Error()) } + // Check that the provided signer matches the request to sign + if updateRequest.User != signer.Address() { + utils.Fatalf("Signer address does not match the update request") + } + // set the new data updateRequest.SetData(data) diff --git a/cmd/swarm/feeds_test.go b/cmd/swarm/feeds_test.go index a0cedf0d3..4c40f62a8 100644 --- a/cmd/swarm/feeds_test.go +++ b/cmd/swarm/feeds_test.go @@ -19,7 +19,6 @@ package main import ( "bytes" "encoding/json" - "fmt" "io/ioutil" "os" "testing" @@ -69,7 +68,7 @@ func TestCLIFeedUpdate(t *testing.T) { hexData} // create an update and expect an exit without errors - log.Info(fmt.Sprintf("updating a feed with 'swarm feed update'")) + log.Info("updating a feed with 'swarm feed update'") cmd := runSwarm(t, flags...) cmd.ExpectExit() @@ -116,7 +115,7 @@ func TestCLIFeedUpdate(t *testing.T) { "--user", address.Hex(), } - log.Info(fmt.Sprintf("getting feed info with 'swarm feed info'")) + log.Info("getting feed info with 'swarm feed info'") cmd = runSwarm(t, flags...) _, matches := cmd.ExpectRegexp(`.*`) // regex hack to extract stdout cmd.ExpectExit() @@ -141,9 +140,9 @@ func TestCLIFeedUpdate(t *testing.T) { "--topic", topic.Hex(), } - log.Info(fmt.Sprintf("Publishing manifest with 'swarm feed create'")) + log.Info("Publishing manifest with 'swarm feed create'") cmd = runSwarm(t, flags...) - _, matches = cmd.ExpectRegexp(`[a-f\d]{64}`) // regex hack to extract stdout + _, matches = cmd.ExpectRegexp(`[a-f\d]{64}`) cmd.ExpectExit() manifestAddress := matches[0] // read the received feed manifest @@ -162,4 +161,36 @@ func TestCLIFeedUpdate(t *testing.T) { if !bytes.Equal(data, retrieved) { t.Fatalf("Received %s, expected %s", retrieved, data) } + + // test publishing a manifest for a different user + flags = []string{ + "--bzzapi", srv.URL, + "feed", "create", + "--topic", topic.Hex(), + "--user", "0xaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa", // different user + } + + log.Info("Publishing manifest with 'swarm feed create' for a different user") + cmd = runSwarm(t, flags...) + _, matches = cmd.ExpectRegexp(`[a-f\d]{64}`) + cmd.ExpectExit() + + manifestAddress = matches[0] // read the received feed manifest + + // now let's try to update that user's manifest which we don't have the private key for + flags = []string{ + "--bzzapi", srv.URL, + "--bzzaccount", pkFileName, + "feed", "update", + "--manifest", manifestAddress, + hexData} + + // create an update and expect an error given there is a user mismatch + log.Info("updating a feed with 'swarm feed update'") + cmd = runSwarm(t, flags...) + cmd.ExpectRegexp("Fatal:.*") // best way so far to detect a failure. + cmd.ExpectExit() + if cmd.ExitStatus() == 0 { + t.Fatal("Expected nonzero exit code when updating a manifest with the wrong user. Got 0.") + } } -- 2.45.2 From c207edf2a3a6f48b4fc78cc55982d648eedab198 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jano=C5=A1=20Gulja=C5=A1?= Date: Mon, 26 Nov 2018 18:49:01 +0100 Subject: [PATCH 24/48] swarm: add database abstractions (shed package) (#18183) --- swarm/shed/db.go | 130 ++++++++++ swarm/shed/db_test.go | 110 ++++++++ swarm/shed/example_store_test.go | 332 ++++++++++++++++++++++++ swarm/shed/field_string.go | 66 +++++ swarm/shed/field_string_test.go | 110 ++++++++ swarm/shed/field_struct.go | 71 ++++++ swarm/shed/field_struct_test.go | 127 +++++++++ swarm/shed/field_uint64.go | 108 ++++++++ swarm/shed/field_uint64_test.go | 194 ++++++++++++++ swarm/shed/index.go | 264 +++++++++++++++++++ swarm/shed/index_test.go | 426 +++++++++++++++++++++++++++++++ swarm/shed/schema.go | 134 ++++++++++ swarm/shed/schema_test.go | 126 +++++++++ swarm/storage/mock/db/db.go | 7 + swarm/storage/mock/mem/mem.go | 16 ++ swarm/storage/mock/mock.go | 7 + swarm/storage/mock/rpc/rpc.go | 6 + swarm/storage/mock/test/test.go | 53 ++++ 18 files changed, 2287 insertions(+) create mode 100644 swarm/shed/db.go create mode 100644 swarm/shed/db_test.go create mode 100644 swarm/shed/example_store_test.go create mode 100644 swarm/shed/field_string.go create mode 100644 swarm/shed/field_string_test.go create mode 100644 swarm/shed/field_struct.go create mode 100644 swarm/shed/field_struct_test.go create mode 100644 swarm/shed/field_uint64.go create mode 100644 swarm/shed/field_uint64_test.go create mode 100644 swarm/shed/index.go create mode 100644 swarm/shed/index_test.go create mode 100644 swarm/shed/schema.go create mode 100644 swarm/shed/schema_test.go diff --git a/swarm/shed/db.go b/swarm/shed/db.go new file mode 100644 index 000000000..e128b8cbc --- /dev/null +++ b/swarm/shed/db.go @@ -0,0 +1,130 @@ +// Copyright 2018 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +// Package shed provides a simple abstraction components to compose +// more complex operations on storage data organized in fields and indexes. +// +// Only type which holds logical information about swarm storage chunks data +// and metadata is IndexItem. This part is not generalized mostly for +// performance reasons. +package shed + +import ( + "github.com/ethereum/go-ethereum/metrics" + "github.com/syndtr/goleveldb/leveldb" + "github.com/syndtr/goleveldb/leveldb/iterator" + "github.com/syndtr/goleveldb/leveldb/opt" +) + +// The limit for LevelDB OpenFilesCacheCapacity. +const openFileLimit = 128 + +// DB provides abstractions over LevelDB in order to +// implement complex structures using fields and ordered indexes. +// It provides a schema functionality to store fields and indexes +// information about naming and types. +type DB struct { + ldb *leveldb.DB +} + +// NewDB constructs a new DB and validates the schema +// if it exists in database on the given path. +func NewDB(path string) (db *DB, err error) { + ldb, err := leveldb.OpenFile(path, &opt.Options{ + OpenFilesCacheCapacity: openFileLimit, + }) + if err != nil { + return nil, err + } + db = &DB{ + ldb: ldb, + } + + if _, err = db.getSchema(); err != nil { + if err == leveldb.ErrNotFound { + // save schema with initialized default fields + if err = db.putSchema(schema{ + Fields: make(map[string]fieldSpec), + Indexes: make(map[byte]indexSpec), + }); err != nil { + return nil, err + } + } else { + return nil, err + } + } + return db, nil +} + +// Put wraps LevelDB Put method to increment metrics counter. +func (db *DB) Put(key []byte, value []byte) (err error) { + err = db.ldb.Put(key, value, nil) + if err != nil { + metrics.GetOrRegisterCounter("DB.putFail", nil).Inc(1) + return err + } + metrics.GetOrRegisterCounter("DB.put", nil).Inc(1) + return nil +} + +// Get wraps LevelDB Get method to increment metrics counter. +func (db *DB) Get(key []byte) (value []byte, err error) { + value, err = db.ldb.Get(key, nil) + if err != nil { + if err == leveldb.ErrNotFound { + metrics.GetOrRegisterCounter("DB.getNotFound", nil).Inc(1) + } else { + metrics.GetOrRegisterCounter("DB.getFail", nil).Inc(1) + } + return nil, err + } + metrics.GetOrRegisterCounter("DB.get", nil).Inc(1) + return value, nil +} + +// Delete wraps LevelDB Delete method to increment metrics counter. +func (db *DB) Delete(key []byte) (err error) { + err = db.ldb.Delete(key, nil) + if err != nil { + metrics.GetOrRegisterCounter("DB.deleteFail", nil).Inc(1) + return err + } + metrics.GetOrRegisterCounter("DB.delete", nil).Inc(1) + return nil +} + +// NewIterator wraps LevelDB NewIterator method to increment metrics counter. +func (db *DB) NewIterator() iterator.Iterator { + metrics.GetOrRegisterCounter("DB.newiterator", nil).Inc(1) + + return db.ldb.NewIterator(nil, nil) +} + +// WriteBatch wraps LevelDB Write method to increment metrics counter. +func (db *DB) WriteBatch(batch *leveldb.Batch) (err error) { + err = db.ldb.Write(batch, nil) + if err != nil { + metrics.GetOrRegisterCounter("DB.writebatchFail", nil).Inc(1) + return err + } + metrics.GetOrRegisterCounter("DB.writebatch", nil).Inc(1) + return nil +} + +// Close closes LevelDB database. +func (db *DB) Close() (err error) { + return db.ldb.Close() +} diff --git a/swarm/shed/db_test.go b/swarm/shed/db_test.go new file mode 100644 index 000000000..45325beeb --- /dev/null +++ b/swarm/shed/db_test.go @@ -0,0 +1,110 @@ +// Copyright 2018 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +package shed + +import ( + "io/ioutil" + "os" + "testing" +) + +// TestNewDB constructs a new DB +// and validates if the schema is initialized properly. +func TestNewDB(t *testing.T) { + db, cleanupFunc := newTestDB(t) + defer cleanupFunc() + + s, err := db.getSchema() + if err != nil { + t.Fatal(err) + } + if s.Fields == nil { + t.Error("schema fields are empty") + } + if len(s.Fields) != 0 { + t.Errorf("got schema fields length %v, want %v", len(s.Fields), 0) + } + if s.Indexes == nil { + t.Error("schema indexes are empty") + } + if len(s.Indexes) != 0 { + t.Errorf("got schema indexes length %v, want %v", len(s.Indexes), 0) + } +} + +// TestDB_persistence creates one DB, saves a field and closes that DB. +// Then, it constructs another DB and trues to retrieve the saved value. +func TestDB_persistence(t *testing.T) { + dir, err := ioutil.TempDir("", "shed-test-persistence") + if err != nil { + t.Fatal(err) + } + defer os.RemoveAll(dir) + + db, err := NewDB(dir) + if err != nil { + t.Fatal(err) + } + stringField, err := db.NewStringField("preserve-me") + if err != nil { + t.Fatal(err) + } + want := "persistent value" + err = stringField.Put(want) + if err != nil { + t.Fatal(err) + } + err = db.Close() + if err != nil { + t.Fatal(err) + } + + db2, err := NewDB(dir) + if err != nil { + t.Fatal(err) + } + stringField2, err := db2.NewStringField("preserve-me") + if err != nil { + t.Fatal(err) + } + got, err := stringField2.Get() + if err != nil { + t.Fatal(err) + } + if got != want { + t.Errorf("got string %q, want %q", got, want) + } +} + +// newTestDB is a helper function that constructs a +// temporary database and returns a cleanup function that must +// be called to remove the data. +func newTestDB(t *testing.T) (db *DB, cleanupFunc func()) { + t.Helper() + + dir, err := ioutil.TempDir("", "shed-test") + if err != nil { + t.Fatal(err) + } + cleanupFunc = func() { os.RemoveAll(dir) } + db, err = NewDB(dir) + if err != nil { + cleanupFunc() + t.Fatal(err) + } + return db, cleanupFunc +} diff --git a/swarm/shed/example_store_test.go b/swarm/shed/example_store_test.go new file mode 100644 index 000000000..2ed0be141 --- /dev/null +++ b/swarm/shed/example_store_test.go @@ -0,0 +1,332 @@ +// Copyright 2018 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +package shed_test + +import ( + "bytes" + "context" + "encoding/binary" + "fmt" + "io/ioutil" + "log" + "os" + "time" + + "github.com/ethereum/go-ethereum/swarm/shed" + "github.com/ethereum/go-ethereum/swarm/storage" + "github.com/syndtr/goleveldb/leveldb" +) + +// Store holds fields and indexes (including their encoding functions) +// and defines operations on them by composing data from them. +// It implements storage.ChunkStore interface. +// It is just an example without any support for parallel operations +// or real world implementation. +type Store struct { + db *shed.DB + + // fields and indexes + schemaName shed.StringField + sizeCounter shed.Uint64Field + accessCounter shed.Uint64Field + retrievalIndex shed.Index + accessIndex shed.Index + gcIndex shed.Index +} + +// New returns new Store. All fields and indexes are initialized +// and possible conflicts with schema from existing database is checked +// automatically. +func New(path string) (s *Store, err error) { + db, err := shed.NewDB(path) + if err != nil { + return nil, err + } + s = &Store{ + db: db, + } + // Identify current storage schema by arbitrary name. + s.schemaName, err = db.NewStringField("schema-name") + if err != nil { + return nil, err + } + // Global ever incrementing index of chunk accesses. + s.accessCounter, err = db.NewUint64Field("access-counter") + if err != nil { + return nil, err + } + // Index storing actual chunk address, data and store timestamp. + s.retrievalIndex, err = db.NewIndex("Address->StoreTimestamp|Data", shed.IndexFuncs{ + EncodeKey: func(fields shed.IndexItem) (key []byte, err error) { + return fields.Address, nil + }, + DecodeKey: func(key []byte) (e shed.IndexItem, err error) { + e.Address = key + return e, nil + }, + EncodeValue: func(fields shed.IndexItem) (value []byte, err error) { + b := make([]byte, 8) + binary.BigEndian.PutUint64(b, uint64(fields.StoreTimestamp)) + value = append(b, fields.Data...) + return value, nil + }, + DecodeValue: func(value []byte) (e shed.IndexItem, err error) { + e.StoreTimestamp = int64(binary.BigEndian.Uint64(value[:8])) + e.Data = value[8:] + return e, nil + }, + }) + if err != nil { + return nil, err + } + // Index storing access timestamp for a particular address. + // It is needed in order to update gc index keys for iteration order. + s.accessIndex, err = db.NewIndex("Address->AccessTimestamp", shed.IndexFuncs{ + EncodeKey: func(fields shed.IndexItem) (key []byte, err error) { + return fields.Address, nil + }, + DecodeKey: func(key []byte) (e shed.IndexItem, err error) { + e.Address = key + return e, nil + }, + EncodeValue: func(fields shed.IndexItem) (value []byte, err error) { + b := make([]byte, 8) + binary.BigEndian.PutUint64(b, uint64(fields.AccessTimestamp)) + return b, nil + }, + DecodeValue: func(value []byte) (e shed.IndexItem, err error) { + e.AccessTimestamp = int64(binary.BigEndian.Uint64(value)) + return e, nil + }, + }) + if err != nil { + return nil, err + } + // Index with keys ordered by access timestamp for garbage collection prioritization. + s.gcIndex, err = db.NewIndex("AccessTimestamp|StoredTimestamp|Address->nil", shed.IndexFuncs{ + EncodeKey: func(fields shed.IndexItem) (key []byte, err error) { + b := make([]byte, 16, 16+len(fields.Address)) + binary.BigEndian.PutUint64(b[:8], uint64(fields.AccessTimestamp)) + binary.BigEndian.PutUint64(b[8:16], uint64(fields.StoreTimestamp)) + key = append(b, fields.Address...) + return key, nil + }, + DecodeKey: func(key []byte) (e shed.IndexItem, err error) { + e.AccessTimestamp = int64(binary.BigEndian.Uint64(key[:8])) + e.StoreTimestamp = int64(binary.BigEndian.Uint64(key[8:16])) + e.Address = key[16:] + return e, nil + }, + EncodeValue: func(fields shed.IndexItem) (value []byte, err error) { + return nil, nil + }, + DecodeValue: func(value []byte) (e shed.IndexItem, err error) { + return e, nil + }, + }) + if err != nil { + return nil, err + } + return s, nil +} + +// Put stores the chunk and sets it store timestamp. +func (s *Store) Put(_ context.Context, ch storage.Chunk) (err error) { + return s.retrievalIndex.Put(shed.IndexItem{ + Address: ch.Address(), + Data: ch.Data(), + StoreTimestamp: time.Now().UTC().UnixNano(), + }) +} + +// Get retrieves a chunk with the provided address. +// It updates access and gc indexes by removing the previous +// items from them and adding new items as keys of index entries +// are changed. +func (s *Store) Get(_ context.Context, addr storage.Address) (c storage.Chunk, err error) { + batch := new(leveldb.Batch) + + // Get the chunk data and storage timestamp. + item, err := s.retrievalIndex.Get(shed.IndexItem{ + Address: addr, + }) + if err != nil { + if err == leveldb.ErrNotFound { + return nil, storage.ErrChunkNotFound + } + return nil, err + } + + // Get the chunk access timestamp. + accessItem, err := s.accessIndex.Get(shed.IndexItem{ + Address: addr, + }) + switch err { + case nil: + // Remove gc index entry if access timestamp is found. + err = s.gcIndex.DeleteInBatch(batch, shed.IndexItem{ + Address: item.Address, + StoreTimestamp: accessItem.AccessTimestamp, + AccessTimestamp: item.StoreTimestamp, + }) + if err != nil { + return nil, err + } + case leveldb.ErrNotFound: + // Access timestamp is not found. Do not do anything. + // This is the firs get request. + default: + return nil, err + } + + // Specify new access timestamp + accessTimestamp := time.Now().UTC().UnixNano() + + // Put new access timestamp in access index. + err = s.accessIndex.PutInBatch(batch, shed.IndexItem{ + Address: addr, + AccessTimestamp: accessTimestamp, + }) + if err != nil { + return nil, err + } + + // Put new access timestamp in gc index. + err = s.gcIndex.PutInBatch(batch, shed.IndexItem{ + Address: item.Address, + AccessTimestamp: accessTimestamp, + StoreTimestamp: item.StoreTimestamp, + }) + if err != nil { + return nil, err + } + + // Increment access counter. + // Currently this information is not used anywhere. + _, err = s.accessCounter.IncInBatch(batch) + if err != nil { + return nil, err + } + + // Write the batch. + err = s.db.WriteBatch(batch) + if err != nil { + return nil, err + } + + // Return the chunk. + return storage.NewChunk(item.Address, item.Data), nil +} + +// CollectGarbage is an example of index iteration. +// It provides no reliable garbage collection functionality. +func (s *Store) CollectGarbage() (err error) { + const maxTrashSize = 100 + maxRounds := 10 // arbitrary number, needs to be calculated + + // Run a few gc rounds. + for roundCount := 0; roundCount < maxRounds; roundCount++ { + var garbageCount int + // New batch for a new cg round. + trash := new(leveldb.Batch) + // Iterate through all index items and break when needed. + err = s.gcIndex.IterateAll(func(item shed.IndexItem) (stop bool, err error) { + // Remove the chunk. + err = s.retrievalIndex.DeleteInBatch(trash, item) + if err != nil { + return false, err + } + // Remove the element in gc index. + err = s.gcIndex.DeleteInBatch(trash, item) + if err != nil { + return false, err + } + // Remove the relation in access index. + err = s.accessIndex.DeleteInBatch(trash, item) + if err != nil { + return false, err + } + garbageCount++ + if garbageCount >= maxTrashSize { + return true, nil + } + return false, nil + }) + if err != nil { + return err + } + if garbageCount == 0 { + return nil + } + err = s.db.WriteBatch(trash) + if err != nil { + return err + } + } + return nil +} + +// GetSchema is an example of retrieveing the most simple +// string from a database field. +func (s *Store) GetSchema() (name string, err error) { + name, err = s.schemaName.Get() + if err == leveldb.ErrNotFound { + return "", nil + } + return name, err +} + +// GetSchema is an example of storing the most simple +// string in a database field. +func (s *Store) PutSchema(name string) (err error) { + return s.schemaName.Put(name) +} + +// Close closes the underlying database. +func (s *Store) Close() error { + return s.db.Close() +} + +// Example_store constructs a simple storage implementation using shed package. +func Example_store() { + dir, err := ioutil.TempDir("", "ephemeral") + if err != nil { + log.Fatal(err) + } + defer os.RemoveAll(dir) + + s, err := New(dir) + if err != nil { + log.Fatal(err) + } + defer s.Close() + + ch := storage.GenerateRandomChunk(1024) + err = s.Put(context.Background(), ch) + if err != nil { + log.Fatal(err) + } + + got, err := s.Get(context.Background(), ch.Address()) + if err != nil { + log.Fatal(err) + } + + fmt.Println(bytes.Equal(got.Data(), ch.Data())) + + //Output: true +} diff --git a/swarm/shed/field_string.go b/swarm/shed/field_string.go new file mode 100644 index 000000000..a7e8f0c75 --- /dev/null +++ b/swarm/shed/field_string.go @@ -0,0 +1,66 @@ +// Copyright 2018 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +package shed + +import ( + "github.com/syndtr/goleveldb/leveldb" +) + +// StringField is the most simple field implementation +// that stores an arbitrary string under a specific LevelDB key. +type StringField struct { + db *DB + key []byte +} + +// NewStringField retruns a new Instance of StringField. +// It validates its name and type against the database schema. +func (db *DB) NewStringField(name string) (f StringField, err error) { + key, err := db.schemaFieldKey(name, "string") + if err != nil { + return f, err + } + return StringField{ + db: db, + key: key, + }, nil +} + +// Get returns a string value from database. +// If the value is not found, an empty string is returned +// an no error. +func (f StringField) Get() (val string, err error) { + b, err := f.db.Get(f.key) + if err != nil { + if err == leveldb.ErrNotFound { + return "", nil + } + return "", err + } + return string(b), nil +} + +// Put stores a string in the database. +func (f StringField) Put(val string) (err error) { + return f.db.Put(f.key, []byte(val)) +} + +// PutInBatch stores a string in a batch that can be +// saved later in database. +func (f StringField) PutInBatch(batch *leveldb.Batch, val string) { + batch.Put(f.key, []byte(val)) +} diff --git a/swarm/shed/field_string_test.go b/swarm/shed/field_string_test.go new file mode 100644 index 000000000..4215075bc --- /dev/null +++ b/swarm/shed/field_string_test.go @@ -0,0 +1,110 @@ +// Copyright 2018 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +package shed + +import ( + "testing" + + "github.com/syndtr/goleveldb/leveldb" +) + +// TestStringField validates put and get operations +// of the StringField. +func TestStringField(t *testing.T) { + db, cleanupFunc := newTestDB(t) + defer cleanupFunc() + + simpleString, err := db.NewStringField("simple-string") + if err != nil { + t.Fatal(err) + } + + t.Run("get empty", func(t *testing.T) { + got, err := simpleString.Get() + if err != nil { + t.Fatal(err) + } + want := "" + if got != want { + t.Errorf("got string %q, want %q", got, want) + } + }) + + t.Run("put", func(t *testing.T) { + want := "simple string value" + err = simpleString.Put(want) + if err != nil { + t.Fatal(err) + } + got, err := simpleString.Get() + if err != nil { + t.Fatal(err) + } + if got != want { + t.Errorf("got string %q, want %q", got, want) + } + + t.Run("overwrite", func(t *testing.T) { + want := "overwritten string value" + err = simpleString.Put(want) + if err != nil { + t.Fatal(err) + } + got, err := simpleString.Get() + if err != nil { + t.Fatal(err) + } + if got != want { + t.Errorf("got string %q, want %q", got, want) + } + }) + }) + + t.Run("put in batch", func(t *testing.T) { + batch := new(leveldb.Batch) + want := "simple string batch value" + simpleString.PutInBatch(batch, want) + err = db.WriteBatch(batch) + if err != nil { + t.Fatal(err) + } + got, err := simpleString.Get() + if err != nil { + t.Fatal(err) + } + if got != want { + t.Errorf("got string %q, want %q", got, want) + } + + t.Run("overwrite", func(t *testing.T) { + batch := new(leveldb.Batch) + want := "overwritten string batch value" + simpleString.PutInBatch(batch, want) + err = db.WriteBatch(batch) + if err != nil { + t.Fatal(err) + } + got, err := simpleString.Get() + if err != nil { + t.Fatal(err) + } + if got != want { + t.Errorf("got string %q, want %q", got, want) + } + }) + }) +} diff --git a/swarm/shed/field_struct.go b/swarm/shed/field_struct.go new file mode 100644 index 000000000..90daee7fc --- /dev/null +++ b/swarm/shed/field_struct.go @@ -0,0 +1,71 @@ +// Copyright 2018 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +package shed + +import ( + "github.com/ethereum/go-ethereum/rlp" + "github.com/syndtr/goleveldb/leveldb" +) + +// StructField is a helper to store complex structure by +// encoding it in RLP format. +type StructField struct { + db *DB + key []byte +} + +// NewStructField returns a new StructField. +// It validates its name and type against the database schema. +func (db *DB) NewStructField(name string) (f StructField, err error) { + key, err := db.schemaFieldKey(name, "struct-rlp") + if err != nil { + return f, err + } + return StructField{ + db: db, + key: key, + }, nil +} + +// Get unmarshals data from the database to a provided val. +// If the data is not found leveldb.ErrNotFound is returned. +func (f StructField) Get(val interface{}) (err error) { + b, err := f.db.Get(f.key) + if err != nil { + return err + } + return rlp.DecodeBytes(b, val) +} + +// Put marshals provided val and saves it to the database. +func (f StructField) Put(val interface{}) (err error) { + b, err := rlp.EncodeToBytes(val) + if err != nil { + return err + } + return f.db.Put(f.key, b) +} + +// PutInBatch marshals provided val and puts it into the batch. +func (f StructField) PutInBatch(batch *leveldb.Batch, val interface{}) (err error) { + b, err := rlp.EncodeToBytes(val) + if err != nil { + return err + } + batch.Put(f.key, b) + return nil +} diff --git a/swarm/shed/field_struct_test.go b/swarm/shed/field_struct_test.go new file mode 100644 index 000000000..cc0be0186 --- /dev/null +++ b/swarm/shed/field_struct_test.go @@ -0,0 +1,127 @@ +// Copyright 2018 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +package shed + +import ( + "testing" + + "github.com/syndtr/goleveldb/leveldb" +) + +// TestStructField validates put and get operations +// of the StructField. +func TestStructField(t *testing.T) { + db, cleanupFunc := newTestDB(t) + defer cleanupFunc() + + complexField, err := db.NewStructField("complex-field") + if err != nil { + t.Fatal(err) + } + + type complexStructure struct { + A string + } + + t.Run("get empty", func(t *testing.T) { + var s complexStructure + err := complexField.Get(&s) + if err != leveldb.ErrNotFound { + t.Fatalf("got error %v, want %v", err, leveldb.ErrNotFound) + } + want := "" + if s.A != want { + t.Errorf("got string %q, want %q", s.A, want) + } + }) + + t.Run("put", func(t *testing.T) { + want := complexStructure{ + A: "simple string value", + } + err = complexField.Put(want) + if err != nil { + t.Fatal(err) + } + var got complexStructure + err = complexField.Get(&got) + if err != nil { + t.Fatal(err) + } + if got.A != want.A { + t.Errorf("got string %q, want %q", got.A, want.A) + } + + t.Run("overwrite", func(t *testing.T) { + want := complexStructure{ + A: "overwritten string value", + } + err = complexField.Put(want) + if err != nil { + t.Fatal(err) + } + var got complexStructure + err = complexField.Get(&got) + if err != nil { + t.Fatal(err) + } + if got.A != want.A { + t.Errorf("got string %q, want %q", got.A, want.A) + } + }) + }) + + t.Run("put in batch", func(t *testing.T) { + batch := new(leveldb.Batch) + want := complexStructure{ + A: "simple string batch value", + } + complexField.PutInBatch(batch, want) + err = db.WriteBatch(batch) + if err != nil { + t.Fatal(err) + } + var got complexStructure + err := complexField.Get(&got) + if err != nil { + t.Fatal(err) + } + if got.A != want.A { + t.Errorf("got string %q, want %q", got, want) + } + + t.Run("overwrite", func(t *testing.T) { + batch := new(leveldb.Batch) + want := complexStructure{ + A: "overwritten string batch value", + } + complexField.PutInBatch(batch, want) + err = db.WriteBatch(batch) + if err != nil { + t.Fatal(err) + } + var got complexStructure + err := complexField.Get(&got) + if err != nil { + t.Fatal(err) + } + if got.A != want.A { + t.Errorf("got string %q, want %q", got, want) + } + }) + }) +} diff --git a/swarm/shed/field_uint64.go b/swarm/shed/field_uint64.go new file mode 100644 index 000000000..80e0069ae --- /dev/null +++ b/swarm/shed/field_uint64.go @@ -0,0 +1,108 @@ +// Copyright 2018 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +package shed + +import ( + "encoding/binary" + + "github.com/syndtr/goleveldb/leveldb" +) + +// Uint64Field provides a way to have a simple counter in the database. +// It transparently encodes uint64 type value to bytes. +type Uint64Field struct { + db *DB + key []byte +} + +// NewUint64Field returns a new Uint64Field. +// It validates its name and type against the database schema. +func (db *DB) NewUint64Field(name string) (f Uint64Field, err error) { + key, err := db.schemaFieldKey(name, "uint64") + if err != nil { + return f, err + } + return Uint64Field{ + db: db, + key: key, + }, nil +} + +// Get retrieves a uint64 value from the database. +// If the value is not found in the database a 0 value +// is returned and no error. +func (f Uint64Field) Get() (val uint64, err error) { + b, err := f.db.Get(f.key) + if err != nil { + if err == leveldb.ErrNotFound { + return 0, nil + } + return 0, err + } + return binary.BigEndian.Uint64(b), nil +} + +// Put encodes uin64 value and stores it in the database. +func (f Uint64Field) Put(val uint64) (err error) { + return f.db.Put(f.key, encodeUint64(val)) +} + +// PutInBatch stores a uint64 value in a batch +// that can be saved later in the database. +func (f Uint64Field) PutInBatch(batch *leveldb.Batch, val uint64) { + batch.Put(f.key, encodeUint64(val)) +} + +// Inc increments a uint64 value in the database. +// This operation is not goroutine save. +func (f Uint64Field) Inc() (val uint64, err error) { + val, err = f.Get() + if err != nil { + if err == leveldb.ErrNotFound { + val = 0 + } else { + return 0, err + } + } + val++ + return val, f.Put(val) +} + +// IncInBatch increments a uint64 value in the batch +// by retreiving a value from the database, not the same batch. +// This operation is not goroutine save. +func (f Uint64Field) IncInBatch(batch *leveldb.Batch) (val uint64, err error) { + val, err = f.Get() + if err != nil { + if err == leveldb.ErrNotFound { + val = 0 + } else { + return 0, err + } + } + val++ + f.PutInBatch(batch, val) + return val, nil +} + +// encode transforms uint64 to 8 byte long +// slice in big endian encoding. +func encodeUint64(val uint64) (b []byte) { + b = make([]byte, 8) + binary.BigEndian.PutUint64(b, val) + return b +} diff --git a/swarm/shed/field_uint64_test.go b/swarm/shed/field_uint64_test.go new file mode 100644 index 000000000..69ade71ba --- /dev/null +++ b/swarm/shed/field_uint64_test.go @@ -0,0 +1,194 @@ +// Copyright 2018 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +package shed + +import ( + "testing" + + "github.com/syndtr/goleveldb/leveldb" +) + +// TestUint64Field validates put and get operations +// of the Uint64Field. +func TestUint64Field(t *testing.T) { + db, cleanupFunc := newTestDB(t) + defer cleanupFunc() + + counter, err := db.NewUint64Field("counter") + if err != nil { + t.Fatal(err) + } + + t.Run("get empty", func(t *testing.T) { + got, err := counter.Get() + if err != nil { + t.Fatal(err) + } + var want uint64 + if got != want { + t.Errorf("got uint64 %v, want %v", got, want) + } + }) + + t.Run("put", func(t *testing.T) { + var want uint64 = 42 + err = counter.Put(want) + if err != nil { + t.Fatal(err) + } + got, err := counter.Get() + if err != nil { + t.Fatal(err) + } + if got != want { + t.Errorf("got uint64 %v, want %v", got, want) + } + + t.Run("overwrite", func(t *testing.T) { + var want uint64 = 84 + err = counter.Put(want) + if err != nil { + t.Fatal(err) + } + got, err := counter.Get() + if err != nil { + t.Fatal(err) + } + if got != want { + t.Errorf("got uint64 %v, want %v", got, want) + } + }) + }) + + t.Run("put in batch", func(t *testing.T) { + batch := new(leveldb.Batch) + var want uint64 = 42 + counter.PutInBatch(batch, want) + err = db.WriteBatch(batch) + if err != nil { + t.Fatal(err) + } + got, err := counter.Get() + if err != nil { + t.Fatal(err) + } + if got != want { + t.Errorf("got uint64 %v, want %v", got, want) + } + + t.Run("overwrite", func(t *testing.T) { + batch := new(leveldb.Batch) + var want uint64 = 84 + counter.PutInBatch(batch, want) + err = db.WriteBatch(batch) + if err != nil { + t.Fatal(err) + } + got, err := counter.Get() + if err != nil { + t.Fatal(err) + } + if got != want { + t.Errorf("got uint64 %v, want %v", got, want) + } + }) + }) +} + +// TestUint64Field_Inc validates Inc operation +// of the Uint64Field. +func TestUint64Field_Inc(t *testing.T) { + db, cleanupFunc := newTestDB(t) + defer cleanupFunc() + + counter, err := db.NewUint64Field("counter") + if err != nil { + t.Fatal(err) + } + + var want uint64 = 1 + got, err := counter.Inc() + if err != nil { + t.Fatal(err) + } + if got != want { + t.Errorf("got uint64 %v, want %v", got, want) + } + + want = 2 + got, err = counter.Inc() + if err != nil { + t.Fatal(err) + } + if got != want { + t.Errorf("got uint64 %v, want %v", got, want) + } +} + +// TestUint64Field_IncInBatch validates IncInBatch operation +// of the Uint64Field. +func TestUint64Field_IncInBatch(t *testing.T) { + db, cleanupFunc := newTestDB(t) + defer cleanupFunc() + + counter, err := db.NewUint64Field("counter") + if err != nil { + t.Fatal(err) + } + + batch := new(leveldb.Batch) + var want uint64 = 1 + got, err := counter.IncInBatch(batch) + if err != nil { + t.Fatal(err) + } + if got != want { + t.Errorf("got uint64 %v, want %v", got, want) + } + err = db.WriteBatch(batch) + if err != nil { + t.Fatal(err) + } + got, err = counter.Get() + if err != nil { + t.Fatal(err) + } + if got != want { + t.Errorf("got uint64 %v, want %v", got, want) + } + + batch2 := new(leveldb.Batch) + want = 2 + got, err = counter.IncInBatch(batch2) + if err != nil { + t.Fatal(err) + } + if got != want { + t.Errorf("got uint64 %v, want %v", got, want) + } + err = db.WriteBatch(batch2) + if err != nil { + t.Fatal(err) + } + got, err = counter.Get() + if err != nil { + t.Fatal(err) + } + if got != want { + t.Errorf("got uint64 %v, want %v", got, want) + } +} diff --git a/swarm/shed/index.go b/swarm/shed/index.go new file mode 100644 index 000000000..ba803e3c2 --- /dev/null +++ b/swarm/shed/index.go @@ -0,0 +1,264 @@ +// Copyright 2018 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +package shed + +import ( + "github.com/syndtr/goleveldb/leveldb" +) + +// IndexItem holds fields relevant to Swarm Chunk data and metadata. +// All information required for swarm storage and operations +// on that storage must be defined here. +// This structure is logically connected to swarm storage, +// the only part of this package that is not generalized, +// mostly for performance reasons. +// +// IndexItem is a type that is used for retrieving, storing and encoding +// chunk data and metadata. It is passed as an argument to Index encoding +// functions, get function and put function. +// But it is also returned with additional data from get function call +// and as the argument in iterator function definition. +type IndexItem struct { + Address []byte + Data []byte + AccessTimestamp int64 + StoreTimestamp int64 + // UseMockStore is a pointer to identify + // an unset state of the field in Join function. + UseMockStore *bool +} + +// Merge is a helper method to construct a new +// IndexItem by filling up fields with default values +// of a particular IndexItem with values from another one. +func (i IndexItem) Merge(i2 IndexItem) (new IndexItem) { + if i.Address == nil { + i.Address = i2.Address + } + if i.Data == nil { + i.Data = i2.Data + } + if i.AccessTimestamp == 0 { + i.AccessTimestamp = i2.AccessTimestamp + } + if i.StoreTimestamp == 0 { + i.StoreTimestamp = i2.StoreTimestamp + } + if i.UseMockStore == nil { + i.UseMockStore = i2.UseMockStore + } + return i +} + +// Index represents a set of LevelDB key value pairs that have common +// prefix. It holds functions for encoding and decoding keys and values +// to provide transparent actions on saved data which inclide: +// - getting a particular IndexItem +// - saving a particular IndexItem +// - iterating over a sorted LevelDB keys +// It implements IndexIteratorInterface interface. +type Index struct { + db *DB + prefix []byte + encodeKeyFunc func(fields IndexItem) (key []byte, err error) + decodeKeyFunc func(key []byte) (e IndexItem, err error) + encodeValueFunc func(fields IndexItem) (value []byte, err error) + decodeValueFunc func(value []byte) (e IndexItem, err error) +} + +// IndexFuncs structure defines functions for encoding and decoding +// LevelDB keys and values for a specific index. +type IndexFuncs struct { + EncodeKey func(fields IndexItem) (key []byte, err error) + DecodeKey func(key []byte) (e IndexItem, err error) + EncodeValue func(fields IndexItem) (value []byte, err error) + DecodeValue func(value []byte) (e IndexItem, err error) +} + +// NewIndex returns a new Index instance with defined name and +// encoding functions. The name must be unique and will be validated +// on database schema for a key prefix byte. +func (db *DB) NewIndex(name string, funcs IndexFuncs) (f Index, err error) { + id, err := db.schemaIndexPrefix(name) + if err != nil { + return f, err + } + prefix := []byte{id} + return Index{ + db: db, + prefix: prefix, + // This function adjusts Index LevelDB key + // by appending the provided index id byte. + // This is needed to avoid collisions between keys of different + // indexes as all index ids are unique. + encodeKeyFunc: func(e IndexItem) (key []byte, err error) { + key, err = funcs.EncodeKey(e) + if err != nil { + return nil, err + } + return append(append(make([]byte, 0, len(key)+1), prefix...), key...), nil + }, + // This function reverses the encodeKeyFunc constructed key + // to transparently work with index keys without their index ids. + // It assumes that index keys are prefixed with only one byte. + decodeKeyFunc: func(key []byte) (e IndexItem, err error) { + return funcs.DecodeKey(key[1:]) + }, + encodeValueFunc: funcs.EncodeValue, + decodeValueFunc: funcs.DecodeValue, + }, nil +} + +// Get accepts key fields represented as IndexItem to retrieve a +// value from the index and return maximum available information +// from the index represented as another IndexItem. +func (f Index) Get(keyFields IndexItem) (out IndexItem, err error) { + key, err := f.encodeKeyFunc(keyFields) + if err != nil { + return out, err + } + value, err := f.db.Get(key) + if err != nil { + return out, err + } + out, err = f.decodeValueFunc(value) + if err != nil { + return out, err + } + return out.Merge(keyFields), nil +} + +// Put accepts IndexItem to encode information from it +// and save it to the database. +func (f Index) Put(i IndexItem) (err error) { + key, err := f.encodeKeyFunc(i) + if err != nil { + return err + } + value, err := f.encodeValueFunc(i) + if err != nil { + return err + } + return f.db.Put(key, value) +} + +// PutInBatch is the same as Put method, but it just +// saves the key/value pair to the batch instead +// directly to the database. +func (f Index) PutInBatch(batch *leveldb.Batch, i IndexItem) (err error) { + key, err := f.encodeKeyFunc(i) + if err != nil { + return err + } + value, err := f.encodeValueFunc(i) + if err != nil { + return err + } + batch.Put(key, value) + return nil +} + +// Delete accepts IndexItem to remove a key/value pair +// from the database based on its fields. +func (f Index) Delete(keyFields IndexItem) (err error) { + key, err := f.encodeKeyFunc(keyFields) + if err != nil { + return err + } + return f.db.Delete(key) +} + +// DeleteInBatch is the same as Delete just the operation +// is performed on the batch instead on the database. +func (f Index) DeleteInBatch(batch *leveldb.Batch, keyFields IndexItem) (err error) { + key, err := f.encodeKeyFunc(keyFields) + if err != nil { + return err + } + batch.Delete(key) + return nil +} + +// IndexIterFunc is a callback on every IndexItem that is decoded +// by iterating on an Index keys. +// By returning a true for stop variable, iteration will +// stop, and by returning the error, that error will be +// propagated to the called iterator method on Index. +type IndexIterFunc func(item IndexItem) (stop bool, err error) + +// IterateAll iterates over all keys of the Index. +func (f Index) IterateAll(fn IndexIterFunc) (err error) { + it := f.db.NewIterator() + defer it.Release() + + for ok := it.Seek(f.prefix); ok; ok = it.Next() { + key := it.Key() + if key[0] != f.prefix[0] { + break + } + keyIndexItem, err := f.decodeKeyFunc(key) + if err != nil { + return err + } + valueIndexItem, err := f.decodeValueFunc(it.Value()) + if err != nil { + return err + } + stop, err := fn(keyIndexItem.Merge(valueIndexItem)) + if err != nil { + return err + } + if stop { + break + } + } + return it.Error() +} + +// IterateFrom iterates over Index keys starting from the key +// encoded from the provided IndexItem. +func (f Index) IterateFrom(start IndexItem, fn IndexIterFunc) (err error) { + startKey, err := f.encodeKeyFunc(start) + if err != nil { + return err + } + it := f.db.NewIterator() + defer it.Release() + + for ok := it.Seek(startKey); ok; ok = it.Next() { + key := it.Key() + if key[0] != f.prefix[0] { + break + } + keyIndexItem, err := f.decodeKeyFunc(key) + if err != nil { + return err + } + valueIndexItem, err := f.decodeValueFunc(it.Value()) + if err != nil { + return err + } + stop, err := fn(keyIndexItem.Merge(valueIndexItem)) + if err != nil { + return err + } + if stop { + break + } + } + return it.Error() +} diff --git a/swarm/shed/index_test.go b/swarm/shed/index_test.go new file mode 100644 index 000000000..ba82216df --- /dev/null +++ b/swarm/shed/index_test.go @@ -0,0 +1,426 @@ +// Copyright 2018 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +package shed + +import ( + "bytes" + "encoding/binary" + "fmt" + "sort" + "testing" + "time" + + "github.com/syndtr/goleveldb/leveldb" +) + +// Index functions for the index that is used in tests in this file. +var retrievalIndexFuncs = IndexFuncs{ + EncodeKey: func(fields IndexItem) (key []byte, err error) { + return fields.Address, nil + }, + DecodeKey: func(key []byte) (e IndexItem, err error) { + e.Address = key + return e, nil + }, + EncodeValue: func(fields IndexItem) (value []byte, err error) { + b := make([]byte, 8) + binary.BigEndian.PutUint64(b, uint64(fields.StoreTimestamp)) + value = append(b, fields.Data...) + return value, nil + }, + DecodeValue: func(value []byte) (e IndexItem, err error) { + e.StoreTimestamp = int64(binary.BigEndian.Uint64(value[:8])) + e.Data = value[8:] + return e, nil + }, +} + +// TestIndex validates put, get and delete functions of the Index implementation. +func TestIndex(t *testing.T) { + db, cleanupFunc := newTestDB(t) + defer cleanupFunc() + + index, err := db.NewIndex("retrieval", retrievalIndexFuncs) + if err != nil { + t.Fatal(err) + } + + t.Run("put", func(t *testing.T) { + want := IndexItem{ + Address: []byte("put-hash"), + Data: []byte("DATA"), + StoreTimestamp: time.Now().UTC().UnixNano(), + } + + err := index.Put(want) + if err != nil { + t.Fatal(err) + } + got, err := index.Get(IndexItem{ + Address: want.Address, + }) + if err != nil { + t.Fatal(err) + } + checkIndexItem(t, got, want) + + t.Run("overwrite", func(t *testing.T) { + want := IndexItem{ + Address: []byte("put-hash"), + Data: []byte("New DATA"), + StoreTimestamp: time.Now().UTC().UnixNano(), + } + + err = index.Put(want) + if err != nil { + t.Fatal(err) + } + got, err := index.Get(IndexItem{ + Address: want.Address, + }) + if err != nil { + t.Fatal(err) + } + checkIndexItem(t, got, want) + }) + }) + + t.Run("put in batch", func(t *testing.T) { + want := IndexItem{ + Address: []byte("put-in-batch-hash"), + Data: []byte("DATA"), + StoreTimestamp: time.Now().UTC().UnixNano(), + } + + batch := new(leveldb.Batch) + index.PutInBatch(batch, want) + err := db.WriteBatch(batch) + if err != nil { + t.Fatal(err) + } + got, err := index.Get(IndexItem{ + Address: want.Address, + }) + if err != nil { + t.Fatal(err) + } + checkIndexItem(t, got, want) + + t.Run("overwrite", func(t *testing.T) { + want := IndexItem{ + Address: []byte("put-in-batch-hash"), + Data: []byte("New DATA"), + StoreTimestamp: time.Now().UTC().UnixNano(), + } + + batch := new(leveldb.Batch) + index.PutInBatch(batch, want) + db.WriteBatch(batch) + if err != nil { + t.Fatal(err) + } + got, err := index.Get(IndexItem{ + Address: want.Address, + }) + if err != nil { + t.Fatal(err) + } + checkIndexItem(t, got, want) + }) + }) + + t.Run("put in batch twice", func(t *testing.T) { + // ensure that the last item of items with the same db keys + // is actually saved + batch := new(leveldb.Batch) + address := []byte("put-in-batch-twice-hash") + + // put the first item + index.PutInBatch(batch, IndexItem{ + Address: address, + Data: []byte("DATA"), + StoreTimestamp: time.Now().UTC().UnixNano(), + }) + + want := IndexItem{ + Address: address, + Data: []byte("New DATA"), + StoreTimestamp: time.Now().UTC().UnixNano(), + } + // then put the item that will produce the same key + // but different value in the database + index.PutInBatch(batch, want) + db.WriteBatch(batch) + if err != nil { + t.Fatal(err) + } + got, err := index.Get(IndexItem{ + Address: address, + }) + if err != nil { + t.Fatal(err) + } + checkIndexItem(t, got, want) + }) + + t.Run("delete", func(t *testing.T) { + want := IndexItem{ + Address: []byte("delete-hash"), + Data: []byte("DATA"), + StoreTimestamp: time.Now().UTC().UnixNano(), + } + + err := index.Put(want) + if err != nil { + t.Fatal(err) + } + got, err := index.Get(IndexItem{ + Address: want.Address, + }) + if err != nil { + t.Fatal(err) + } + checkIndexItem(t, got, want) + + err = index.Delete(IndexItem{ + Address: want.Address, + }) + if err != nil { + t.Fatal(err) + } + + wantErr := leveldb.ErrNotFound + got, err = index.Get(IndexItem{ + Address: want.Address, + }) + if err != wantErr { + t.Fatalf("got error %v, want %v", err, wantErr) + } + }) + + t.Run("delete in batch", func(t *testing.T) { + want := IndexItem{ + Address: []byte("delete-in-batch-hash"), + Data: []byte("DATA"), + StoreTimestamp: time.Now().UTC().UnixNano(), + } + + err := index.Put(want) + if err != nil { + t.Fatal(err) + } + got, err := index.Get(IndexItem{ + Address: want.Address, + }) + if err != nil { + t.Fatal(err) + } + checkIndexItem(t, got, want) + + batch := new(leveldb.Batch) + index.DeleteInBatch(batch, IndexItem{ + Address: want.Address, + }) + err = db.WriteBatch(batch) + if err != nil { + t.Fatal(err) + } + + wantErr := leveldb.ErrNotFound + got, err = index.Get(IndexItem{ + Address: want.Address, + }) + if err != wantErr { + t.Fatalf("got error %v, want %v", err, wantErr) + } + }) +} + +// TestIndex_iterate validates index iterator functions for correctness. +func TestIndex_iterate(t *testing.T) { + db, cleanupFunc := newTestDB(t) + defer cleanupFunc() + + index, err := db.NewIndex("retrieval", retrievalIndexFuncs) + if err != nil { + t.Fatal(err) + } + + items := []IndexItem{ + { + Address: []byte("iterate-hash-01"), + Data: []byte("data80"), + }, + { + Address: []byte("iterate-hash-03"), + Data: []byte("data22"), + }, + { + Address: []byte("iterate-hash-05"), + Data: []byte("data41"), + }, + { + Address: []byte("iterate-hash-02"), + Data: []byte("data84"), + }, + { + Address: []byte("iterate-hash-06"), + Data: []byte("data1"), + }, + } + batch := new(leveldb.Batch) + for _, i := range items { + index.PutInBatch(batch, i) + } + err = db.WriteBatch(batch) + if err != nil { + t.Fatal(err) + } + item04 := IndexItem{ + Address: []byte("iterate-hash-04"), + Data: []byte("data0"), + } + err = index.Put(item04) + if err != nil { + t.Fatal(err) + } + items = append(items, item04) + + sort.SliceStable(items, func(i, j int) bool { + return bytes.Compare(items[i].Address, items[j].Address) < 0 + }) + + t.Run("all", func(t *testing.T) { + var i int + err := index.IterateAll(func(item IndexItem) (stop bool, err error) { + if i > len(items)-1 { + return true, fmt.Errorf("got unexpected index item: %#v", item) + } + want := items[i] + checkIndexItem(t, item, want) + i++ + return false, nil + }) + if err != nil { + t.Fatal(err) + } + }) + + t.Run("from", func(t *testing.T) { + startIndex := 2 + i := startIndex + err := index.IterateFrom(items[startIndex], func(item IndexItem) (stop bool, err error) { + if i > len(items)-1 { + return true, fmt.Errorf("got unexpected index item: %#v", item) + } + want := items[i] + checkIndexItem(t, item, want) + i++ + return false, nil + }) + if err != nil { + t.Fatal(err) + } + }) + + t.Run("stop", func(t *testing.T) { + var i int + stopIndex := 3 + var count int + err := index.IterateAll(func(item IndexItem) (stop bool, err error) { + if i > len(items)-1 { + return true, fmt.Errorf("got unexpected index item: %#v", item) + } + want := items[i] + checkIndexItem(t, item, want) + count++ + if i == stopIndex { + return true, nil + } + i++ + return false, nil + }) + if err != nil { + t.Fatal(err) + } + wantItemsCount := stopIndex + 1 + if count != wantItemsCount { + t.Errorf("got %v items, expected %v", count, wantItemsCount) + } + }) + + t.Run("no overflow", func(t *testing.T) { + secondIndex, err := db.NewIndex("second-index", retrievalIndexFuncs) + if err != nil { + t.Fatal(err) + } + + secondIndexItem := IndexItem{ + Address: []byte("iterate-hash-10"), + Data: []byte("data-second"), + } + err = secondIndex.Put(secondIndexItem) + if err != nil { + t.Fatal(err) + } + + var i int + err = index.IterateAll(func(item IndexItem) (stop bool, err error) { + if i > len(items)-1 { + return true, fmt.Errorf("got unexpected index item: %#v", item) + } + want := items[i] + checkIndexItem(t, item, want) + i++ + return false, nil + }) + if err != nil { + t.Fatal(err) + } + + i = 0 + err = secondIndex.IterateAll(func(item IndexItem) (stop bool, err error) { + if i > 1 { + return true, fmt.Errorf("got unexpected index item: %#v", item) + } + checkIndexItem(t, item, secondIndexItem) + i++ + return false, nil + }) + if err != nil { + t.Fatal(err) + } + }) +} + +// checkIndexItem is a test helper function that compares if two Index items are the same. +func checkIndexItem(t *testing.T, got, want IndexItem) { + t.Helper() + + if !bytes.Equal(got.Address, want.Address) { + t.Errorf("got hash %q, expected %q", string(got.Address), string(want.Address)) + } + if !bytes.Equal(got.Data, want.Data) { + t.Errorf("got data %q, expected %q", string(got.Data), string(want.Data)) + } + if got.StoreTimestamp != want.StoreTimestamp { + t.Errorf("got store timestamp %v, expected %v", got.StoreTimestamp, want.StoreTimestamp) + } + if got.AccessTimestamp != want.AccessTimestamp { + t.Errorf("got access timestamp %v, expected %v", got.AccessTimestamp, want.AccessTimestamp) + } +} diff --git a/swarm/shed/schema.go b/swarm/shed/schema.go new file mode 100644 index 000000000..cfb7c6d64 --- /dev/null +++ b/swarm/shed/schema.go @@ -0,0 +1,134 @@ +// Copyright 2018 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +package shed + +import ( + "encoding/json" + "errors" + "fmt" +) + +var ( + // LevelDB key value for storing the schema. + keySchema = []byte{0} + // LevelDB key prefix for all field type. + // LevelDB keys will be constructed by appending name values to this prefix. + keyPrefixFields byte = 1 + // LevelDB key prefix from which indexing keys start. + // Every index has its own key prefix and this value defines the first one. + keyPrefixIndexStart byte = 2 // Q: or maybe a higher number like 7, to have more space for potential specific perfixes +) + +// schema is used to serialize known database structure information. +type schema struct { + Fields map[string]fieldSpec `json:"fields"` // keys are field names + Indexes map[byte]indexSpec `json:"indexes"` // keys are index prefix bytes +} + +// fieldSpec holds information about a particular field. +// It does not need Name field as it is contained in the +// schema.Field map key. +type fieldSpec struct { + Type string `json:"type"` +} + +// indxSpec holds information about a particular index. +// It does not contain index type, as indexes do not have type. +type indexSpec struct { + Name string `json:"name"` +} + +// schemaFieldKey retrives the complete LevelDB key for +// a particular field form the schema definition. +func (db *DB) schemaFieldKey(name, fieldType string) (key []byte, err error) { + if name == "" { + return nil, errors.New("field name can not be blank") + } + if fieldType == "" { + return nil, errors.New("field type can not be blank") + } + s, err := db.getSchema() + if err != nil { + return nil, err + } + var found bool + for n, f := range s.Fields { + if n == name { + if f.Type != fieldType { + return nil, fmt.Errorf("field %q of type %q stored as %q in db", name, fieldType, f.Type) + } + break + } + } + if !found { + s.Fields[name] = fieldSpec{ + Type: fieldType, + } + err := db.putSchema(s) + if err != nil { + return nil, err + } + } + return append([]byte{keyPrefixFields}, []byte(name)...), nil +} + +// schemaIndexID retrieves the complete LevelDB prefix for +// a particular index. +func (db *DB) schemaIndexPrefix(name string) (id byte, err error) { + if name == "" { + return 0, errors.New("index name can not be blank") + } + s, err := db.getSchema() + if err != nil { + return 0, err + } + nextID := keyPrefixIndexStart + for i, f := range s.Indexes { + if i >= nextID { + nextID = i + 1 + } + if f.Name == name { + return i, nil + } + } + id = nextID + s.Indexes[id] = indexSpec{ + Name: name, + } + return id, db.putSchema(s) +} + +// getSchema retrieves the complete schema from +// the database. +func (db *DB) getSchema() (s schema, err error) { + b, err := db.Get(keySchema) + if err != nil { + return s, err + } + err = json.Unmarshal(b, &s) + return s, err +} + +// putSchema stores the complete schema to +// the database. +func (db *DB) putSchema(s schema) (err error) { + b, err := json.Marshal(s) + if err != nil { + return err + } + return db.Put(keySchema, b) +} diff --git a/swarm/shed/schema_test.go b/swarm/shed/schema_test.go new file mode 100644 index 000000000..a0c1838c8 --- /dev/null +++ b/swarm/shed/schema_test.go @@ -0,0 +1,126 @@ +// Copyright 2018 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +package shed + +import ( + "bytes" + "testing" +) + +// TestDB_schemaFieldKey validates correctness of schemaFieldKey. +func TestDB_schemaFieldKey(t *testing.T) { + db, cleanupFunc := newTestDB(t) + defer cleanupFunc() + + t.Run("empty name or type", func(t *testing.T) { + _, err := db.schemaFieldKey("", "") + if err == nil { + t.Errorf("error not returned, but expected") + } + _, err = db.schemaFieldKey("", "type") + if err == nil { + t.Errorf("error not returned, but expected") + } + + _, err = db.schemaFieldKey("test", "") + if err == nil { + t.Errorf("error not returned, but expected") + } + }) + + t.Run("same field", func(t *testing.T) { + key1, err := db.schemaFieldKey("test", "undefined") + if err != nil { + t.Fatal(err) + } + + key2, err := db.schemaFieldKey("test", "undefined") + if err != nil { + t.Fatal(err) + } + + if !bytes.Equal(key1, key2) { + t.Errorf("schema keys for the same field name are not the same: %q, %q", string(key1), string(key2)) + } + }) + + t.Run("different fields", func(t *testing.T) { + key1, err := db.schemaFieldKey("test1", "undefined") + if err != nil { + t.Fatal(err) + } + + key2, err := db.schemaFieldKey("test2", "undefined") + if err != nil { + t.Fatal(err) + } + + if bytes.Equal(key1, key2) { + t.Error("schema keys for the same field name are the same, but must not be") + } + }) + + t.Run("same field name different types", func(t *testing.T) { + _, err := db.schemaFieldKey("the-field", "one-type") + if err != nil { + t.Fatal(err) + } + + _, err = db.schemaFieldKey("the-field", "another-type") + if err == nil { + t.Errorf("error not returned, but expected") + } + }) +} + +// TestDB_schemaIndexPrefix validates correctness of schemaIndexPrefix. +func TestDB_schemaIndexPrefix(t *testing.T) { + db, cleanupFunc := newTestDB(t) + defer cleanupFunc() + + t.Run("same name", func(t *testing.T) { + id1, err := db.schemaIndexPrefix("test") + if err != nil { + t.Fatal(err) + } + + id2, err := db.schemaIndexPrefix("test") + if err != nil { + t.Fatal(err) + } + + if id1 != id2 { + t.Errorf("schema keys for the same field name are not the same: %v, %v", id1, id2) + } + }) + + t.Run("different names", func(t *testing.T) { + id1, err := db.schemaIndexPrefix("test1") + if err != nil { + t.Fatal(err) + } + + id2, err := db.schemaIndexPrefix("test2") + if err != nil { + t.Fatal(err) + } + + if id1 == id2 { + t.Error("schema ids for the same index name are the same, but must not be") + } + }) +} diff --git a/swarm/storage/mock/db/db.go b/swarm/storage/mock/db/db.go index 43bfa24f0..73ae199e8 100644 --- a/swarm/storage/mock/db/db.go +++ b/swarm/storage/mock/db/db.go @@ -86,6 +86,13 @@ func (s *GlobalStore) Put(addr common.Address, key []byte, data []byte) error { return s.db.Write(batch, nil) } +// Delete removes the chunk reference to node with address addr. +func (s *GlobalStore) Delete(addr common.Address, key []byte) error { + batch := new(leveldb.Batch) + batch.Delete(nodeDBKey(addr, key)) + return s.db.Write(batch, nil) +} + // HasKey returns whether a node with addr contains the key. func (s *GlobalStore) HasKey(addr common.Address, key []byte) bool { has, err := s.db.Has(nodeDBKey(addr, key), nil) diff --git a/swarm/storage/mock/mem/mem.go b/swarm/storage/mock/mem/mem.go index 8878309d0..3a0a2beb8 100644 --- a/swarm/storage/mock/mem/mem.go +++ b/swarm/storage/mock/mem/mem.go @@ -83,6 +83,22 @@ func (s *GlobalStore) Put(addr common.Address, key []byte, data []byte) error { return nil } +// Delete removes the chunk data for node with address addr. +func (s *GlobalStore) Delete(addr common.Address, key []byte) error { + s.mu.Lock() + defer s.mu.Unlock() + + var count int + if _, ok := s.nodes[string(key)]; ok { + delete(s.nodes[string(key)], addr) + count = len(s.nodes[string(key)]) + } + if count == 0 { + delete(s.data, string(key)) + } + return nil +} + // HasKey returns whether a node with addr contains the key. func (s *GlobalStore) HasKey(addr common.Address, key []byte) bool { s.mu.Lock() diff --git a/swarm/storage/mock/mock.go b/swarm/storage/mock/mock.go index 81340f927..1fb71b70a 100644 --- a/swarm/storage/mock/mock.go +++ b/swarm/storage/mock/mock.go @@ -70,6 +70,12 @@ func (n *NodeStore) Put(key []byte, data []byte) error { return n.store.Put(n.addr, key, data) } +// Delete removes chunk data for a key for a node that has the address +// provided on NodeStore initialization. +func (n *NodeStore) Delete(key []byte) error { + return n.store.Delete(n.addr, key) +} + // GlobalStorer defines methods for mock db store // that stores chunk data for all swarm nodes. // It is used in tests to construct mock NodeStores @@ -77,6 +83,7 @@ func (n *NodeStore) Put(key []byte, data []byte) error { type GlobalStorer interface { Get(addr common.Address, key []byte) (data []byte, err error) Put(addr common.Address, key []byte, data []byte) error + Delete(addr common.Address, key []byte) error HasKey(addr common.Address, key []byte) bool // NewNodeStore creates an instance of NodeStore // to be used by a single swarm node with diff --git a/swarm/storage/mock/rpc/rpc.go b/swarm/storage/mock/rpc/rpc.go index 6e735f698..8cd6c83a7 100644 --- a/swarm/storage/mock/rpc/rpc.go +++ b/swarm/storage/mock/rpc/rpc.go @@ -73,6 +73,12 @@ func (s *GlobalStore) Put(addr common.Address, key []byte, data []byte) error { return err } +// Delete calls a Delete method to RPC server. +func (s *GlobalStore) Delete(addr common.Address, key []byte) error { + err := s.client.Call(nil, "mockStore_delete", addr, key) + return err +} + // HasKey calls a HasKey method to RPC server. func (s *GlobalStore) HasKey(addr common.Address, key []byte) bool { var has bool diff --git a/swarm/storage/mock/test/test.go b/swarm/storage/mock/test/test.go index 02da3af55..10180985f 100644 --- a/swarm/storage/mock/test/test.go +++ b/swarm/storage/mock/test/test.go @@ -72,6 +72,31 @@ func MockStore(t *testing.T, globalStore mock.GlobalStorer, n int) { } } } + t.Run("delete", func(t *testing.T) { + chunkAddr := storage.Address([]byte("1234567890abcd")) + for _, addr := range addrs { + err := globalStore.Put(addr, chunkAddr, []byte("data")) + if err != nil { + t.Fatalf("put data to store %s key %s: %v", addr.Hex(), chunkAddr.Hex(), err) + } + } + firstNodeAddr := addrs[0] + if err := globalStore.Delete(firstNodeAddr, chunkAddr); err != nil { + t.Fatalf("delete from store %s key %s: %v", firstNodeAddr.Hex(), chunkAddr.Hex(), err) + } + for i, addr := range addrs { + _, err := globalStore.Get(addr, chunkAddr) + if i == 0 { + if err != mock.ErrNotFound { + t.Errorf("get data from store %s key %s: expected mock.ErrNotFound error, got %v", addr.Hex(), chunkAddr.Hex(), err) + } + } else { + if err != nil { + t.Errorf("get data from store %s key %s: %v", addr.Hex(), chunkAddr.Hex(), err) + } + } + } + }) }) t.Run("NodeStore", func(t *testing.T) { @@ -114,6 +139,34 @@ func MockStore(t *testing.T, globalStore mock.GlobalStorer, n int) { } } } + t.Run("delete", func(t *testing.T) { + chunkAddr := storage.Address([]byte("1234567890abcd")) + var chosenStore *mock.NodeStore + for addr, store := range nodes { + if chosenStore == nil { + chosenStore = store + } + err := store.Put(chunkAddr, []byte("data")) + if err != nil { + t.Fatalf("put data to store %s key %s: %v", addr.Hex(), chunkAddr.Hex(), err) + } + } + if err := chosenStore.Delete(chunkAddr); err != nil { + t.Fatalf("delete key %s: %v", chunkAddr.Hex(), err) + } + for addr, store := range nodes { + _, err := store.Get(chunkAddr) + if store == chosenStore { + if err != mock.ErrNotFound { + t.Errorf("get data from store %s key %s: expected mock.ErrNotFound error, got %v", addr.Hex(), chunkAddr.Hex(), err) + } + } else { + if err != nil { + t.Errorf("get data from store %s key %s: %v", addr.Hex(), chunkAddr.Hex(), err) + } + } + } + }) }) } -- 2.45.2 From 695a5cce1e4c0db173dd2a798ed0dc14ee9ca2a1 Mon Sep 17 00:00:00 2001 From: holisticode Date: Mon, 26 Nov 2018 13:34:40 -0500 Subject: [PATCH 25/48] Increase bzz version (#18184) * swarm/network/stream/: added stream protocol version match tests * Increase BZZ version due to streamer version change; version tests * swarm/network: increased hive and test protocol version --- swarm/network/protocol.go | 4 +- swarm/network/protocol_test.go | 2 +- swarm/network/stream/syncer_test.go | 167 ++++++++++++++++++++++++++++ 3 files changed, 170 insertions(+), 3 deletions(-) diff --git a/swarm/network/protocol.go b/swarm/network/protocol.go index 66ae94a88..4b9b28cdc 100644 --- a/swarm/network/protocol.go +++ b/swarm/network/protocol.go @@ -44,7 +44,7 @@ const ( // BzzSpec is the spec of the generic swarm handshake var BzzSpec = &protocols.Spec{ Name: "bzz", - Version: 7, + Version: 8, MaxMsgSize: 10 * 1024 * 1024, Messages: []interface{}{ HandshakeMsg{}, @@ -54,7 +54,7 @@ var BzzSpec = &protocols.Spec{ // DiscoverySpec is the spec for the bzz discovery subprotocols var DiscoverySpec = &protocols.Spec{ Name: "hive", - Version: 6, + Version: 8, MaxMsgSize: 10 * 1024 * 1024, Messages: []interface{}{ peersMsg{}, diff --git a/swarm/network/protocol_test.go b/swarm/network/protocol_test.go index f0d266628..53ceda744 100644 --- a/swarm/network/protocol_test.go +++ b/swarm/network/protocol_test.go @@ -31,7 +31,7 @@ import ( ) const ( - TestProtocolVersion = 7 + TestProtocolVersion = 8 TestProtocolNetworkID = 3 ) diff --git a/swarm/network/stream/syncer_test.go b/swarm/network/stream/syncer_test.go index df1a98c92..3e3cee18d 100644 --- a/swarm/network/stream/syncer_test.go +++ b/swarm/network/stream/syncer_test.go @@ -234,3 +234,170 @@ func testSyncBetweenNodes(t *testing.T, nodes, conns, chunkCount int, skipCheck t.Fatal(result.Error) } } + +//TestSameVersionID just checks that if the version is not changed, +//then streamer peers see each other +func TestSameVersionID(t *testing.T) { + //test version ID + v := uint(1) + sim := simulation.New(map[string]simulation.ServiceFunc{ + "streamer": func(ctx *adapters.ServiceContext, bucket *sync.Map) (s node.Service, cleanup func(), err error) { + var store storage.ChunkStore + var datadir string + + node := ctx.Config.Node() + addr := network.NewAddr(node) + + store, datadir, err = createTestLocalStorageForID(node.ID(), addr) + if err != nil { + return nil, nil, err + } + bucket.Store(bucketKeyStore, store) + cleanup = func() { + store.Close() + os.RemoveAll(datadir) + } + localStore := store.(*storage.LocalStore) + netStore, err := storage.NewNetStore(localStore, nil) + if err != nil { + return nil, nil, err + } + bucket.Store(bucketKeyDB, netStore) + kad := network.NewKademlia(addr.Over(), network.NewKadParams()) + delivery := NewDelivery(kad, netStore) + netStore.NewNetFetcherFunc = network.NewFetcherFactory(delivery.RequestFromPeers, true).New + + bucket.Store(bucketKeyDelivery, delivery) + + r := NewRegistry(addr.ID(), delivery, netStore, state.NewInmemoryStore(), &RegistryOptions{ + Retrieval: RetrievalDisabled, + Syncing: SyncingAutoSubscribe, + }, nil) + //assign to each node the same version ID + r.spec.Version = v + + bucket.Store(bucketKeyRegistry, r) + + return r, cleanup, nil + + }, + }) + defer sim.Close() + + //connect just two nodes + log.Info("Adding nodes to simulation") + _, err := sim.AddNodesAndConnectChain(2) + if err != nil { + t.Fatal(err) + } + + log.Info("Starting simulation") + ctx := context.Background() + //make sure they have time to connect + time.Sleep(200 * time.Millisecond) + result := sim.Run(ctx, func(ctx context.Context, sim *simulation.Simulation) error { + //get the pivot node's filestore + nodes := sim.UpNodeIDs() + + item, ok := sim.NodeItem(nodes[0], bucketKeyRegistry) + if !ok { + return fmt.Errorf("No filestore") + } + registry := item.(*Registry) + + //the peers should connect, thus getting the peer should not return nil + if registry.getPeer(nodes[1]) == nil { + t.Fatal("Expected the peer to not be nil, but it is") + } + return nil + }) + if result.Error != nil { + t.Fatal(result.Error) + } + log.Info("Simulation ended") +} + +//TestDifferentVersionID proves that if the streamer protocol version doesn't match, +//then the peers are not connected at streamer level +func TestDifferentVersionID(t *testing.T) { + //create a variable to hold the version ID + v := uint(0) + sim := simulation.New(map[string]simulation.ServiceFunc{ + "streamer": func(ctx *adapters.ServiceContext, bucket *sync.Map) (s node.Service, cleanup func(), err error) { + var store storage.ChunkStore + var datadir string + + node := ctx.Config.Node() + addr := network.NewAddr(node) + + store, datadir, err = createTestLocalStorageForID(node.ID(), addr) + if err != nil { + return nil, nil, err + } + bucket.Store(bucketKeyStore, store) + cleanup = func() { + store.Close() + os.RemoveAll(datadir) + } + localStore := store.(*storage.LocalStore) + netStore, err := storage.NewNetStore(localStore, nil) + if err != nil { + return nil, nil, err + } + bucket.Store(bucketKeyDB, netStore) + kad := network.NewKademlia(addr.Over(), network.NewKadParams()) + delivery := NewDelivery(kad, netStore) + netStore.NewNetFetcherFunc = network.NewFetcherFactory(delivery.RequestFromPeers, true).New + + bucket.Store(bucketKeyDelivery, delivery) + + r := NewRegistry(addr.ID(), delivery, netStore, state.NewInmemoryStore(), &RegistryOptions{ + Retrieval: RetrievalDisabled, + Syncing: SyncingAutoSubscribe, + }, nil) + + //increase the version ID for each node + v++ + r.spec.Version = v + + bucket.Store(bucketKeyRegistry, r) + + return r, cleanup, nil + + }, + }) + defer sim.Close() + + //connect the nodes + log.Info("Adding nodes to simulation") + _, err := sim.AddNodesAndConnectChain(2) + if err != nil { + t.Fatal(err) + } + + log.Info("Starting simulation") + ctx := context.Background() + //make sure they have time to connect + time.Sleep(200 * time.Millisecond) + result := sim.Run(ctx, func(ctx context.Context, sim *simulation.Simulation) error { + //get the pivot node's filestore + nodes := sim.UpNodeIDs() + + item, ok := sim.NodeItem(nodes[0], bucketKeyRegistry) + if !ok { + return fmt.Errorf("No filestore") + } + registry := item.(*Registry) + + //getting the other peer should fail due to the different version numbers + if registry.getPeer(nodes[1]) != nil { + t.Fatal("Expected the peer to be nil, but it is not") + } + return nil + }) + if result.Error != nil { + t.Fatal(result.Error) + } + log.Info("Simulation ended") + +} -- 2.45.2 From 976bff614a201ff33af0bc0286fb568c20b4c725 Mon Sep 17 00:00:00 2001 From: Ian Norden Date: Mon, 26 Nov 2018 02:37:25 -0600 Subject: [PATCH 26/48] work towards persisting created statediffs in ipfs; based off github.com/vulcanize/eth-block-extractor --- core/blockchain.go | 21 ++-- statediff/builder.go | 96 +++++++++---------- statediff/config.go | 15 +-- statediff/extractor.go | 33 ++++--- statediff/helpers.go | 17 +--- statediff/ipfs/adder.go | 58 +++++++++++ statediff/ipfs/dag_putter.go | 79 +++++++++++++++ statediff/{persister.go => ipfs/helpers.go} | 32 +++---- statediff/ipfs/node.go | 78 +++++++++++++++ statediff/publisher.go | 63 ++++++++++++ .../{persister_test.go => publisher_test.go} | 0 statediff/{statediff.go => struct.go} | 2 +- 12 files changed, 383 insertions(+), 111 deletions(-) create mode 100644 statediff/ipfs/adder.go create mode 100644 statediff/ipfs/dag_putter.go rename statediff/{persister.go => ipfs/helpers.go} (71%) create mode 100644 statediff/ipfs/node.go create mode 100644 statediff/publisher.go rename statediff/{persister_test.go => publisher_test.go} (100%) rename statediff/{statediff.go => struct.go} (98%) diff --git a/core/blockchain.go b/core/blockchain.go index fed6b4b74..77295b888 100644 --- a/core/blockchain.go +++ b/core/blockchain.go @@ -20,7 +20,6 @@ package core import ( "errors" "fmt" - "github.com/ethereum/go-ethereum/statediff" "io" "math/big" mrand "math/rand" @@ -45,6 +44,7 @@ import ( "github.com/ethereum/go-ethereum/rlp" "github.com/ethereum/go-ethereum/trie" "github.com/hashicorp/golang-lru" + "github.com/ethereum/go-ethereum/statediff" ) var ( @@ -73,7 +73,7 @@ type CacheConfig struct { TrieCleanLimit int // Memory allowance (MB) to use for caching trie nodes in memory TrieDirtyLimit int // Memory limit (MB) at which to start flushing dirty trie nodes to disk TrieTimeLimit time.Duration // Time limit after which to flush the current in-memory trie to disk - StateDiff bool // Whether or not to calculate and persist state diffs + StateDiff statediff.Config // Settings for state diff extraction } // BlockChain represents the canonical chain given a database with a genesis @@ -177,11 +177,14 @@ func NewBlockChain(db ethdb.Database, cacheConfig *CacheConfig, chainConfig *par bc.SetValidator(NewBlockValidator(chainConfig, bc, engine)) bc.SetProcessor(NewStateProcessor(chainConfig, bc, engine)) - if cacheConfig.StateDiff { - bc.diffExtractor = statediff.NewExtractor(db) + var err error + if cacheConfig.StateDiff.On { + bc.diffExtractor, err = statediff.NewExtractor(db, cacheConfig.StateDiff) + if err != nil { + return nil, err + } } - var err error bc.hc, err = NewHeaderChain(db, chainConfig, engine, bc.getProcInterrupt) if err != nil { return nil, err @@ -1214,8 +1217,12 @@ func (bc *BlockChain) insertChain(chain types.Blocks) (int, []interface{}, []*ty proctime := time.Since(bstart) // If extracting statediffs, do so now - if bc.cacheConfig.StateDiff { - bc.diffExtractor.Extract(*parent, *block) + if bc.cacheConfig.StateDiff.On { + // Currently not doing anything with returned cid... + _, err = bc.diffExtractor.ExtractStateDiff(*parent, *block) + if err != nil { + return i, events, coalescedLogs, err + } } // Write the block to the chain and get the status. diff --git a/statediff/builder.go b/statediff/builder.go index 9017c240d..433353591 100644 --- a/statediff/builder.go +++ b/statediff/builder.go @@ -78,7 +78,7 @@ func (sdb *builder) BuildStateDiff(oldStateRoot, newStateRoot common.Hash, block updatedKeys := findIntersection(createKeys, deleteKeys) // Build and return the statediff - updatedAccounts, err := sdb.buildDiffIncremental(creations, deletions, &updatedKeys) + updatedAccounts, err := sdb.buildDiffIncremental(creations, deletions, updatedKeys) if err != nil { return nil, err } @@ -133,6 +133,7 @@ func (sdb *builder) collectDiffNodes(a, b trie.NodeIterator) (map[common.Address break } } + return diffAccounts, nil } @@ -140,66 +141,63 @@ func (sdb *builder) buildDiffEventual(accounts map[common.Address]*state.Account accountDiffs := make(map[common.Address]AccountDiffEventual) for addr, val := range accounts { sr := val.Root - if storageDiffs, err := sdb.buildStorageDiffsEventual(sr, created); err != nil { + storageDiffs, err := sdb.buildStorageDiffsEventual(sr, created) + if err != nil { log.Error("Failed building eventual storage diffs", "Address", val, "error", err) return nil, err - } else { - code := "" - codeBytes, err := sdb.chainDB.Get(val.CodeHash) - if err == nil && len(codeBytes) != 0 { - code = common.ToHex(codeBytes) - } else { - log.Debug("No code field.", "codehash", val.CodeHash, "Address", val, "error", err) + } + + codeBytes, err := sdb.chainDB.Get(val.CodeHash) + + codeHash := common.ToHex(val.CodeHash) + hexRoot := val.Root.Hex() + + if created { + nonce := diffUint64{ + NewValue: &val.Nonce, } - codeHash := common.ToHex(val.CodeHash) - if created { - nonce := diffUint64{ - NewValue: &val.Nonce, - } - balance := diffBigInt{ - NewValue: val.Balance, - } + balance := diffBigInt{ + NewValue: val.Balance, + } - hexRoot := val.Root.Hex() - contractRoot := diffString{ - NewValue: &hexRoot, - } - accountDiffs[addr] = AccountDiffEventual{ - Nonce: nonce, - Balance: balance, - CodeHash: codeHash, - Code: code, - ContractRoot: contractRoot, - Storage: storageDiffs, - } - } else { - nonce := diffUint64{ - OldValue: &val.Nonce, - } - balance := diffBigInt{ - OldValue: val.Balance, - } - hexRoot := val.Root.Hex() - contractRoot := diffString{ - OldValue: &hexRoot, - } - accountDiffs[addr] = AccountDiffEventual{ - Nonce: nonce, - Balance: balance, - CodeHash: codeHash, - ContractRoot: contractRoot, - Storage: storageDiffs, - } + contractRoot := diffString{ + NewValue: &hexRoot, + } + accountDiffs[addr] = AccountDiffEventual{ + Nonce: nonce, + Balance: balance, + CodeHash: codeHash, + Code: codeBytes, + ContractRoot: contractRoot, + Storage: storageDiffs, + } + } else { + nonce := diffUint64{ + OldValue: &val.Nonce, + } + balance := diffBigInt{ + OldValue: val.Balance, + } + contractRoot := diffString{ + OldValue: &hexRoot, + } + accountDiffs[addr] = AccountDiffEventual{ + Nonce: nonce, + Balance: balance, + CodeHash: codeHash, + ContractRoot: contractRoot, + Storage: storageDiffs, } } } + return accountDiffs, nil } -func (sdb *builder) buildDiffIncremental(creations map[common.Address]*state.Account, deletions map[common.Address]*state.Account, updatedKeys *[]string) (map[common.Address]AccountDiffIncremental, error) { +func (sdb *builder) buildDiffIncremental(creations map[common.Address]*state.Account, deletions map[common.Address]*state.Account, updatedKeys []string) (map[common.Address]AccountDiffIncremental, error) { updatedAccounts := make(map[common.Address]AccountDiffIncremental) - for _, val := range *updatedKeys { + for _, val := range updatedKeys { createdAcc := creations[common.HexToAddress(val)] deletedAcc := deletions[common.HexToAddress(val)] oldSR := deletedAcc.Root diff --git a/statediff/config.go b/statediff/config.go index ee31f271e..ac207acd1 100644 --- a/statediff/config.go +++ b/statediff/config.go @@ -22,26 +22,27 @@ package statediff import "fmt" type Config struct { - On bool - Mode StateDiffMode + On bool // Whether or not to extract state diffs + Mode StateDiffMode // Mode for storing diffs + Path string // Path for storing diffs } type StateDiffMode int const ( - IPFS StateDiffMode = iota + IPLD StateDiffMode = iota LDB SQL ) func (mode StateDiffMode) IsValid() bool { - return mode >= IPFS && mode <= SQL + return mode >= IPLD && mode <= SQL } // String implements the stringer interface. func (mode StateDiffMode) String() string { switch mode { - case IPFS: + case IPLD: return "ipfs" case LDB: return "ldb" @@ -54,7 +55,7 @@ func (mode StateDiffMode) String() string { func (mode StateDiffMode) MarshalText() ([]byte, error) { switch mode { - case IPFS: + case IPLD: return []byte("ipfs"), nil case LDB: return []byte("ldb"), nil @@ -68,7 +69,7 @@ func (mode StateDiffMode) MarshalText() ([]byte, error) { func (mode *StateDiffMode) UnmarshalText(text []byte) error { switch string(text) { case "ipfs": - *mode = IPFS + *mode = IPLD case "ldb": *mode = LDB case "sql": diff --git a/statediff/extractor.go b/statediff/extractor.go index 2abe3ebb3..3cbb5a311 100644 --- a/statediff/extractor.go +++ b/statediff/extractor.go @@ -25,26 +25,31 @@ import ( ) type Extractor interface { - ExtractStateDiff(parent, current types.Block) error + ExtractStateDiff(parent, current types.Block) (string, error) } type extractor struct { - b *builder - p *persister + *builder // Interface for building state diff objects from two blocks + *publisher // Interface for publishing state diff objects to a datastore (e.g. IPFS) } -func NewExtractor(db ethdb.Database) *extractor { - return &extractor{ - b: NewBuilder(db), - p: NewPersister(), - } -} - -func (e *extractor) Extract(parent, current types.Block) error { - stateDiff, err := e.b.BuildStateDiff(parent.Root(), current.Root(), current.Number().Int64(), current.Hash()) +func NewExtractor(db ethdb.Database, config Config) (*extractor, error) { + publisher, err := NewPublisher(config) if err != nil { - return err + return nil, err } - return e.p.PersistStateDiff(stateDiff) + return &extractor{ + builder: NewBuilder(db), + publisher: publisher, + }, nil +} + +func (e *extractor) ExtractStateDiff(parent, current types.Block) (string, error) { + stateDiff, err := e.BuildStateDiff(parent.Root(), current.Root(), current.Number().Int64(), current.Hash()) + if err != nil { + return "", err + } + + return e.PublishStateDiff(stateDiff) } \ No newline at end of file diff --git a/statediff/helpers.go b/statediff/helpers.go index 50626ab2c..8a0a6466e 100644 --- a/statediff/helpers.go +++ b/statediff/helpers.go @@ -112,22 +112,7 @@ func decodeNibbles(nibbles []byte, bytes []byte) { } } -// prefixLen returns the length of the common prefix of a and b. -func prefixLen(a, b []byte) int { - var i, length = 0, len(a) - if len(b) < length { - length = len(b) - } - for ; i < length; i++ { - if a[i] != b[i] { - break - } - } - - return i -} - // hasTerm returns whether a hex key has the terminator flag. func hasTerm(s []byte) bool { return len(s) > 0 && s[len(s)-1] == 16 -} +} \ No newline at end of file diff --git a/statediff/ipfs/adder.go b/statediff/ipfs/adder.go new file mode 100644 index 000000000..23eea1c1f --- /dev/null +++ b/statediff/ipfs/adder.go @@ -0,0 +1,58 @@ +// Copyright 2015 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +// Contains a batch of utility type declarations used by the tests. As the node +// operates on unique types, a lot of them are needed to check various features. + +package ipfs + +import ( + "context" + + "github.com/ipfs/go-ipfs/core" + "github.com/ipfs/go-ipfs/repo/fsrepo" + ipld "gx/ipfs/QmWi2BYBL5gJ3CiAiQchg6rn1A8iBsrWy51EYxvHVjFvLb/go-ipld-format" +) + +type Adder interface { + Add(node ipld.Node) error +} + +type adder struct { + n *core.IpfsNode + ctx context.Context +} + +func (a adder) Add(node ipld.Node) error { + return a.n.DAG.Add(a.n.Context(), node) // For some reason DAG.Add method is not being exposed by the ipld.DAGService +} + +func NewAdder(repoPath string) (*adder, error) { + r, err := fsrepo.Open(repoPath) + if err != nil { + return nil, err + } + ctx := context.Background() + cfg := &core.BuildCfg{ + Online: false, + Repo: r, + } + ipfsNode, err := core.NewNode(ctx, cfg) + if err != nil { + return nil, err + } + return &adder{n: ipfsNode, ctx: ctx}, nil +} \ No newline at end of file diff --git a/statediff/ipfs/dag_putter.go b/statediff/ipfs/dag_putter.go new file mode 100644 index 000000000..92ffa1e8a --- /dev/null +++ b/statediff/ipfs/dag_putter.go @@ -0,0 +1,79 @@ +// Copyright 2015 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +// Contains a batch of utility type declarations used by the tests. As the node +// operates on unique types, a lot of them are needed to check various features. + +package ipfs + +import ( + "bytes" + "encoding/gob" + "github.com/i-norden/go-ethereum/statediff" + + ipld "gx/ipfs/QmWi2BYBL5gJ3CiAiQchg6rn1A8iBsrWy51EYxvHVjFvLb/go-ipld-format" +) + +const ( + EthStateDiffCode = 0x99 // Register custom codec for state diff? +) + +type DagPutter interface { + DagPut(sd *statediff.StateDiff) (string, error) +} + +type dagPutter struct { + Adder +} + +func NewDagPutter(adder Adder) *dagPutter { + return &dagPutter{Adder: adder} +} + +func (bhdp *dagPutter) DagPut(sd *statediff.StateDiff) (string, error) { + nd, err := bhdp.getNode(sd) + if err != nil { + return "", err + } + err = bhdp.Add(nd) + if err != nil { + return "", err + } + return nd.Cid().String(), nil +} + +func (bhdp *dagPutter) getNode(sd *statediff.StateDiff) (ipld.Node, error) { + + var buff bytes.Buffer + enc := gob.NewEncoder(&buff) + + err := enc.Encode(sd) + if err != nil { + return nil, err + } + + raw := buff.Bytes() + cid, err := RawToCid(EthStateDiffCode, raw) + if err != nil { + return nil, err + } + + return &StateDiffNode{ + StateDiff: sd, + cid: cid, + rawdata: raw, + }, nil +} \ No newline at end of file diff --git a/statediff/persister.go b/statediff/ipfs/helpers.go similarity index 71% rename from statediff/persister.go rename to statediff/ipfs/helpers.go index a3eba0826..a9904afa5 100644 --- a/statediff/persister.go +++ b/statediff/ipfs/helpers.go @@ -17,24 +17,22 @@ // Contains a batch of utility type declarations used by the tests. As the node // operates on unique types, a lot of them are needed to check various features. -package statediff +package ipfs -type Persister interface { - PersistStateDiff(sd *StateDiff) error -} - -type persister struct { - -} - -func NewPersister() *persister { - return &persister{ +import ( + mh "gx/ipfs/QmZyZDi491cCNTLfAhwcaDii2Kg4pwKRkhqQzURGDvY6ua/go-multihash" + "gx/ipfs/QmapdYm1b22Frv3k17fqrBYTFRxwiaVJkB299Mfn33edeB/go-cid" +) +func RawToCid(codec uint64, raw []byte) (*cid.Cid, error) { + c, err := cid.Prefix{ + Codec: codec, + Version: 1, + MhType: mh.KECCAK_256, + MhLength: -1, + }.Sum(raw) + if err != nil { + return nil, err } -} - -func (p *persister) PersistStateDiff(sd *StateDiff) error { - //TODO: Persist state diff in IPFS - - return nil + return c, nil } \ No newline at end of file diff --git a/statediff/ipfs/node.go b/statediff/ipfs/node.go new file mode 100644 index 000000000..dd7447a80 --- /dev/null +++ b/statediff/ipfs/node.go @@ -0,0 +1,78 @@ +// Copyright 2015 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +// Contains a batch of utility type declarations used by the tests. As the node +// operates on unique types, a lot of them are needed to check various features. + +package ipfs + +import ( + ipld "gx/ipfs/QmWi2BYBL5gJ3CiAiQchg6rn1A8iBsrWy51EYxvHVjFvLb/go-ipld-format" + "gx/ipfs/QmapdYm1b22Frv3k17fqrBYTFRxwiaVJkB299Mfn33edeB/go-cid" + + "github.com/i-norden/go-ethereum/statediff" +) + +type StateDiffNode struct { + *statediff.StateDiff + + cid *cid.Cid + rawdata []byte +} + +func (sdn *StateDiffNode) RawData() []byte { + return sdn.rawdata +} + +func (sdn *StateDiffNode) Cid() *cid.Cid { + return sdn.cid +} + +func (sdn StateDiffNode) String() string { + return sdn.cid.String() +} + +func (sdn StateDiffNode) Loggable() map[string]interface{} { + return sdn.cid.Loggable() +} + +func (sdn StateDiffNode) Resolve(path []string) (interface{}, []string, error) { + panic("implement me") +} + +func (sdn StateDiffNode) Tree(path string, depth int) []string { + panic("implement me") +} + +func (sdn StateDiffNode) ResolveLink(path []string) (*ipld.Link, []string, error) { + panic("implement me") +} + +func (sdn StateDiffNode) Copy() ipld.Node { + panic("implement me") +} + +func (sdn StateDiffNode) Links() []*ipld.Link { + panic("implement me") +} + +func (sdn StateDiffNode) Stat() (*ipld.NodeStat, error) { + panic("implement me") +} + +func (sdn StateDiffNode) Size() (uint64, error) { + panic("implement me") +} \ No newline at end of file diff --git a/statediff/publisher.go b/statediff/publisher.go new file mode 100644 index 000000000..163f8d3b0 --- /dev/null +++ b/statediff/publisher.go @@ -0,0 +1,63 @@ +// Copyright 2015 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +// Contains a batch of utility type declarations used by the tests. As the node +// operates on unique types, a lot of them are needed to check various features. + +package statediff + +import ( + "errors" + "github.com/i-norden/go-ethereum/statediff/ipfs" +) + +type Publisher interface { + PublishStateDiff(sd *StateDiff) (string, error) +} + +type publisher struct { + ipfs.DagPutter + Config +} + +func NewPublisher(config Config) (*publisher, error) { + adder, err := ipfs.NewAdder(config.Path) + if err != nil { + return nil, err + } + + return &publisher{ + DagPutter: ipfs.NewDagPutter(adder), + Config: config, + }, nil +} + +func (p *publisher) PublishStateDiff(sd *StateDiff) (string, error) { + switch p.Mode { + case IPLD: + cidStr, err := p.DagPut(sd) + if err != nil { + return "", err + } + + return cidStr, err + case LDB: + case SQL: + default: + } + + return "", errors.New("state diff publisher: unhandled publishing mode") +} \ No newline at end of file diff --git a/statediff/persister_test.go b/statediff/publisher_test.go similarity index 100% rename from statediff/persister_test.go rename to statediff/publisher_test.go diff --git a/statediff/statediff.go b/statediff/struct.go similarity index 98% rename from statediff/statediff.go rename to statediff/struct.go index f2e606936..2142ec755 100644 --- a/statediff/statediff.go +++ b/statediff/struct.go @@ -58,7 +58,7 @@ func (sd *StateDiff) Encode() ([]byte, error) { type AccountDiffEventual struct { Nonce diffUint64 `json:"nonce" gencodec:"required"` Balance diffBigInt `json:"balance" gencodec:"required"` - Code string `json:"code" gencodec:"required"` + Code []byte `json:"code" gencodec:"required"` CodeHash string `json:"codeHash" gencodec:"required"` ContractRoot diffString `json:"contractRoot" gencodec:"required"` Storage map[string]diffString `json:"storage" gencodec:"required"` -- 2.45.2 From 8b9f46941924a04ce1091532a74def03e70c7559 Mon Sep 17 00:00:00 2001 From: Liang Ma Date: Tue, 27 Nov 2018 10:52:30 +0000 Subject: [PATCH 27/48] p2p/protocols: fix minor comments typo (#18185) --- p2p/protocols/protocol.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/p2p/protocols/protocol.go b/p2p/protocols/protocol.go index 7dddd852f..b16720dd3 100644 --- a/p2p/protocols/protocol.go +++ b/p2p/protocols/protocol.go @@ -381,7 +381,7 @@ func (p *Peer) handleIncoming(handle func(ctx context.Context, msg interface{}) // * arguments // * context // * the local handshake to be sent to the remote peer -// * funcion to be called on the remote handshake (can be nil) +// * function to be called on the remote handshake (can be nil) // * expects a remote handshake back of the same type // * the dialing peer needs to send the handshake first and then waits for remote // * the listening peer waits for the remote handshake and then sends it -- 2.45.2 From 89fe24bbcc2395b9fbd449f5aeb9da8df073991d Mon Sep 17 00:00:00 2001 From: ANOTHEL Date: Tue, 27 Nov 2018 21:00:57 +0900 Subject: [PATCH 28/48] p2p/discv5: minor code simplification (#18188) * Update net.go more simple * Update net.go --- p2p/discv5/net.go | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/p2p/discv5/net.go b/p2p/discv5/net.go index a6cabf080..976eae7df 100644 --- a/p2p/discv5/net.go +++ b/p2p/discv5/net.go @@ -567,12 +567,11 @@ loop: net.ticketStore.searchLookupDone(res.target, res.nodes, func(n *Node, topic Topic) []byte { if n.state != nil && n.state.canQuery { return net.conn.send(n, topicQueryPacket, topicQuery{Topic: topic}) // TODO: set expiration - } else { - if n.state == unknown { - net.ping(n, n.addr()) - } - return nil } + if n.state == unknown { + net.ping(n, n.addr()) + } + return nil }) case <-statsDump.C: -- 2.45.2 From edc39aaedf526bfb7757a445c1f9dd42f45dc8d4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?P=C3=A9ter=20Szil=C3=A1gyi?= Date: Tue, 27 Nov 2018 14:50:47 +0200 Subject: [PATCH 29/48] p2p/discv5: gofmt --- p2p/discv5/net.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/p2p/discv5/net.go b/p2p/discv5/net.go index 976eae7df..cdeb28dd5 100644 --- a/p2p/discv5/net.go +++ b/p2p/discv5/net.go @@ -571,7 +571,7 @@ loop: if n.state == unknown { net.ping(n, n.addr()) } - return nil + return nil }) case <-statsDump.C: -- 2.45.2 From bfed28a42187c7bf7b8f50401fe239278a4ed530 Mon Sep 17 00:00:00 2001 From: Martin Holst Swende Date: Wed, 28 Nov 2018 09:29:05 +0100 Subject: [PATCH 30/48] core: more detailed metrics for block processing (#18119) --- core/blockchain.go | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/core/blockchain.go b/core/blockchain.go index bcd53d498..a672c0ab6 100644 --- a/core/blockchain.go +++ b/core/blockchain.go @@ -47,7 +47,10 @@ import ( ) var ( - blockInsertTimer = metrics.NewRegisteredTimer("chain/inserts", nil) + blockInsertTimer = metrics.NewRegisteredTimer("chain/inserts", nil) + blockValidationTimer = metrics.NewRegisteredTimer("chain/validation", nil) + blockExecutionTimer = metrics.NewRegisteredTimer("chain/execution", nil) + blockWriteTimer = metrics.NewRegisteredTimer("chain/write", nil) ErrNoGenesis = errors.New("Genesis not found in chain") ) @@ -1188,7 +1191,9 @@ func (bc *BlockChain) insertChain(chain types.Blocks, verifySeals bool) (int, [] return it.index, events, coalescedLogs, err } // Process block using the parent state as reference point. + t0 := time.Now() receipts, logs, usedGas, err := bc.processor.Process(block, state, bc.vmConfig) + t1 := time.Now() if err != nil { bc.reportBlock(block, receipts, err) return it.index, events, coalescedLogs, err @@ -1198,13 +1203,19 @@ func (bc *BlockChain) insertChain(chain types.Blocks, verifySeals bool) (int, [] bc.reportBlock(block, receipts, err) return it.index, events, coalescedLogs, err } + t2 := time.Now() proctime := time.Since(start) // Write the block to the chain and get the status. status, err := bc.WriteBlockWithState(block, receipts, state) + t3 := time.Now() if err != nil { return it.index, events, coalescedLogs, err } + blockInsertTimer.UpdateSince(start) + blockExecutionTimer.Update(t1.Sub(t0)) + blockValidationTimer.Update(t2.Sub(t1)) + blockWriteTimer.Update(t3.Sub(t2)) switch status { case CanonStatTy: log.Debug("Inserted new block", "number", block.Number(), "hash", block.Hash(), -- 2.45.2 From 174083c3ae0b6217dff0ced98ec7f73e53e24b04 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?P=C3=A9ter=20Szil=C3=A1gyi?= Date: Wed, 28 Nov 2018 13:31:42 +0200 Subject: [PATCH 31/48] eth/downloader: fix light client cht binary search issue --- eth/downloader/downloader.go | 22 ++++++++++++++++++++++ 1 file changed, 22 insertions(+) diff --git a/eth/downloader/downloader.go b/eth/downloader/downloader.go index f81a5cbac..3a177ab9d 100644 --- a/eth/downloader/downloader.go +++ b/eth/downloader/downloader.go @@ -99,6 +99,7 @@ type Downloader struct { mode SyncMode // Synchronisation mode defining the strategy used (per sync cycle) mux *event.TypeMux // Event multiplexer to announce sync operation events + genesis uint64 // Genesis block number to limit sync to (e.g. light client CHT) queue *queue // Scheduler for selecting the hashes to download peers *peerSet // Set of active peers from which download can proceed stateDB ethdb.Database @@ -664,7 +665,28 @@ func (d *Downloader) findAncestor(p *peerConnection, remoteHeader *types.Header) } p.log.Debug("Looking for common ancestor", "local", localHeight, "remote", remoteHeight) if localHeight >= MaxForkAncestry { + // We're above the max reorg threshold, find the earliest fork point floor = int64(localHeight - MaxForkAncestry) + + // If we're doing a light sync, ensure the floor doesn't go below the CHT, as + // all headers before that point will be missing. + if d.mode == LightSync { + // If we dont know the current CHT position, find it + if d.genesis == 0 { + header := d.lightchain.CurrentHeader() + for header != nil { + d.genesis = header.Number.Uint64() + if floor >= int64(d.genesis)-1 { + break + } + header = d.lightchain.GetHeaderByHash(header.ParentHash) + } + } + // We already know the "genesis" block number, cap floor to that + if floor < int64(d.genesis)-1 { + floor = int64(d.genesis) - 1 + } + } } from, count, skip, max := calculateRequestSpan(remoteHeight, localHeight) -- 2.45.2 From d606a7a46a62483f7fd8110de97ce91f70c4766e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?P=C3=A9ter=20Szil=C3=A1gyi?= Date: Wed, 28 Nov 2018 13:53:33 +0200 Subject: [PATCH 32/48] params: update CHTs for the v1.8.19 release --- params/config.go | 24 ++++++++++++------------ 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/params/config.go b/params/config.go index 3570cb204..ba9fac1c4 100644 --- a/params/config.go +++ b/params/config.go @@ -49,10 +49,10 @@ var ( // MainnetTrustedCheckpoint contains the light client trusted checkpoint for the main network. MainnetTrustedCheckpoint = &TrustedCheckpoint{ Name: "mainnet", - SectionIndex: 203, - SectionHead: common.HexToHash("0xc9e05fc67c6a9815adc8072eb18805b53da53a9a6a273e05541e1b7542cf937a"), - CHTRoot: common.HexToHash("0xb85f42447d59f7c3e6679b9a37ed983593fd52efd6251b883592662e95769d5b"), - BloomRoot: common.HexToHash("0xf93d50cb4c49b403c6fd33cd60896d3b36184275be0a51bae4df5e8844ac624c"), + SectionIndex: 206, + SectionHead: common.HexToHash("0x9fa677c7c0580136f5a86d9b2fd29b112e531f0284396298b8809bcb6787b538"), + CHTRoot: common.HexToHash("0x7f32dfb29e341b4c8c10ea2e06a812bcea470366f635b7a8b3d0856684cd76f4"), + BloomRoot: common.HexToHash("0x0169e174f0a8172aec217556d8a25c7ba7ca52aacff170325230a75740ff1eaf"), } // TestnetChainConfig contains the chain parameters to run a node on the Ropsten test network. @@ -73,10 +73,10 @@ var ( // TestnetTrustedCheckpoint contains the light client trusted checkpoint for the Ropsten test network. TestnetTrustedCheckpoint = &TrustedCheckpoint{ Name: "testnet", - SectionIndex: 134, - SectionHead: common.HexToHash("0x17053ecbe045bebefaa01e7716cc85a4e22647e181416cc1098ccbb73a088931"), - CHTRoot: common.HexToHash("0x4d2b86422e46ed76f0e3f50f06632c409f809c8375e53c8bc0f782bcb93dd49a"), - BloomRoot: common.HexToHash("0xccba62232ee56c2967afc58f136a47ba7dc545ae586e6be666430d94516306c7"), + SectionIndex: 136, + SectionHead: common.HexToHash("0xe5d80bb08d92bbc12dfe510c64cba01eafcbb4ba585e7c7ab7f8a93c6f295ab3"), + CHTRoot: common.HexToHash("0xe3ca77ab0cb51eec74f4f7458e36aee207c68768387b39cb0bcff0940a6264d8"), + BloomRoot: common.HexToHash("0x30c8eeadac5539d3dcd6e88915d1a07cb2f3a1d6ebe7e553e3ee783c04c68c2d"), } // RinkebyChainConfig contains the chain parameters to run a node on the Rinkeby test network. @@ -100,10 +100,10 @@ var ( // RinkebyTrustedCheckpoint contains the light client trusted checkpoint for the Rinkeby test network. RinkebyTrustedCheckpoint = &TrustedCheckpoint{ Name: "rinkeby", - SectionIndex: 100, - SectionHead: common.HexToHash("0xf18f9b43e16f37b12e68818536ffe455ff18d676274ffdd856a8520ed61bb514"), - CHTRoot: common.HexToHash("0x473f5d603b1fedad75d97fd58692130b9ac9ade1aca01eb9363d79bd1c43c791"), - BloomRoot: common.HexToHash("0xa39ced3ddbb87e909c7531df2afb6414bea9c9a60ab94da9c6b467535f05326e"), + SectionIndex: 103, + SectionHead: common.HexToHash("0x9f38b903852831bf4fa7992f7fd43d8b26da2deb82b421fb845cf6faee54e056"), + CHTRoot: common.HexToHash("0x2d710c2cea468d2e604838000d658ee213e4abb07f90c4f71f5cd7f8510aa708"), + BloomRoot: common.HexToHash("0xcc401060280c2cc82697ea5ecef8cac61e52063c37533a2e9609332419704d5f"), } // AllEthashProtocolChanges contains every protocol change (EIPs) introduced -- 2.45.2 From dae82f098570e15d44584f0d7f350713f4774727 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?P=C3=A9ter=20Szil=C3=A1gyi?= Date: Wed, 28 Nov 2018 14:42:37 +0200 Subject: [PATCH 33/48] params, swarm: release Geth v1.8.19 and Swarm v0.3.7 --- params/version.go | 8 ++++---- swarm/version/version.go | 8 ++++---- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/params/version.go b/params/version.go index b9dcc2a84..cbb23e16d 100644 --- a/params/version.go +++ b/params/version.go @@ -21,10 +21,10 @@ import ( ) const ( - VersionMajor = 1 // Major version component of the current release - VersionMinor = 8 // Minor version component of the current release - VersionPatch = 19 // Patch version component of the current release - VersionMeta = "unstable" // Version metadata to append to the version string + VersionMajor = 1 // Major version component of the current release + VersionMinor = 8 // Minor version component of the current release + VersionPatch = 19 // Patch version component of the current release + VersionMeta = "stable" // Version metadata to append to the version string ) // Version holds the textual version string. diff --git a/swarm/version/version.go b/swarm/version/version.go index 17ef34f5f..d75763fe9 100644 --- a/swarm/version/version.go +++ b/swarm/version/version.go @@ -21,10 +21,10 @@ import ( ) const ( - VersionMajor = 0 // Major version component of the current release - VersionMinor = 3 // Minor version component of the current release - VersionPatch = 7 // Patch version component of the current release - VersionMeta = "unstable" // Version metadata to append to the version string + VersionMajor = 0 // Major version component of the current release + VersionMinor = 3 // Minor version component of the current release + VersionPatch = 7 // Patch version component of the current release + VersionMeta = "stable" // Version metadata to append to the version string ) // Version holds the textual version string. -- 2.45.2 From a7501d0c419f8c1dc8b4004c115dd2d4e3b524b7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?P=C3=A9ter=20Szil=C3=A1gyi?= Date: Wed, 28 Nov 2018 14:44:00 +0200 Subject: [PATCH 34/48] params, swarm: start Geth v1.8.20 and Swarm v0.3.8 release cycle --- params/version.go | 8 ++++---- swarm/version/version.go | 8 ++++---- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/params/version.go b/params/version.go index cbb23e16d..2291d5f3c 100644 --- a/params/version.go +++ b/params/version.go @@ -21,10 +21,10 @@ import ( ) const ( - VersionMajor = 1 // Major version component of the current release - VersionMinor = 8 // Minor version component of the current release - VersionPatch = 19 // Patch version component of the current release - VersionMeta = "stable" // Version metadata to append to the version string + VersionMajor = 1 // Major version component of the current release + VersionMinor = 8 // Minor version component of the current release + VersionPatch = 20 // Patch version component of the current release + VersionMeta = "unstable" // Version metadata to append to the version string ) // Version holds the textual version string. diff --git a/swarm/version/version.go b/swarm/version/version.go index d75763fe9..57ac05a86 100644 --- a/swarm/version/version.go +++ b/swarm/version/version.go @@ -21,10 +21,10 @@ import ( ) const ( - VersionMajor = 0 // Major version component of the current release - VersionMinor = 3 // Minor version component of the current release - VersionPatch = 7 // Patch version component of the current release - VersionMeta = "stable" // Version metadata to append to the version string + VersionMajor = 0 // Major version component of the current release + VersionMinor = 3 // Minor version component of the current release + VersionPatch = 8 // Patch version component of the current release + VersionMeta = "unstable" // Version metadata to append to the version string ) // Version holds the textual version string. -- 2.45.2 From 3d21d455dca6e75a635ab5742c66c80ee175c3e7 Mon Sep 17 00:00:00 2001 From: Matthew Di Ferrante Date: Thu, 29 Nov 2018 09:29:12 +0100 Subject: [PATCH 35/48] cmd/evm: commit statedb if dump is requested (#18208) Add a call `statedb.Commit(true)` if the `Dump` flag is on, as otherwise the `storage` output in the dump is always empty. --- cmd/evm/runner.go | 1 + 1 file changed, 1 insertion(+) diff --git a/cmd/evm/runner.go b/cmd/evm/runner.go index 962fc021d..a9c8a38ca 100644 --- a/cmd/evm/runner.go +++ b/cmd/evm/runner.go @@ -206,6 +206,7 @@ func runCmd(ctx *cli.Context) error { execTime := time.Since(tstart) if ctx.GlobalBool(DumpFlag.Name) { + statedb.Commit(true) statedb.IntermediateRoot(true) fmt.Println(string(statedb.Dump())) } -- 2.45.2 From 7c657fc78957d7ed302eba68bb6122b515364ac4 Mon Sep 17 00:00:00 2001 From: Martin Holst Swende Date: Thu, 29 Nov 2018 09:51:57 +0100 Subject: [PATCH 36/48] tests, core: update tests and make STATICCALL cause touch-delete (#18187) --- core/vm/evm.go | 6 ++++++ tests/init.go | 9 +++++++++ tests/testdata | 2 +- 3 files changed, 16 insertions(+), 1 deletion(-) diff --git a/core/vm/evm.go b/core/vm/evm.go index 968d2219e..ba4d1e9eb 100644 --- a/core/vm/evm.go +++ b/core/vm/evm.go @@ -339,6 +339,12 @@ func (evm *EVM) StaticCall(caller ContractRef, addr common.Address, input []byte contract := NewContract(caller, to, new(big.Int), gas) contract.SetCallCode(&addr, evm.StateDB.GetCodeHash(addr), evm.StateDB.GetCode(addr)) + // We do an AddBalance of zero here, just in order to trigger a touch. + // This doesn't matter on Mainnet, where all empties are gone at the time of Byzantium, + // but is the correct thing to do and matters on other networks, in tests, and potential + // future scenarios + evm.StateDB.AddBalance(addr, bigZero) + // When an error was returned by the EVM or when setting the creation code // above we revert to the snapshot and consume any gas remaining. Additionally // when we're in Homestead this also counts for code storage gas errors. diff --git a/tests/init.go b/tests/init.go index f0a4943c1..db0457b6d 100644 --- a/tests/init.go +++ b/tests/init.go @@ -86,6 +86,15 @@ var Forks = map[string]*params.ChainConfig{ EIP158Block: big.NewInt(0), ByzantiumBlock: big.NewInt(5), }, + "ByzantiumToConstantinopleAt5": { + ChainID: big.NewInt(1), + HomesteadBlock: big.NewInt(0), + EIP150Block: big.NewInt(0), + EIP155Block: big.NewInt(0), + EIP158Block: big.NewInt(0), + ByzantiumBlock: big.NewInt(0), + ConstantinopleBlock: big.NewInt(5), + }, } // UnsupportedForkError is returned when a test requests a fork that isn't implemented. diff --git a/tests/testdata b/tests/testdata index 95a309203..c02a2a17c 160000 --- a/tests/testdata +++ b/tests/testdata @@ -1 +1 @@ -Subproject commit 95a309203890e6244c6d4353ca411671973c13b5 +Subproject commit c02a2a17c0288a255572b37dc7ec1fcb838b9dbf -- 2.45.2 From 8380a1303ce6a2fa4760a8641e47e7196a941b4c Mon Sep 17 00:00:00 2001 From: Martin Holst Swende Date: Wed, 28 Nov 2018 20:53:02 +0100 Subject: [PATCH 37/48] vendor: update leveldb --- .../syndtr/goleveldb/leveldb/cache/cache.go | 1 - .../leveldb/comparer/bytes_comparer.go | 4 +- .../goleveldb/leveldb/comparer/comparer.go | 2 +- .../github.com/syndtr/goleveldb/leveldb/db.go | 2 +- .../syndtr/goleveldb/leveldb/db_compaction.go | 24 +++----- .../syndtr/goleveldb/leveldb/db_util.go | 2 +- .../syndtr/goleveldb/leveldb/iterator/iter.go | 4 +- .../syndtr/goleveldb/leveldb/opt/options.go | 13 +++++ .../syndtr/goleveldb/leveldb/session_util.go | 4 +- .../syndtr/goleveldb/leveldb/table.go | 30 +++++----- vendor/vendor.json | 58 +++++++++---------- 11 files changed, 76 insertions(+), 68 deletions(-) diff --git a/vendor/github.com/syndtr/goleveldb/leveldb/cache/cache.go b/vendor/github.com/syndtr/goleveldb/leveldb/cache/cache.go index c5940b232..c36ad3235 100644 --- a/vendor/github.com/syndtr/goleveldb/leveldb/cache/cache.go +++ b/vendor/github.com/syndtr/goleveldb/leveldb/cache/cache.go @@ -331,7 +331,6 @@ func (r *Cache) delete(n *Node) bool { return deleted } } - return false } // Nodes returns number of 'cache node' in the map. diff --git a/vendor/github.com/syndtr/goleveldb/leveldb/comparer/bytes_comparer.go b/vendor/github.com/syndtr/goleveldb/leveldb/comparer/bytes_comparer.go index 14dddf88d..abf9fb65c 100644 --- a/vendor/github.com/syndtr/goleveldb/leveldb/comparer/bytes_comparer.go +++ b/vendor/github.com/syndtr/goleveldb/leveldb/comparer/bytes_comparer.go @@ -29,7 +29,7 @@ func (bytesComparer) Separator(dst, a, b []byte) []byte { // Do not shorten if one string is a prefix of the other } else if c := a[i]; c < 0xff && c+1 < b[i] { dst = append(dst, a[:i+1]...) - dst[i]++ + dst[len(dst)-1]++ return dst } return nil @@ -39,7 +39,7 @@ func (bytesComparer) Successor(dst, b []byte) []byte { for i, c := range b { if c != 0xff { dst = append(dst, b[:i+1]...) - dst[i]++ + dst[len(dst)-1]++ return dst } } diff --git a/vendor/github.com/syndtr/goleveldb/leveldb/comparer/comparer.go b/vendor/github.com/syndtr/goleveldb/leveldb/comparer/comparer.go index 14a28f16f..2c522db23 100644 --- a/vendor/github.com/syndtr/goleveldb/leveldb/comparer/comparer.go +++ b/vendor/github.com/syndtr/goleveldb/leveldb/comparer/comparer.go @@ -36,7 +36,7 @@ type Comparer interface { // by any users of this package. Name() string - // Bellow are advanced functions used used to reduce the space requirements + // Bellow are advanced functions used to reduce the space requirements // for internal data structures such as index blocks. // Separator appends a sequence of bytes x to dst such that a <= x && x < b, diff --git a/vendor/github.com/syndtr/goleveldb/leveldb/db.go b/vendor/github.com/syndtr/goleveldb/leveldb/db.go index e7ac06541..b27c38d37 100644 --- a/vendor/github.com/syndtr/goleveldb/leveldb/db.go +++ b/vendor/github.com/syndtr/goleveldb/leveldb/db.go @@ -182,7 +182,7 @@ func Open(stor storage.Storage, o *opt.Options) (db *DB, err error) { err = s.recover() if err != nil { - if !os.IsNotExist(err) || s.o.GetErrorIfMissing() { + if !os.IsNotExist(err) || s.o.GetErrorIfMissing() || s.o.GetReadOnly() { return } err = s.create() diff --git a/vendor/github.com/syndtr/goleveldb/leveldb/db_compaction.go b/vendor/github.com/syndtr/goleveldb/leveldb/db_compaction.go index 28e50906a..0c1b9a53b 100644 --- a/vendor/github.com/syndtr/goleveldb/leveldb/db_compaction.go +++ b/vendor/github.com/syndtr/goleveldb/leveldb/db_compaction.go @@ -663,7 +663,7 @@ type cCmd interface { } type cAuto struct { - // Note for table compaction, an empty ackC represents it's a compaction waiting command. + // Note for table compaction, an non-empty ackC represents it's a compaction waiting command. ackC chan<- error } @@ -777,8 +777,8 @@ func (db *DB) mCompaction() { func (db *DB) tCompaction() { var ( - x cCmd - ackQ, waitQ []cCmd + x cCmd + waitQ []cCmd ) defer func() { @@ -787,10 +787,6 @@ func (db *DB) tCompaction() { panic(x) } } - for i := range ackQ { - ackQ[i].ack(ErrClosed) - ackQ[i] = nil - } for i := range waitQ { waitQ[i].ack(ErrClosed) waitQ[i] = nil @@ -821,11 +817,6 @@ func (db *DB) tCompaction() { waitQ = waitQ[:0] } } else { - for i := range ackQ { - ackQ[i].ack(nil) - ackQ[i] = nil - } - ackQ = ackQ[:0] for i := range waitQ { waitQ[i].ack(nil) waitQ[i] = nil @@ -844,9 +835,12 @@ func (db *DB) tCompaction() { switch cmd := x.(type) { case cAuto: if cmd.ackC != nil { - waitQ = append(waitQ, x) - } else { - ackQ = append(ackQ, x) + // Check the write pause state before caching it. + if db.resumeWrite() { + x.ack(nil) + } else { + waitQ = append(waitQ, x) + } } case cRange: x.ack(db.tableRangeCompaction(cmd.level, cmd.min, cmd.max)) diff --git a/vendor/github.com/syndtr/goleveldb/leveldb/db_util.go b/vendor/github.com/syndtr/goleveldb/leveldb/db_util.go index 7ecd960d2..3f0654894 100644 --- a/vendor/github.com/syndtr/goleveldb/leveldb/db_util.go +++ b/vendor/github.com/syndtr/goleveldb/leveldb/db_util.go @@ -84,7 +84,7 @@ func (db *DB) checkAndCleanFiles() error { var mfds []storage.FileDesc for num, present := range tmap { if !present { - mfds = append(mfds, storage.FileDesc{storage.TypeTable, num}) + mfds = append(mfds, storage.FileDesc{Type: storage.TypeTable, Num: num}) db.logf("db@janitor table missing @%d", num) } } diff --git a/vendor/github.com/syndtr/goleveldb/leveldb/iterator/iter.go b/vendor/github.com/syndtr/goleveldb/leveldb/iterator/iter.go index b16e3a704..96fb0f685 100644 --- a/vendor/github.com/syndtr/goleveldb/leveldb/iterator/iter.go +++ b/vendor/github.com/syndtr/goleveldb/leveldb/iterator/iter.go @@ -40,11 +40,11 @@ type IteratorSeeker interface { Seek(key []byte) bool // Next moves the iterator to the next key/value pair. - // It returns whether the iterator is exhausted. + // It returns false if the iterator is exhausted. Next() bool // Prev moves the iterator to the previous key/value pair. - // It returns whether the iterator is exhausted. + // It returns false if the iterator is exhausted. Prev() bool } diff --git a/vendor/github.com/syndtr/goleveldb/leveldb/opt/options.go b/vendor/github.com/syndtr/goleveldb/leveldb/opt/options.go index 44e7d9adc..528b16423 100644 --- a/vendor/github.com/syndtr/goleveldb/leveldb/opt/options.go +++ b/vendor/github.com/syndtr/goleveldb/leveldb/opt/options.go @@ -158,6 +158,12 @@ type Options struct { // The default value is 8MiB. BlockCacheCapacity int + // BlockCacheEvictRemoved allows enable forced-eviction on cached block belonging + // to removed 'sorted table'. + // + // The default if false. + BlockCacheEvictRemoved bool + // BlockRestartInterval is the number of keys between restart points for // delta encoding of keys. // @@ -384,6 +390,13 @@ func (o *Options) GetBlockCacheCapacity() int { return o.BlockCacheCapacity } +func (o *Options) GetBlockCacheEvictRemoved() bool { + if o == nil { + return false + } + return o.BlockCacheEvictRemoved +} + func (o *Options) GetBlockRestartInterval() int { if o == nil || o.BlockRestartInterval <= 0 { return DefaultBlockRestartInterval diff --git a/vendor/github.com/syndtr/goleveldb/leveldb/session_util.go b/vendor/github.com/syndtr/goleveldb/leveldb/session_util.go index 92328933c..40cb2cf95 100644 --- a/vendor/github.com/syndtr/goleveldb/leveldb/session_util.go +++ b/vendor/github.com/syndtr/goleveldb/leveldb/session_util.go @@ -36,7 +36,7 @@ func (s *session) logf(format string, v ...interface{}) { s.stor.Log(fmt.Sprintf func (s *session) newTemp() storage.FileDesc { num := atomic.AddInt64(&s.stTempFileNum, 1) - 1 - return storage.FileDesc{storage.TypeTemp, num} + return storage.FileDesc{Type: storage.TypeTemp, Num: num} } func (s *session) addFileRef(fd storage.FileDesc, ref int) int { @@ -190,7 +190,7 @@ func (s *session) recordCommited(rec *sessionRecord) { // Create a new manifest file; need external synchronization. func (s *session) newManifest(rec *sessionRecord, v *version) (err error) { - fd := storage.FileDesc{storage.TypeManifest, s.allocFileNum()} + fd := storage.FileDesc{Type: storage.TypeManifest, Num: s.allocFileNum()} writer, err := s.stor.Create(fd) if err != nil { return diff --git a/vendor/github.com/syndtr/goleveldb/leveldb/table.go b/vendor/github.com/syndtr/goleveldb/leveldb/table.go index 81d18a531..1fac60d05 100644 --- a/vendor/github.com/syndtr/goleveldb/leveldb/table.go +++ b/vendor/github.com/syndtr/goleveldb/leveldb/table.go @@ -78,7 +78,7 @@ func newTableFile(fd storage.FileDesc, size int64, imin, imax internalKey) *tFil } func tableFileFromRecord(r atRecord) *tFile { - return newTableFile(storage.FileDesc{storage.TypeTable, r.num}, r.size, r.imin, r.imax) + return newTableFile(storage.FileDesc{Type: storage.TypeTable, Num: r.num}, r.size, r.imin, r.imax) } // tFiles hold multiple tFile. @@ -290,16 +290,17 @@ func (x *tFilesSortByNum) Less(i, j int) bool { // Table operations. type tOps struct { - s *session - noSync bool - cache *cache.Cache - bcache *cache.Cache - bpool *util.BufferPool + s *session + noSync bool + evictRemoved bool + cache *cache.Cache + bcache *cache.Cache + bpool *util.BufferPool } // Creates an empty table and returns table writer. func (t *tOps) create() (*tWriter, error) { - fd := storage.FileDesc{storage.TypeTable, t.s.allocFileNum()} + fd := storage.FileDesc{Type: storage.TypeTable, Num: t.s.allocFileNum()} fw, err := t.s.stor.Create(fd) if err != nil { return nil, err @@ -422,7 +423,7 @@ func (t *tOps) remove(f *tFile) { } else { t.s.logf("table@remove removed @%d", f.fd.Num) } - if t.bcache != nil { + if t.evictRemoved && t.bcache != nil { t.bcache.EvictNS(uint64(f.fd.Num)) } }) @@ -451,7 +452,7 @@ func newTableOps(s *session) *tOps { if !s.o.GetDisableBlockCache() { var bcacher cache.Cacher if s.o.GetBlockCacheCapacity() > 0 { - bcacher = cache.NewLRU(s.o.GetBlockCacheCapacity()) + bcacher = s.o.GetBlockCacher().New(s.o.GetBlockCacheCapacity()) } bcache = cache.NewCache(bcacher) } @@ -459,11 +460,12 @@ func newTableOps(s *session) *tOps { bpool = util.NewBufferPool(s.o.GetBlockSize() + 5) } return &tOps{ - s: s, - noSync: s.o.GetNoSync(), - cache: cache.NewCache(cacher), - bcache: bcache, - bpool: bpool, + s: s, + noSync: s.o.GetNoSync(), + evictRemoved: s.o.GetBlockCacheEvictRemoved(), + cache: cache.NewCache(cacher), + bcache: bcache, + bpool: bpool, } } diff --git a/vendor/vendor.json b/vendor/vendor.json index 1bfe09da7..a4fee11aa 100644 --- a/vendor/vendor.json +++ b/vendor/vendor.json @@ -455,76 +455,76 @@ "revisionTime": "2017-07-05T02:17:15Z" }, { - "checksumSHA1": "k6zbR5hiI10hkWtiK91rIY5s5/E=", + "checksumSHA1": "LV0VMVON7xY1ttV+s2ph83ntmDQ=", "path": "github.com/syndtr/goleveldb/leveldb", - "revision": "c4c61651e9e37fa117f53c5a906d3b63090d8445", - "revisionTime": "2018-07-08T03:05:51Z" + "revision": "b001fa50d6b27f3f0bb175a87d0cb55426d0a0ae", + "revisionTime": "2018-11-28T10:09:59Z" }, { - "checksumSHA1": "EKIow7XkgNdWvR/982ffIZxKG8Y=", + "checksumSHA1": "mPNraL2edpk/2FYq26rSXfMHbJg=", "path": "github.com/syndtr/goleveldb/leveldb/cache", - "revision": "c4c61651e9e37fa117f53c5a906d3b63090d8445", - "revisionTime": "2018-07-08T03:05:51Z" + "revision": "b001fa50d6b27f3f0bb175a87d0cb55426d0a0ae", + "revisionTime": "2018-11-28T10:09:59Z" }, { - "checksumSHA1": "5KPgnvCPlR0ysDAqo6jApzRQ3tw=", + "checksumSHA1": "UA+PKDKWlDnE2OZblh23W6wZwbY=", "path": "github.com/syndtr/goleveldb/leveldb/comparer", - "revision": "c4c61651e9e37fa117f53c5a906d3b63090d8445", - "revisionTime": "2018-07-08T03:05:51Z" + "revision": "b001fa50d6b27f3f0bb175a87d0cb55426d0a0ae", + "revisionTime": "2018-11-28T10:09:59Z" }, { "checksumSHA1": "1DRAxdlWzS4U0xKN/yQ/fdNN7f0=", "path": "github.com/syndtr/goleveldb/leveldb/errors", - "revision": "c4c61651e9e37fa117f53c5a906d3b63090d8445", - "revisionTime": "2018-07-08T03:05:51Z" + "revision": "b001fa50d6b27f3f0bb175a87d0cb55426d0a0ae", + "revisionTime": "2018-11-28T10:09:59Z" }, { "checksumSHA1": "eqKeD6DS7eNCtxVYZEHHRKkyZrw=", "path": "github.com/syndtr/goleveldb/leveldb/filter", - "revision": "c4c61651e9e37fa117f53c5a906d3b63090d8445", - "revisionTime": "2018-07-08T03:05:51Z" + "revision": "b001fa50d6b27f3f0bb175a87d0cb55426d0a0ae", + "revisionTime": "2018-11-28T10:09:59Z" }, { - "checksumSHA1": "weSsccMav4BCerDpSLzh3mMxAYo=", + "checksumSHA1": "hPyFsMiqZ1OB7MX+6wIAA6nsdtc=", "path": "github.com/syndtr/goleveldb/leveldb/iterator", - "revision": "c4c61651e9e37fa117f53c5a906d3b63090d8445", - "revisionTime": "2018-07-08T03:05:51Z" + "revision": "b001fa50d6b27f3f0bb175a87d0cb55426d0a0ae", + "revisionTime": "2018-11-28T10:09:59Z" }, { "checksumSHA1": "gJY7bRpELtO0PJpZXgPQ2BYFJ88=", "path": "github.com/syndtr/goleveldb/leveldb/journal", - "revision": "c4c61651e9e37fa117f53c5a906d3b63090d8445", - "revisionTime": "2018-07-08T03:05:51Z" + "revision": "b001fa50d6b27f3f0bb175a87d0cb55426d0a0ae", + "revisionTime": "2018-11-28T10:09:59Z" }, { "checksumSHA1": "MtYY1b2234y/MlS+djL8tXVAcQs=", "path": "github.com/syndtr/goleveldb/leveldb/memdb", - "revision": "c4c61651e9e37fa117f53c5a906d3b63090d8445", - "revisionTime": "2018-07-08T03:05:51Z" + "revision": "b001fa50d6b27f3f0bb175a87d0cb55426d0a0ae", + "revisionTime": "2018-11-28T10:09:59Z" }, { - "checksumSHA1": "UmQeotV+m8/FduKEfLOhjdp18rs=", + "checksumSHA1": "o2TorI3z+vc+EBMJ8XeFoUmXBtU=", "path": "github.com/syndtr/goleveldb/leveldb/opt", - "revision": "c4c61651e9e37fa117f53c5a906d3b63090d8445", - "revisionTime": "2018-07-08T03:05:51Z" + "revision": "b001fa50d6b27f3f0bb175a87d0cb55426d0a0ae", + "revisionTime": "2018-11-28T10:09:59Z" }, { "checksumSHA1": "ZnyuciM+R19NG8L5YS3TIJdo1e8=", "path": "github.com/syndtr/goleveldb/leveldb/storage", - "revision": "c4c61651e9e37fa117f53c5a906d3b63090d8445", - "revisionTime": "2018-07-08T03:05:51Z" + "revision": "b001fa50d6b27f3f0bb175a87d0cb55426d0a0ae", + "revisionTime": "2018-11-28T10:09:59Z" }, { "checksumSHA1": "gWFPMz8OQeul0t54RM66yMTX49g=", "path": "github.com/syndtr/goleveldb/leveldb/table", - "revision": "c4c61651e9e37fa117f53c5a906d3b63090d8445", - "revisionTime": "2018-07-08T03:05:51Z" + "revision": "b001fa50d6b27f3f0bb175a87d0cb55426d0a0ae", + "revisionTime": "2018-11-28T10:09:59Z" }, { "checksumSHA1": "V/Dh7NV0/fy/5jX1KaAjmGcNbzI=", "path": "github.com/syndtr/goleveldb/leveldb/util", - "revision": "c4c61651e9e37fa117f53c5a906d3b63090d8445", - "revisionTime": "2018-07-08T03:05:51Z" + "revision": "b001fa50d6b27f3f0bb175a87d0cb55426d0a0ae", + "revisionTime": "2018-11-28T10:09:59Z" }, { "checksumSHA1": "nD6S4KB0S+YHxVMDDE+w3PyXaMk=", -- 2.45.2 From 55a4ff806fc1c8018055c56abcc6857502447902 Mon Sep 17 00:00:00 2001 From: zah Date: Thu, 29 Nov 2018 11:57:00 +0200 Subject: [PATCH 38/48] remove a no-op line in the code (#17760) --- les/flowcontrol/control.go | 1 - 1 file changed, 1 deletion(-) diff --git a/les/flowcontrol/control.go b/les/flowcontrol/control.go index d50eb809c..8ef4ba511 100644 --- a/les/flowcontrol/control.go +++ b/les/flowcontrol/control.go @@ -82,7 +82,6 @@ func (peer *ClientNode) RequestProcessed(cost uint64) (bv, realCost uint64) { time := mclock.Now() peer.recalcBV(time) peer.bufValue -= cost - peer.recalcBV(time) rcValue, rcost := peer.cm.processed(peer.cmNode, time) if rcValue < peer.params.BufLimit { bv := peer.params.BufLimit - rcValue -- 2.45.2 From a4428c505e84aaa1982d4cc522e0c0ffe608890a Mon Sep 17 00:00:00 2001 From: Eugene Valeyev Date: Thu, 29 Nov 2018 13:02:31 +0300 Subject: [PATCH 39/48] mobile: added constructor for BigInts (#17828) --- mobile/big.go | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/mobile/big.go b/mobile/big.go index dd7b15878..86ea93245 100644 --- a/mobile/big.go +++ b/mobile/big.go @@ -84,6 +84,13 @@ func (bi *BigInt) SetString(x string, base int) { // BigInts represents a slice of big ints. type BigInts struct{ bigints []*big.Int } +// NewBigInts creates a slice of uninitialized big numbers. +func NewBigInts(size int) *BigInts { + return &BigInts{ + bigints: make([]*big.Int, size), + } +} + // Size returns the number of big ints in the slice. func (bi *BigInts) Size() int { return len(bi.bigints) -- 2.45.2 From 32d35c9c088463efac49aeb0f3e6d48cfb373a40 Mon Sep 17 00:00:00 2001 From: Wenbiao Zheng Date: Thu, 29 Nov 2018 18:04:56 +0800 Subject: [PATCH 40/48] accounts/keystore: delete the redundant keystore in filename (#17930) * accounts/keystore: reduce file name length * accounts/keystore: reduce code line width --- accounts/keystore/account_cache.go | 5 ++++- accounts/keystore/key.go | 8 ++++++-- .../keystore/{keystore_passphrase.go => passphrase.go} | 1 + .../{keystore_passphrase_test.go => passphrase_test.go} | 0 accounts/keystore/{keystore_plain.go => plain.go} | 0 .../keystore/{keystore_plain_test.go => plain_test.go} | 0 accounts/keystore/presale.go | 8 +++++++- accounts/keystore/{keystore_wallet.go => wallet.go} | 0 8 files changed, 18 insertions(+), 4 deletions(-) rename accounts/keystore/{keystore_passphrase.go => passphrase.go} (99%) rename accounts/keystore/{keystore_passphrase_test.go => passphrase_test.go} (100%) rename accounts/keystore/{keystore_plain.go => plain.go} (100%) rename accounts/keystore/{keystore_plain_test.go => plain_test.go} (100%) rename accounts/keystore/{keystore_wallet.go => wallet.go} (100%) diff --git a/accounts/keystore/account_cache.go b/accounts/keystore/account_cache.go index da3a46eb8..8f660e282 100644 --- a/accounts/keystore/account_cache.go +++ b/accounts/keystore/account_cache.go @@ -265,7 +265,10 @@ func (ac *accountCache) scanAccounts() error { case (addr == common.Address{}): log.Debug("Failed to decode keystore key", "path", path, "err", "missing or zero address") default: - return &accounts.Account{Address: addr, URL: accounts.URL{Scheme: KeyStoreScheme, Path: path}} + return &accounts.Account{ + Address: addr, + URL: accounts.URL{Scheme: KeyStoreScheme, Path: path}, + } } return nil } diff --git a/accounts/keystore/key.go b/accounts/keystore/key.go index 0564751c4..84d8df0c5 100644 --- a/accounts/keystore/key.go +++ b/accounts/keystore/key.go @@ -171,7 +171,10 @@ func storeNewKey(ks keyStore, rand io.Reader, auth string) (*Key, accounts.Accou if err != nil { return nil, accounts.Account{}, err } - a := accounts.Account{Address: key.Address, URL: accounts.URL{Scheme: KeyStoreScheme, Path: ks.JoinPath(keyFileName(key.Address))}} + a := accounts.Account{ + Address: key.Address, + URL: accounts.URL{Scheme: KeyStoreScheme, Path: ks.JoinPath(keyFileName(key.Address))}, + } if err := ks.StoreKey(a.URL.Path, key, auth); err != nil { zeroKey(key.PrivateKey) return nil, a, err @@ -224,5 +227,6 @@ func toISO8601(t time.Time) string { } else { tz = fmt.Sprintf("%03d00", offset/3600) } - return fmt.Sprintf("%04d-%02d-%02dT%02d-%02d-%02d.%09d%s", t.Year(), t.Month(), t.Day(), t.Hour(), t.Minute(), t.Second(), t.Nanosecond(), tz) + return fmt.Sprintf("%04d-%02d-%02dT%02d-%02d-%02d.%09d%s", + t.Year(), t.Month(), t.Day(), t.Hour(), t.Minute(), t.Second(), t.Nanosecond(), tz) } diff --git a/accounts/keystore/keystore_passphrase.go b/accounts/keystore/passphrase.go similarity index 99% rename from accounts/keystore/keystore_passphrase.go rename to accounts/keystore/passphrase.go index 9794f32fe..a0b6cf538 100644 --- a/accounts/keystore/keystore_passphrase.go +++ b/accounts/keystore/passphrase.go @@ -233,6 +233,7 @@ func DecryptKey(keyjson []byte, auth string) (*Key, error) { PrivateKey: key, }, nil } + func DecryptDataV3(cryptoJson CryptoJSON, auth string) ([]byte, error) { if cryptoJson.Cipher != "aes-128-ctr" { return nil, fmt.Errorf("Cipher not supported: %v", cryptoJson.Cipher) diff --git a/accounts/keystore/keystore_passphrase_test.go b/accounts/keystore/passphrase_test.go similarity index 100% rename from accounts/keystore/keystore_passphrase_test.go rename to accounts/keystore/passphrase_test.go diff --git a/accounts/keystore/keystore_plain.go b/accounts/keystore/plain.go similarity index 100% rename from accounts/keystore/keystore_plain.go rename to accounts/keystore/plain.go diff --git a/accounts/keystore/keystore_plain_test.go b/accounts/keystore/plain_test.go similarity index 100% rename from accounts/keystore/keystore_plain_test.go rename to accounts/keystore/plain_test.go diff --git a/accounts/keystore/presale.go b/accounts/keystore/presale.go index 1554294e1..03055245f 100644 --- a/accounts/keystore/presale.go +++ b/accounts/keystore/presale.go @@ -38,7 +38,13 @@ func importPreSaleKey(keyStore keyStore, keyJSON []byte, password string) (accou return accounts.Account{}, nil, err } key.Id = uuid.NewRandom() - a := accounts.Account{Address: key.Address, URL: accounts.URL{Scheme: KeyStoreScheme, Path: keyStore.JoinPath(keyFileName(key.Address))}} + a := accounts.Account{ + Address: key.Address, + URL: accounts.URL{ + Scheme: KeyStoreScheme, + Path: keyStore.JoinPath(keyFileName(key.Address)), + }, + } err = keyStore.StoreKey(a.URL.Path, key, password) return a, key, err } diff --git a/accounts/keystore/keystore_wallet.go b/accounts/keystore/wallet.go similarity index 100% rename from accounts/keystore/keystore_wallet.go rename to accounts/keystore/wallet.go -- 2.45.2 From 7957c5a92b19dde4ad440cc0720c60373fb3b46c Mon Sep 17 00:00:00 2001 From: Elizabeth Engelman Date: Wed, 28 Nov 2018 16:15:19 -0600 Subject: [PATCH 41/48] Add a state diff service --- cmd/geth/usage.go | 6 +++++ cmd/utils/flags.go | 13 +++++++++++ statediff/service.go | 54 ++++++++++++++++++++++++++++++++++++++++++++ 3 files changed, 73 insertions(+) create mode 100644 statediff/service.go diff --git a/cmd/geth/usage.go b/cmd/geth/usage.go index 82f17e0ee..152ac059b 100644 --- a/cmd/geth/usage.go +++ b/cmd/geth/usage.go @@ -245,6 +245,12 @@ var AppHelpFlagGroups = []flagGroup{ utils.MinerLegacyExtraDataFlag, }, }, + { + Name: "STATE DIFF", + Flags: []cli.Flag{ + utils.StateDiffFlag, + }, + }, { Name: "MISC", }, diff --git a/cmd/utils/flags.go b/cmd/utils/flags.go index c18d98383..b16f9cdca 100644 --- a/cmd/utils/flags.go +++ b/cmd/utils/flags.go @@ -58,6 +58,7 @@ import ( "github.com/ethereum/go-ethereum/params" whisper "github.com/ethereum/go-ethereum/whisper/whisperv6" "gopkg.in/urfave/cli.v1" + "github.com/ethereum/go-ethereum/statediff" ) var ( @@ -1338,6 +1339,18 @@ func RegisterEthStatsService(stack *node.Node, url string) { } } +func RegisterStateDiffService(stack *node.Node) { + if err := stack.Register(func(ctx *node.ServiceContext) (node.Service, error) { + var ethServ *eth.Ethereum + ctx.Service(ðServ) + chainDb := ethServ.ChainDb() + blockChain := ethServ.BlockChain() + return statediff.NewStateDiffService(chainDb, blockChain) + }); err != nil { + Fatalf("Failed to register State Diff Service", err) + } +} + func SetupMetrics(ctx *cli.Context) { if metrics.Enabled { log.Info("Enabling metrics collection") diff --git a/statediff/service.go b/statediff/service.go new file mode 100644 index 000000000..695e3d638 --- /dev/null +++ b/statediff/service.go @@ -0,0 +1,54 @@ +package statediff + +import ( + "github.com/ethereum/go-ethereum/p2p" + "github.com/ethereum/go-ethereum/rpc" + "github.com/ethereum/go-ethereum/core" + "github.com/ethereum/go-ethereum/ethdb" + "fmt" +) + +type StateDiffService struct { + builder *builder + extractor *extractor + blockchain *core.BlockChain +} + +func NewStateDiffService(db ethdb.Database, blockChain *core.BlockChain) (*StateDiffService, error) { + config := Config{} + extractor, _ := NewExtractor(db, config) + return &StateDiffService{ + blockchain: blockChain, + extractor: extractor, + }, nil +} + +func (StateDiffService) Protocols() []p2p.Protocol { + return []p2p.Protocol{} + +} + +func (StateDiffService) APIs() []rpc.API { + return []rpc.API{} +} + +func (sds *StateDiffService) Start(server *p2p.Server) error { + fmt.Println("starting the state diff service") + blockChannel := make(chan core.ChainHeadEvent) + sds.blockchain.SubscribeChainHeadEvent(blockChannel) + for { + select { + case <-blockChannel: + headOfChainEvent := <-blockChannel + previousBlock := headOfChainEvent.Block + //TODO: figure out the best way to get the previous block + currentBlock := headOfChainEvent.Block + sds.extractor.ExtractStateDiff(*previousBlock, *currentBlock) + } + } + return nil +} +func (StateDiffService) Stop() error { + fmt.Println("stopping the state diff service") + return nil +} \ No newline at end of file -- 2.45.2 From 6fd20597a77fa0a1a5d0fd83d9c6bb82542589bc Mon Sep 17 00:00:00 2001 From: Elizabeth Engelman Date: Wed, 28 Nov 2018 16:20:25 -0600 Subject: [PATCH 42/48] Remove diff extractor from blockchain --- core/blockchain.go | 19 ------------------- eth/backend.go | 1 - 2 files changed, 20 deletions(-) diff --git a/core/blockchain.go b/core/blockchain.go index 77295b888..1b48020a6 100644 --- a/core/blockchain.go +++ b/core/blockchain.go @@ -44,7 +44,6 @@ import ( "github.com/ethereum/go-ethereum/rlp" "github.com/ethereum/go-ethereum/trie" "github.com/hashicorp/golang-lru" - "github.com/ethereum/go-ethereum/statediff" ) var ( @@ -73,7 +72,6 @@ type CacheConfig struct { TrieCleanLimit int // Memory allowance (MB) to use for caching trie nodes in memory TrieDirtyLimit int // Memory limit (MB) at which to start flushing dirty trie nodes to disk TrieTimeLimit time.Duration // Time limit after which to flush the current in-memory trie to disk - StateDiff statediff.Config // Settings for state diff extraction } // BlockChain represents the canonical chain given a database with a genesis @@ -135,8 +133,6 @@ type BlockChain struct { badBlocks *lru.Cache // Bad block cache shouldPreserve func(*types.Block) bool // Function used to determine whether should preserve the given block. - - diffExtractor statediff.Extractor // State diff processing interface } // NewBlockChain returns a fully initialised block chain using information @@ -178,12 +174,6 @@ func NewBlockChain(db ethdb.Database, cacheConfig *CacheConfig, chainConfig *par bc.SetProcessor(NewStateProcessor(chainConfig, bc, engine)) var err error - if cacheConfig.StateDiff.On { - bc.diffExtractor, err = statediff.NewExtractor(db, cacheConfig.StateDiff) - if err != nil { - return nil, err - } - } bc.hc, err = NewHeaderChain(db, chainConfig, engine, bc.getProcInterrupt) if err != nil { @@ -1216,15 +1206,6 @@ func (bc *BlockChain) insertChain(chain types.Blocks) (int, []interface{}, []*ty } proctime := time.Since(bstart) - // If extracting statediffs, do so now - if bc.cacheConfig.StateDiff.On { - // Currently not doing anything with returned cid... - _, err = bc.diffExtractor.ExtractStateDiff(*parent, *block) - if err != nil { - return i, events, coalescedLogs, err - } - } - // Write the block to the chain and get the status. status, err := bc.WriteBlockWithState(block, receipts, state) if err != nil { diff --git a/eth/backend.go b/eth/backend.go index d96175643..0de0a1980 100644 --- a/eth/backend.go +++ b/eth/backend.go @@ -159,7 +159,6 @@ func New(ctx *node.ServiceContext, config *Config) (*Ethereum, error) { TrieCleanLimit: config.TrieCleanCache, TrieDirtyLimit: config.TrieDirtyCache, TrieTimeLimit: config.TrieTimeout, - StateDiff: config.StateDiff.On, } ) eth.blockchain, err = core.NewBlockChain(chainDb, cacheConfig, eth.chainConfig, eth.engine, vmConfig, eth.shouldPreserve) -- 2.45.2 From 32a0a238216db3fd988bf42f818cefb9d6fe5c9c Mon Sep 17 00:00:00 2001 From: Elizabeth Engelman Date: Wed, 28 Nov 2018 16:51:16 -0600 Subject: [PATCH 43/48] Update imports --- statediff/ipfs/dag_putter.go | 3 ++- statediff/publisher.go | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/statediff/ipfs/dag_putter.go b/statediff/ipfs/dag_putter.go index 92ffa1e8a..4f99bdae3 100644 --- a/statediff/ipfs/dag_putter.go +++ b/statediff/ipfs/dag_putter.go @@ -22,9 +22,10 @@ package ipfs import ( "bytes" "encoding/gob" - "github.com/i-norden/go-ethereum/statediff" ipld "gx/ipfs/QmWi2BYBL5gJ3CiAiQchg6rn1A8iBsrWy51EYxvHVjFvLb/go-ipld-format" + "github.com/ethereum/go-ethereum/statediff" + "github.com/ethereum/go-ethereum/common" ) const ( diff --git a/statediff/publisher.go b/statediff/publisher.go index 163f8d3b0..48ad7cf64 100644 --- a/statediff/publisher.go +++ b/statediff/publisher.go @@ -21,7 +21,7 @@ package statediff import ( "errors" - "github.com/i-norden/go-ethereum/statediff/ipfs" + "github.com/ethereum/go-ethereum/statediff/ipfs" ) type Publisher interface { -- 2.45.2 From 70d531dc1ef8418d1f03341891fbd06d20a7a93a Mon Sep 17 00:00:00 2001 From: Elizabeth Engelman Date: Thu, 29 Nov 2018 11:05:00 -0600 Subject: [PATCH 44/48] Move statediff on/off check to geth cmd config --- cmd/geth/config.go | 4 ++++ cmd/utils/flags.go | 11 ----------- eth/config.go | 4 ---- 3 files changed, 4 insertions(+), 15 deletions(-) diff --git a/cmd/geth/config.go b/cmd/geth/config.go index b0749d232..a05e78b11 100644 --- a/cmd/geth/config.go +++ b/cmd/geth/config.go @@ -178,6 +178,10 @@ func makeFullNode(ctx *cli.Context) *node.Node { if cfg.Ethstats.URL != "" { utils.RegisterEthStatsService(stack, cfg.Ethstats.URL) } + + if ctx.GlobalBool(utils.StateDiffFlag.Name) { + utils.RegisterStateDiffService(stack) + } return stack } diff --git a/cmd/utils/flags.go b/cmd/utils/flags.go index b16f9cdca..54e3dd83d 100644 --- a/cmd/utils/flags.go +++ b/cmd/utils/flags.go @@ -1134,13 +1134,6 @@ func SetShhConfig(ctx *cli.Context, stack *node.Node, cfg *whisper.Config) { } } -// Check if state diff flags are on and applies them to eth context -func setStateDiff(ctx *cli.Context, cfg *eth.Config) { - if ctx.GlobalBool(StateDiffFlag.Name) && cfg.NoPruning && cfg.SyncMode == downloader.FullSync { - cfg.StateDiff.On = true - } -} - // SetEthConfig applies eth-related command line flags to the config. func SetEthConfig(ctx *cli.Context, stack *node.Node, cfg *eth.Config) { // Avoid conflicting network flags @@ -1176,10 +1169,6 @@ func SetEthConfig(ctx *cli.Context, stack *node.Node, cfg *eth.Config) { } cfg.NoPruning = ctx.GlobalString(GCModeFlag.Name) == "archive" - if ctx.GlobalIsSet(StateDiffFlag.Name) { - setStateDiff(ctx, cfg) - } - if ctx.GlobalIsSet(CacheFlag.Name) || ctx.GlobalIsSet(CacheTrieFlag.Name) { cfg.TrieCleanCache = ctx.GlobalInt(CacheFlag.Name) * ctx.GlobalInt(CacheTrieFlag.Name) / 100 } diff --git a/eth/config.go b/eth/config.go index c675b6d08..601f4735e 100644 --- a/eth/config.go +++ b/eth/config.go @@ -17,7 +17,6 @@ package eth import ( - "github.com/ethereum/go-ethereum/statediff" "math/big" "os" "os/user" @@ -129,9 +128,6 @@ type Config struct { EWASMInterpreter string // Type of the EVM interpreter ("" for default) EVMInterpreter string - - // Config for state diff building - StateDiff statediff.Config } type configMarshaling struct { -- 2.45.2 From 71147861f70b808669f453550ee5bdf44db0610d Mon Sep 17 00:00:00 2001 From: Elizabeth Engelman Date: Thu, 29 Nov 2018 13:52:02 -0600 Subject: [PATCH 45/48] Update starting state diff service --- statediff/service.go | 38 ++++++++++++++++++++++++-------------- 1 file changed, 24 insertions(+), 14 deletions(-) diff --git a/statediff/service.go b/statediff/service.go index 695e3d638..6870ce047 100644 --- a/statediff/service.go +++ b/statediff/service.go @@ -1,11 +1,12 @@ package statediff import ( - "github.com/ethereum/go-ethereum/p2p" - "github.com/ethereum/go-ethereum/rpc" "github.com/ethereum/go-ethereum/core" "github.com/ethereum/go-ethereum/ethdb" - "fmt" + "github.com/ethereum/go-ethereum/p2p" + "github.com/ethereum/go-ethereum/rpc" + "github.com/ethereum/go-ethereum/event" + "log" ) type StateDiffService struct { @@ -19,7 +20,7 @@ func NewStateDiffService(db ethdb.Database, blockChain *core.BlockChain) (*State extractor, _ := NewExtractor(db, config) return &StateDiffService{ blockchain: blockChain, - extractor: extractor, + extractor: extractor, }, nil } @@ -32,23 +33,32 @@ func (StateDiffService) APIs() []rpc.API { return []rpc.API{} } -func (sds *StateDiffService) Start(server *p2p.Server) error { - fmt.Println("starting the state diff service") - blockChannel := make(chan core.ChainHeadEvent) - sds.blockchain.SubscribeChainHeadEvent(blockChannel) +func (sds *StateDiffService) loop (sub event.Subscription, events chan core.ChainHeadEvent) { + defer sub.Unsubscribe() + for { select { - case <-blockChannel: - headOfChainEvent := <-blockChannel - previousBlock := headOfChainEvent.Block + case ev, ok := <-events: + if !ok { + log.Fatalf("Error getting chain head event from subscription.") + } + log.Println("doing something with an event", ev) + previousBlock := ev.Block //TODO: figure out the best way to get the previous block - currentBlock := headOfChainEvent.Block + currentBlock := ev.Block sds.extractor.ExtractStateDiff(*previousBlock, *currentBlock) } } + +} +func (sds *StateDiffService) Start(server *p2p.Server) error { + events := make(chan core.ChainHeadEvent, 10) + sub := sds.blockchain.SubscribeChainHeadEvent(events) + + go sds.loop(sub, events) + return nil } func (StateDiffService) Stop() error { - fmt.Println("stopping the state diff service") return nil -} \ No newline at end of file +} -- 2.45.2 From cf4ed9c2b378904a24cb42fe90d6af4e99a0aa66 Mon Sep 17 00:00:00 2001 From: Elizabeth Engelman Date: Thu, 29 Nov 2018 14:29:24 -0600 Subject: [PATCH 46/48] Add debugging logs for creating diff --- statediff/builder.go | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/statediff/builder.go b/statediff/builder.go index 433353591..edfb50ca8 100644 --- a/statediff/builder.go +++ b/statediff/builder.go @@ -49,10 +49,13 @@ func (sdb *builder) BuildStateDiff(oldStateRoot, newStateRoot common.Hash, block // Generate tries for old and new states oldTrie, err := trie.New(oldStateRoot, sdb.trieDB) if err != nil { + log.Debug("error creating oldTrie", err) + //getting this error: error creating oldTrie missing trie node ddfbb83966d870891aa47147269447a83564d1defaefad5f9844a3a3a2a08433 (path ) return nil, err } newTrie, err := trie.New(newStateRoot, sdb.trieDB) if err != nil { + log.Debug("error creating newTrie", err) return nil, err } @@ -61,6 +64,7 @@ func (sdb *builder) BuildStateDiff(oldStateRoot, newStateRoot common.Hash, block newIt := newTrie.NodeIterator([]byte{}) creations, err := sdb.collectDiffNodes(oldIt, newIt) if err != nil { + log.Debug("error collecting creation diff nodes", err) return nil, err } @@ -69,6 +73,7 @@ func (sdb *builder) BuildStateDiff(oldStateRoot, newStateRoot common.Hash, block newIt = newTrie.NodeIterator(make([]byte, 0)) deletions, err := sdb.collectDiffNodes(newIt, oldIt) if err != nil { + log.Debug("error collecting deletion diff nodes", err) return nil, err } @@ -80,14 +85,17 @@ func (sdb *builder) BuildStateDiff(oldStateRoot, newStateRoot common.Hash, block // Build and return the statediff updatedAccounts, err := sdb.buildDiffIncremental(creations, deletions, updatedKeys) if err != nil { + log.Debug("error building diff incremental for updated", err) return nil, err } createdAccounts, err := sdb.buildDiffEventual(creations, true) if err != nil { + log.Debug("error building diff incremental for created", err) return nil, err } deletedAccounts, err := sdb.buildDiffEventual(deletions, false) if err != nil { + log.Debug("error building diff incremental for deleted", err) return nil, err } -- 2.45.2 From 097a9071ccee5df4f5f3b2a63936d89f945b6577 Mon Sep 17 00:00:00 2001 From: Elizabeth Engelman Date: Mon, 3 Dec 2018 16:23:15 -0600 Subject: [PATCH 47/48] Add state diff extractor test and small refactoring WIP - Add state diff builder test Clean up builder test --- statediff/builder.go | 34 ++-- statediff/builder_test.go | 299 ++++++++++++++++++++++++++++++ statediff/extractor.go | 20 +- statediff/extractor_test.go | 81 +++++++- statediff/service.go | 8 +- statediff/statediff_suite_test.go | 13 ++ statediff/struct.go | 22 +-- statediff/testhelpers/mocks.go | 49 +++++ 8 files changed, 483 insertions(+), 43 deletions(-) create mode 100644 statediff/statediff_suite_test.go create mode 100644 statediff/testhelpers/mocks.go diff --git a/statediff/builder.go b/statediff/builder.go index edfb50ca8..da58b3bf4 100644 --- a/statediff/builder.go +++ b/statediff/builder.go @@ -109,7 +109,7 @@ func (sdb *builder) BuildStateDiff(oldStateRoot, newStateRoot common.Hash, block } func (sdb *builder) collectDiffNodes(a, b trie.NodeIterator) (map[common.Address]*state.Account, error) { - var diffAccounts map[common.Address]*state.Account + var diffAccounts = make(map[common.Address]*state.Account) it, _ := trie.NewDifferenceIterator(a, b) for { @@ -161,15 +161,15 @@ func (sdb *builder) buildDiffEventual(accounts map[common.Address]*state.Account hexRoot := val.Root.Hex() if created { - nonce := diffUint64{ + nonce := DiffUint64{ NewValue: &val.Nonce, } - balance := diffBigInt{ + balance := DiffBigInt{ NewValue: val.Balance, } - contractRoot := diffString{ + contractRoot := DiffString{ NewValue: &hexRoot, } accountDiffs[addr] = AccountDiffEventual{ @@ -181,13 +181,13 @@ func (sdb *builder) buildDiffEventual(accounts map[common.Address]*state.Account Storage: storageDiffs, } } else { - nonce := diffUint64{ + nonce := DiffUint64{ OldValue: &val.Nonce, } - balance := diffBigInt{ + balance := DiffBigInt{ OldValue: val.Balance, } - contractRoot := diffString{ + contractRoot := DiffString{ OldValue: &hexRoot, } accountDiffs[addr] = AccountDiffEventual{ @@ -214,12 +214,12 @@ func (sdb *builder) buildDiffIncremental(creations map[common.Address]*state.Acc log.Error("Failed building storage diffs", "Address", val, "error", err) return nil, err } else { - nonce := diffUint64{ + nonce := DiffUint64{ NewValue: &createdAcc.Nonce, OldValue: &deletedAcc.Nonce, } - balance := diffBigInt{ + balance := DiffBigInt{ NewValue: createdAcc.Balance, OldValue: deletedAcc.Balance, } @@ -227,7 +227,7 @@ func (sdb *builder) buildDiffIncremental(creations map[common.Address]*state.Acc nHexRoot := createdAcc.Root.Hex() oHexRoot := deletedAcc.Root.Hex() - contractRoot := diffString{ + contractRoot := DiffString{ NewValue: &nHexRoot, OldValue: &oHexRoot, } @@ -246,14 +246,14 @@ func (sdb *builder) buildDiffIncremental(creations map[common.Address]*state.Acc return updatedAccounts, nil } -func (sdb *builder) buildStorageDiffsEventual(sr common.Hash, creation bool) (map[string]diffString, error) { +func (sdb *builder) buildStorageDiffsEventual(sr common.Hash, creation bool) (map[string]DiffString, error) { log.Debug("Storage Root For Eventual Diff", "root", sr.Hex()) sTrie, err := trie.New(sr, sdb.trieDB) if err != nil { return nil, err } it := sTrie.NodeIterator(make([]byte, 0)) - storageDiffs := make(map[string]diffString) + storageDiffs := make(map[string]DiffString) for { log.Debug("Iterating over state at path ", "path", pathToStr(it)) if it.Leaf() { @@ -261,9 +261,9 @@ func (sdb *builder) buildStorageDiffsEventual(sr common.Hash, creation bool) (ma path := pathToStr(it) value := common.ToHex(it.LeafBlob()) if creation { - storageDiffs[path] = diffString{NewValue: &value} + storageDiffs[path] = DiffString{NewValue: &value} } else { - storageDiffs[path] = diffString{OldValue: &value} + storageDiffs[path] = DiffString{OldValue: &value} } } cont := it.Next(true) @@ -274,7 +274,7 @@ func (sdb *builder) buildStorageDiffsEventual(sr common.Hash, creation bool) (ma return storageDiffs, nil } -func (sdb *builder) buildStorageDiffsIncremental(oldSR common.Hash, newSR common.Hash) (map[string]diffString, error) { +func (sdb *builder) buildStorageDiffsIncremental(oldSR common.Hash, newSR common.Hash) (map[string]DiffString, error) { log.Debug("Storage Roots for Incremental Diff", "old", oldSR.Hex(), "new", newSR.Hex()) oldTrie, err := trie.New(oldSR, sdb.trieDB) if err != nil { @@ -288,7 +288,7 @@ func (sdb *builder) buildStorageDiffsIncremental(oldSR common.Hash, newSR common oldIt := oldTrie.NodeIterator(make([]byte, 0)) newIt := newTrie.NodeIterator(make([]byte, 0)) it, _ := trie.NewDifferenceIterator(oldIt, newIt) - storageDiffs := make(map[string]diffString) + storageDiffs := make(map[string]DiffString) for { if it.Leaf() { log.Debug("Found leaf in storage", "path", pathToStr(it)) @@ -298,7 +298,7 @@ func (sdb *builder) buildStorageDiffsIncremental(oldSR common.Hash, newSR common log.Error("Failed to look up value in oldTrie", "path", path, "error", err) } else { hexOldVal := common.ToHex(oldVal) - storageDiffs[path] = diffString{OldValue: &hexOldVal, NewValue: &value} + storageDiffs[path] = DiffString{OldValue: &hexOldVal, NewValue: &value} } } diff --git a/statediff/builder_test.go b/statediff/builder_test.go index 6433b167a..64986af10 100644 --- a/statediff/builder_test.go +++ b/statediff/builder_test.go @@ -19,3 +19,302 @@ package statediff_test +import ( + "github.com/onsi/ginkgo" + "github.com/ethereum/go-ethereum/ethdb" + "github.com/ethereum/go-ethereum/core" + "github.com/ethereum/go-ethereum/params" + "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/consensus/ethash" + "math/big" + "github.com/ethereum/go-ethereum/crypto" + "github.com/ethereum/go-ethereum/statediff" + "github.com/onsi/gomega" +) + + +var ( + testdb = ethdb.NewMemDatabase() + + testBankKey, _ = crypto.HexToECDSA("b71c71a67e1177ad4e901695e1b4b9ee17ae16c6668d313eac2f96dbcda3f291") + testBankAddress = crypto.PubkeyToAddress(testBankKey.PublicKey) //0x71562b71999873DB5b286dF957af199Ec94617F7 + testBankFunds = big.NewInt(100000000) + genesis = core.GenesisBlockForTesting(testdb, testBankAddress, testBankFunds) + + account1Key, _ = crypto.HexToECDSA("8a1f9a8f95be41cd7ccb6168179afb4504aefe388d1e14474d32c45c72ce7b7a") + account2Key, _ = crypto.HexToECDSA("49a7b37aa6f6645917e7b807e9d1c00d4fa71f18343b0d4122a4d2df64dd6fee") + account1Addr = crypto.PubkeyToAddress(account1Key.PublicKey) //0x703c4b2bD70c169f5717101CaeE543299Fc946C7 + account2Addr = crypto.PubkeyToAddress(account2Key.PublicKey) //0x0D3ab14BBaD3D99F4203bd7a11aCB94882050E7e + + contractCode = common.Hex2Bytes("606060405260cc8060106000396000f360606040526000357c01000000000000000000000000000000000000000000000000000000009004806360cd2685146041578063c16431b914606b57603f565b005b6055600480803590602001909190505060a9565b6040518082815260200191505060405180910390f35b60886004808035906020019091908035906020019091905050608a565b005b80600060005083606481101560025790900160005b50819055505b5050565b6000600060005082606481101560025790900160005b5054905060c7565b91905056") + contractAddr common.Address + + emptyAccountDiffEventualMap = make(map[common.Address]statediff.AccountDiffEventual) + emptyAccountDiffIncrementalMap = make(map[common.Address]statediff.AccountDiffIncremental) +) +/* +contract test { + + uint256[100] data; + + function Put(uint256 addr, uint256 value) { + data[addr] = value; + } + + function Get(uint256 addr) constant returns (uint256 value) { + return data[addr]; + } +} +*/ + + +// makeChain creates a chain of n blocks starting at and including parent. +// the returned hash chain is ordered head->parent. In addition, every 3rd block +// contains a transaction and every 5th an uncle to allow testing correct block +// reassembly. +func makeChain(n int, seed byte, parent *types.Block) ([]common.Hash, map[common.Hash]*types.Block) { + blocks, _ := core.GenerateChain(params.TestChainConfig, parent, ethash.NewFaker(), testdb, n, testChainGen) + hashes := make([]common.Hash, n+1) + hashes[len(hashes)-1] = parent.Hash() + blockm := make(map[common.Hash]*types.Block, n+1) + blockm[parent.Hash()] = parent + for i, b := range blocks { + hashes[len(hashes)-i-2] = b.Hash() + blockm[b.Hash()] = b + } + return hashes, blockm +} + + +func testChainGen(i int, block *core.BlockGen) { + signer := types.HomesteadSigner{} + switch i { + case 0: + // In block 1, the test bank sends account #1 some ether. + tx, _ := types.SignTx(types.NewTransaction(block.TxNonce(testBankAddress), account1Addr, big.NewInt(10000), params.TxGas, nil, nil), signer, testBankKey) + block.AddTx(tx) + case 1: + // In block 2, the test bank sends some more ether to account #1. + // account1Addr passes it on to account #2. + // account1Addr creates a test contract. + tx1, _ := types.SignTx(types.NewTransaction(block.TxNonce(testBankAddress), account1Addr, big.NewInt(1000), params.TxGas, nil, nil), signer, testBankKey) + nonce := block.TxNonce(account1Addr) + tx2, _ := types.SignTx(types.NewTransaction(nonce, account2Addr, big.NewInt(1000), params.TxGas, nil, nil), signer, account1Key) + nonce++ + tx3, _ := types.SignTx(types.NewContractCreation(nonce, big.NewInt(0), 1000000, big.NewInt(0), contractCode), signer, account1Key) + contractAddr = crypto.CreateAddress(account1Addr, nonce) //0xaE9BEa628c4Ce503DcFD7E305CaB4e29E7476592 + block.AddTx(tx1) + block.AddTx(tx2) + block.AddTx(tx3) + case 2: + // Block 3 is empty but was mined by account #2. + block.SetCoinbase(account2Addr) + //get function: 60cd2685 + //put function: c16431b9 + data := common.Hex2Bytes("C16431B900000000000000000000000000000000000000000000000000000000000000020000000000000000000000000000000000000000000000000000000000000003") + tx, _ := types.SignTx(types.NewTransaction(block.TxNonce(testBankAddress), contractAddr, big.NewInt(0), 100000, nil, data), signer, testBankKey) + block.AddTx(tx) + } +} +var _ = ginkgo.FDescribe("", func() { + var ( + block0Hash, block1Hash, block2Hash, block3Hash common.Hash + block0, block1, block2, block3 *types.Block + builder statediff.Builder + miningReward = int64(3000000000000000000) + burnAddress = common.HexToAddress("0x0") + diff *statediff.StateDiff + err error + ) + + ginkgo.BeforeEach(func() { + _, blocks := makeChain(3, 0, genesis) + block0Hash = common.HexToHash("0xd1721cfd0b29c36fd7a68f25c128e86413fb666a6e1d68e89b875bd299262661") + block1Hash = common.HexToHash("0x47c398dd688eaa4dd11b006888156783fe32df83d59b197c0fcd303408103d39") + block2Hash = common.HexToHash("0x351b2f531838683ba457e8ca4d3a844cc48147dceafbcb589dc6e3227856ee75") + block3Hash = common.HexToHash("0xfa40fbe2d98d98b3363a778d52f2bcd29d6790b9b3f3cab2b167fd12d3550f73") + + block0 = blocks[block0Hash] + block1 = blocks[block1Hash] + block2 = blocks[block2Hash] + block3 = blocks[block3Hash] + builder = statediff.NewBuilder(testdb) + }) + + ginkgo.It("returns empty account diff collections when the state root hasn't changed", func() { + expectedDiff := statediff.StateDiff{ + BlockNumber: block0.Number().Int64(), + BlockHash: block0Hash, + CreatedAccounts: emptyAccountDiffEventualMap, + DeletedAccounts: emptyAccountDiffEventualMap, + UpdatedAccounts: emptyAccountDiffIncrementalMap, + } + + diff, err := builder.BuildStateDiff(block0.Root(), block0.Root(), block0.Number().Int64(), block0Hash) + + gomega.Expect(err).NotTo(gomega.HaveOccurred()) + gomega.Expect(diff).To(gomega.Equal(&expectedDiff)) + }) + + ginkgo.Context("Block 1", func() { + //10000 transferred from testBankAddress to account1Addr + var balanceChange = int64(10000) + + ginkgo.BeforeEach(func() { + diff, err = builder.BuildStateDiff(block0.Root(), block1.Root(), block1.Number().Int64(), block1Hash) + + gomega.Expect(err).NotTo(gomega.HaveOccurred()) + }) + + ginkgo.It("includes the block number and hash", func() { + gomega.Expect(diff.BlockNumber).To(gomega.Equal(block1.Number().Int64())) + gomega.Expect(diff.BlockHash).To(gomega.Equal(block1Hash)) + }) + + ginkgo.It("returns an empty collection for deleted accounts", func() { + gomega.Expect(diff.DeletedAccounts).To(gomega.Equal(emptyAccountDiffEventualMap)) + }) + + ginkgo.It("returns balance diffs for updated accounts", func() { + expectedBankBalanceDiff := statediff.DiffBigInt{ + NewValue: big.NewInt(testBankFunds.Int64() - balanceChange), + OldValue: testBankFunds, + } + + gomega.Expect(len(diff.UpdatedAccounts)).To(gomega.Equal(1)) + gomega.Expect(diff.UpdatedAccounts[testBankAddress].Balance).To(gomega.Equal(expectedBankBalanceDiff)) + }) + + ginkgo.It("returns balance diffs for new accounts", func() { + expectedAccount1BalanceDiff := statediff.DiffBigInt{ + NewValue: big.NewInt(balanceChange), + OldValue: nil, + } + + expectedBurnAddrBalanceDiff := statediff.DiffBigInt{ + NewValue: big.NewInt(miningReward), + OldValue: nil, + } + + gomega.Expect(len(diff.CreatedAccounts)).To(gomega.Equal(2)) + gomega.Expect(diff.CreatedAccounts[account1Addr].Balance).To(gomega.Equal(expectedAccount1BalanceDiff)) + gomega.Expect(diff.CreatedAccounts[burnAddress].Balance).To(gomega.Equal(expectedBurnAddrBalanceDiff)) + }) + }) + + ginkgo.Context("Block 2", func() { + //1000 transferred from testBankAddress to account1Addr + //1000 transferred from account1Addr to account2Addr + var ( + balanceChange = int64(1000) + block1BankBalance = int64(99990000) + block1Account1Balance = int64(10000) + ) + + ginkgo.BeforeEach(func() { + diff, err = builder.BuildStateDiff(block1.Root(), block2.Root(), block2.Number().Int64(), block2Hash) + + gomega.Expect(err).NotTo(gomega.HaveOccurred()) + }) + + ginkgo.It("includes the block number and hash", func() { + gomega.Expect(diff.BlockNumber).To(gomega.Equal(block2.Number().Int64())) + gomega.Expect(diff.BlockHash).To(gomega.Equal(block2Hash)) + }) + + ginkgo.It("returns an empty collection for deleted accounts", func() { + gomega.Expect(diff.DeletedAccounts).To(gomega.Equal(emptyAccountDiffEventualMap)) + }) + + ginkgo.It("returns balance diffs for updated accounts", func() { + expectedBankBalanceDiff := statediff.DiffBigInt{ + NewValue: big.NewInt(block1BankBalance - balanceChange), + OldValue: big.NewInt(block1BankBalance), + } + + expectedAccount1BalanceDiff := statediff.DiffBigInt{ + NewValue: big.NewInt(block1Account1Balance - balanceChange + balanceChange), + OldValue: big.NewInt(block1Account1Balance), + } + + expectedBurnBalanceDiff := statediff.DiffBigInt{ + NewValue: big.NewInt(miningReward + miningReward), + OldValue: big.NewInt(miningReward), + } + + gomega.Expect(len(diff.UpdatedAccounts)).To(gomega.Equal(3)) + gomega.Expect(diff.UpdatedAccounts[testBankAddress].Balance).To(gomega.Equal(expectedBankBalanceDiff)) + gomega.Expect(diff.UpdatedAccounts[account1Addr].Balance).To(gomega.Equal(expectedAccount1BalanceDiff)) + gomega.Expect(diff.UpdatedAccounts[burnAddress].Balance).To(gomega.Equal(expectedBurnBalanceDiff)) + }) + + ginkgo.It("returns balance diffs for new accounts", func() { + expectedAccount2BalanceDiff := statediff.DiffBigInt{ + NewValue: big.NewInt(balanceChange), + OldValue: nil, + } + + expectedContractBalanceDiff := statediff.DiffBigInt{ + NewValue: big.NewInt(0), + OldValue: nil, + } + + gomega.Expect(len(diff.CreatedAccounts)).To(gomega.Equal(2)) + gomega.Expect(diff.CreatedAccounts[account2Addr].Balance).To(gomega.Equal(expectedAccount2BalanceDiff)) + gomega.Expect(diff.CreatedAccounts[contractAddr].Balance).To(gomega.Equal(expectedContractBalanceDiff)) + }) + }) + + ginkgo.Context("Block 3", func() { + //the contract's storage is changed + //and the block is mined by account 2 + ginkgo.BeforeEach(func() { + diff, err = builder.BuildStateDiff(block2.Root(), block3.Root(), block3.Number().Int64(), block3Hash) + + gomega.Expect(err).NotTo(gomega.HaveOccurred()) + }) + + ginkgo.It("includes the block number and hash", func() { + gomega.Expect(diff.BlockNumber).To(gomega.Equal(block3.Number().Int64())) + gomega.Expect(diff.BlockHash).To(gomega.Equal(block3Hash)) + }) + + ginkgo.It("returns an empty collection for deleted accounts", func() { + gomega.Expect(diff.DeletedAccounts).To(gomega.Equal(emptyAccountDiffEventualMap)) + }) + + ginkgo.It("returns an empty collection for created accounts", func() { + gomega.Expect(diff.CreatedAccounts).To(gomega.Equal(emptyAccountDiffEventualMap)) + }) + + ginkgo.It("returns balance, storage and nonce diffs for updated accounts", func() { + block2Account2Balance := int64(1000) + expectedAcct2BalanceDiff := statediff.DiffBigInt{ + NewValue: big.NewInt(block2Account2Balance + miningReward), + OldValue: big.NewInt(block2Account2Balance), + } + + expectedContractStorageDiff := make(map[string]statediff.DiffString) + newVal := "0x03" + oldVal := "0x0" + path := "0x405787fa12a823e0f2b7631cc41b3ba8828b3321ca811111fa75cd3aa3bb5ace" + expectedContractStorageDiff[path] = statediff.DiffString{ + NewValue: &newVal, + OldValue: &oldVal, + } + + oldNonce := uint64(2) + newNonce := uint64(3) + expectedBankNonceDiff := statediff.DiffUint64{ + NewValue: &newNonce, + OldValue: &oldNonce, + } + + gomega.Expect(len(diff.UpdatedAccounts)).To(gomega.Equal(3)) + gomega.Expect(diff.UpdatedAccounts[account2Addr].Balance).To(gomega.Equal(expectedAcct2BalanceDiff)) + gomega.Expect(diff.UpdatedAccounts[contractAddr].Storage).To(gomega.Equal(expectedContractStorageDiff)) + gomega.Expect(diff.UpdatedAccounts[testBankAddress].Nonce).To(gomega.Equal(expectedBankNonceDiff)) + }) + }) +}) \ No newline at end of file diff --git a/statediff/extractor.go b/statediff/extractor.go index 3cbb5a311..9d9b1f080 100644 --- a/statediff/extractor.go +++ b/statediff/extractor.go @@ -21,7 +21,6 @@ package statediff import ( "github.com/ethereum/go-ethereum/core/types" - "github.com/ethereum/go-ethereum/ethdb" ) type Extractor interface { @@ -29,27 +28,22 @@ type Extractor interface { } type extractor struct { - *builder // Interface for building state diff objects from two blocks - *publisher // Interface for publishing state diff objects to a datastore (e.g. IPFS) + Builder Builder // Interface for building state diff objects from two blocks + Publisher Publisher // Interface for publishing state diff objects to a datastore (e.g. IPFS) } -func NewExtractor(db ethdb.Database, config Config) (*extractor, error) { - publisher, err := NewPublisher(config) - if err != nil { - return nil, err - } - +func NewExtractor(builder Builder, publisher Publisher) (*extractor, error) { return &extractor{ - builder: NewBuilder(db), - publisher: publisher, + Builder: builder, + Publisher: publisher, }, nil } func (e *extractor) ExtractStateDiff(parent, current types.Block) (string, error) { - stateDiff, err := e.BuildStateDiff(parent.Root(), current.Root(), current.Number().Int64(), current.Hash()) + stateDiff, err := e.Builder.BuildStateDiff(parent.Root(), current.Root(), current.Number().Int64(), current.Hash()) if err != nil { return "", err } - return e.PublishStateDiff(stateDiff) + return e.Publisher.PublishStateDiff(stateDiff) } \ No newline at end of file diff --git a/statediff/extractor_test.go b/statediff/extractor_test.go index de5e17095..9ccd250f6 100644 --- a/statediff/extractor_test.go +++ b/statediff/extractor_test.go @@ -17,4 +17,83 @@ // Contains a batch of utility type declarations used by the tests. As the node // operates on unique types, a lot of them are needed to check various features. -package statediff_test \ No newline at end of file +package statediff_test + +import ( + "github.com/onsi/ginkgo" + "github.com/ethereum/go-ethereum/statediff" + "github.com/onsi/gomega" + "github.com/ethereum/go-ethereum/core/types" + "math/rand" + "github.com/ethereum/go-ethereum/statediff/testhelpers" + "math/big" +) +var _ = ginkgo.Describe("Extractor", func() { + var publisher testhelpers.MockPublisher + var builder testhelpers.MockBuilder + var currentBlockNumber *big.Int + var parentBlock, currentBlock *types.Block + var expectedStateDiff statediff.StateDiff + var extractor statediff.Extractor + var err error + + ginkgo.BeforeEach(func() { + publisher = testhelpers.MockPublisher{} + builder = testhelpers.MockBuilder{} + extractor, err = statediff.NewExtractor(&builder, &publisher) + gomega.Expect(err).NotTo(gomega.HaveOccurred()) + + blockNumber := rand.Int63() + parentBlockNumber := big.NewInt(blockNumber - int64(1)) + currentBlockNumber = big.NewInt(blockNumber) + parentBlock = types.NewBlock(&types.Header{Number: parentBlockNumber}, nil, nil, nil) + currentBlock = types.NewBlock(&types.Header{Number: currentBlockNumber}, nil, nil, nil) + + expectedStateDiff = statediff.StateDiff{ + BlockNumber: blockNumber, + BlockHash: currentBlock.Hash(), + CreatedAccounts: nil, + DeletedAccounts: nil, + UpdatedAccounts: nil, + } + }) + + ginkgo.It("builds a state diff struct", func() { + builder.SetStateDiffToBuild(&expectedStateDiff) + + _, err = extractor.ExtractStateDiff(*parentBlock, *currentBlock) + + gomega.Expect(err).NotTo(gomega.HaveOccurred()) + gomega.Expect(builder.OldStateRoot).To(gomega.Equal(parentBlock.Root())) + gomega.Expect(builder.NewStateRoot).To(gomega.Equal(currentBlock.Root())) + gomega.Expect(builder.BlockNumber).To(gomega.Equal(currentBlockNumber.Int64())) + gomega.Expect(builder.BlockHash).To(gomega.Equal(currentBlock.Hash())) + }) + + ginkgo.It("returns an error if building the state diff fails", func() { + builder.SetBuilderError(testhelpers.MockError) + + _, err = extractor.ExtractStateDiff(*parentBlock, *currentBlock) + + gomega.Expect(err).To(gomega.HaveOccurred()) + gomega.Expect(err).To(gomega.MatchError(testhelpers.MockError)) + }) + + ginkgo.It("publishes the state diff struct", func() { + builder.SetStateDiffToBuild(&expectedStateDiff) + + _, err = extractor.ExtractStateDiff(*parentBlock, *currentBlock) + + gomega.Expect(err).NotTo(gomega.HaveOccurred()) + gomega.Expect(publisher.StateDiff).To(gomega.Equal(&expectedStateDiff)) + }) + + ginkgo.It("returns an error if publishing the diff fails", func() { + publisher.SetPublisherError(testhelpers.MockError) + + _, err = extractor.ExtractStateDiff(*parentBlock, *currentBlock) + + gomega.Expect(err).To(gomega.HaveOccurred()) + gomega.Expect(err).To(gomega.MatchError(testhelpers.MockError)) + }) +}) diff --git a/statediff/service.go b/statediff/service.go index 6870ce047..86839cf3b 100644 --- a/statediff/service.go +++ b/statediff/service.go @@ -17,7 +17,13 @@ type StateDiffService struct { func NewStateDiffService(db ethdb.Database, blockChain *core.BlockChain) (*StateDiffService, error) { config := Config{} - extractor, _ := NewExtractor(db, config) + builder := NewBuilder(db) + publisher, err := NewPublisher(config) + if err != nil { + return nil, nil + } + + extractor, _ := NewExtractor(builder, publisher) return &StateDiffService{ blockchain: blockChain, extractor: extractor, diff --git a/statediff/statediff_suite_test.go b/statediff/statediff_suite_test.go new file mode 100644 index 000000000..7118ea792 --- /dev/null +++ b/statediff/statediff_suite_test.go @@ -0,0 +1,13 @@ +package statediff_test + +import ( + "testing" + + . "github.com/onsi/ginkgo" + . "github.com/onsi/gomega" +) + +func TestStatediff(t *testing.T) { + RegisterFailHandler(Fail) + RunSpecs(t, "Statediff Suite") +} diff --git a/statediff/struct.go b/statediff/struct.go index 2142ec755..009af4c2e 100644 --- a/statediff/struct.go +++ b/statediff/struct.go @@ -56,31 +56,31 @@ func (sd *StateDiff) Encode() ([]byte, error) { } type AccountDiffEventual struct { - Nonce diffUint64 `json:"nonce" gencodec:"required"` - Balance diffBigInt `json:"balance" gencodec:"required"` + Nonce DiffUint64 `json:"nonce" gencodec:"required"` + Balance DiffBigInt `json:"balance" gencodec:"required"` Code []byte `json:"code" gencodec:"required"` CodeHash string `json:"codeHash" gencodec:"required"` - ContractRoot diffString `json:"contractRoot" gencodec:"required"` - Storage map[string]diffString `json:"storage" gencodec:"required"` + ContractRoot DiffString `json:"contractRoot" gencodec:"required"` + Storage map[string]DiffString `json:"storage" gencodec:"required"` } type AccountDiffIncremental struct { - Nonce diffUint64 `json:"nonce" gencodec:"required"` - Balance diffBigInt `json:"balance" gencodec:"required"` + Nonce DiffUint64 `json:"nonce" gencodec:"required"` + Balance DiffBigInt `json:"balance" gencodec:"required"` CodeHash string `json:"codeHash" gencodec:"required"` - ContractRoot diffString `json:"contractRoot" gencodec:"required"` - Storage map[string]diffString `json:"storage" gencodec:"required"` + ContractRoot DiffString `json:"contractRoot" gencodec:"required"` + Storage map[string]DiffString `json:"storage" gencodec:"required"` } -type diffString struct { +type DiffString struct { NewValue *string `json:"newValue" gencodec:"optional"` OldValue *string `json:"oldValue" gencodec:"optional"` } -type diffUint64 struct { +type DiffUint64 struct { NewValue *uint64 `json:"newValue" gencodec:"optional"` OldValue *uint64 `json:"oldValue" gencodec:"optional"` } -type diffBigInt struct { +type DiffBigInt struct { NewValue *big.Int `json:"newValue" gencodec:"optional"` OldValue *big.Int `json:"oldValue" gencodec:"optional"` } diff --git a/statediff/testhelpers/mocks.go b/statediff/testhelpers/mocks.go new file mode 100644 index 000000000..4bff3c028 --- /dev/null +++ b/statediff/testhelpers/mocks.go @@ -0,0 +1,49 @@ +package testhelpers + +import ( + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/statediff" + "errors" +) + +var MockError = errors.New("mock error") + +type MockBuilder struct { + OldStateRoot common.Hash + NewStateRoot common.Hash + BlockNumber int64 + BlockHash common.Hash + stateDiff *statediff.StateDiff + builderError error +} + +func (builder *MockBuilder) BuildStateDiff(oldStateRoot, newStateRoot common.Hash, blockNumber int64, blockHash common.Hash) (*statediff.StateDiff, error) { + builder.OldStateRoot = oldStateRoot + builder.NewStateRoot = newStateRoot + builder.BlockNumber = blockNumber + builder.BlockHash = blockHash + + return builder.stateDiff, builder.builderError +} + +func (builder *MockBuilder) SetStateDiffToBuild(stateDiff *statediff.StateDiff) { + builder.stateDiff = stateDiff +} + +func (builder *MockBuilder) SetBuilderError(err error) { + builder.builderError = err +} + +type MockPublisher struct{ + StateDiff *statediff.StateDiff + publisherError error +} + +func (publisher *MockPublisher) PublishStateDiff(sd *statediff.StateDiff) (string, error) { + publisher.StateDiff = sd + return "", publisher.publisherError +} + +func (publisher *MockPublisher) SetPublisherError(err error) { + publisher.publisherError = err +} -- 2.45.2 From c1e3af009348391aa93ce0322dea0facdd5d203b Mon Sep 17 00:00:00 2001 From: Ian Norden Date: Wed, 12 Dec 2018 11:54:51 -0600 Subject: [PATCH 48/48] reorganize to fix unallowed import cycles between statediff and statediff/ipfs --- statediff/{ => builder}/builder.go | 2 +- statediff/builder/builder_suite_test.go | 32 +++++++ statediff/{ => builder}/builder_test.go | 10 +- statediff/{ => builder}/helpers.go | 2 +- statediff/{ => builder}/struct.go | 2 +- statediff/{ => extractor}/extractor.go | 14 +-- statediff/extractor/extractor_suite_test.go | 32 +++++++ statediff/{ => extractor}/extractor_test.go | 29 +++--- statediff/{ => publisher}/ipfs/adder.go | 2 +- statediff/{ => publisher}/ipfs/dag_putter.go | 12 +-- statediff/{ => publisher}/ipfs/helpers.go | 4 +- statediff/{ => publisher}/ipfs/node.go | 12 +-- statediff/{ => publisher}/publisher.go | 21 +++-- statediff/publisher/publisher_suite_test.go | 32 +++++++ statediff/publisher/publisher_test.go | 20 ++++ statediff/service.go | 70 -------------- statediff/service/service.go | 94 +++++++++++++++++++ statediff/service/service_suite_test.go | 32 +++++++ statediff/service/service_test.go | 20 ++++ ..._test.go => statediff_integration_test.go} | 0 statediff/statediff_suite_test.go | 23 ++++- statediff/testhelpers/mocks.go | 33 +++++-- 22 files changed, 369 insertions(+), 129 deletions(-) rename statediff/{ => builder}/builder.go (99%) create mode 100644 statediff/builder/builder_suite_test.go rename statediff/{ => builder}/builder_test.go (99%) rename statediff/{ => builder}/helpers.go (99%) rename statediff/{ => builder}/struct.go (99%) rename statediff/{ => extractor}/extractor.go (77%) create mode 100644 statediff/extractor/extractor_suite_test.go rename statediff/{ => extractor}/extractor_test.go (87%) rename statediff/{ => publisher}/ipfs/adder.go (96%) rename statediff/{ => publisher}/ipfs/dag_putter.go (82%) rename statediff/{ => publisher}/ipfs/helpers.go (94%) rename statediff/{ => publisher}/ipfs/node.go (87%) rename statediff/{ => publisher}/publisher.go (75%) create mode 100644 statediff/publisher/publisher_suite_test.go create mode 100644 statediff/publisher/publisher_test.go delete mode 100644 statediff/service.go create mode 100644 statediff/service/service.go create mode 100644 statediff/service/service_suite_test.go create mode 100644 statediff/service/service_test.go rename statediff/{publisher_test.go => statediff_integration_test.go} (100%) diff --git a/statediff/builder.go b/statediff/builder/builder.go similarity index 99% rename from statediff/builder.go rename to statediff/builder/builder.go index da58b3bf4..30479a60c 100644 --- a/statediff/builder.go +++ b/statediff/builder/builder.go @@ -17,7 +17,7 @@ // Contains a batch of utility type declarations used by the tests. As the node // operates on unique types, a lot of them are needed to check various features. -package statediff +package builder import ( "github.com/ethereum/go-ethereum/common" diff --git a/statediff/builder/builder_suite_test.go b/statediff/builder/builder_suite_test.go new file mode 100644 index 000000000..bbbf63078 --- /dev/null +++ b/statediff/builder/builder_suite_test.go @@ -0,0 +1,32 @@ +// Copyright 2015 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +// Contains a batch of utility type declarations used by the tests. As the node +// operates on unique types, a lot of them are needed to check various features. + +package builder_test + +import ( + "testing" + + . "github.com/onsi/ginkgo" + . "github.com/onsi/gomega" +) + +func TestStateDiffBuilder(t *testing.T) { + RegisterFailHandler(Fail) + RunSpecs(t, "StateDiff Builder Suite") +} \ No newline at end of file diff --git a/statediff/builder_test.go b/statediff/builder/builder_test.go similarity index 99% rename from statediff/builder_test.go rename to statediff/builder/builder_test.go index 64986af10..793a6347d 100644 --- a/statediff/builder_test.go +++ b/statediff/builder/builder_test.go @@ -17,20 +17,22 @@ // Contains a batch of utility type declarations used by the tests. As the node // operates on unique types, a lot of them are needed to check various features. -package statediff_test +package builder_test import ( + "math/big" + "github.com/onsi/ginkgo" + "github.com/onsi/gomega" + "github.com/ethereum/go-ethereum/ethdb" "github.com/ethereum/go-ethereum/core" "github.com/ethereum/go-ethereum/params" "github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/consensus/ethash" - "math/big" "github.com/ethereum/go-ethereum/crypto" - "github.com/ethereum/go-ethereum/statediff" - "github.com/onsi/gomega" + statediff "github.com/ethereum/go-ethereum/statediff/builder" ) diff --git a/statediff/helpers.go b/statediff/builder/helpers.go similarity index 99% rename from statediff/helpers.go rename to statediff/builder/helpers.go index 8a0a6466e..31aae926b 100644 --- a/statediff/helpers.go +++ b/statediff/builder/helpers.go @@ -17,7 +17,7 @@ // Contains a batch of utility type declarations used by the tests. As the node // operates on unique types, a lot of them are needed to check various features. -package statediff +package builder import ( "sort" diff --git a/statediff/struct.go b/statediff/builder/struct.go similarity index 99% rename from statediff/struct.go rename to statediff/builder/struct.go index 009af4c2e..7b48bba72 100644 --- a/statediff/struct.go +++ b/statediff/builder/struct.go @@ -17,7 +17,7 @@ // Contains a batch of utility type declarations used by the tests. As the node // operates on unique types, a lot of them are needed to check various features. -package statediff +package builder import ( "encoding/json" diff --git a/statediff/extractor.go b/statediff/extractor/extractor.go similarity index 77% rename from statediff/extractor.go rename to statediff/extractor/extractor.go index 9d9b1f080..d485f5e13 100644 --- a/statediff/extractor.go +++ b/statediff/extractor/extractor.go @@ -17,10 +17,12 @@ // Contains a batch of utility type declarations used by the tests. As the node // operates on unique types, a lot of them are needed to check various features. -package statediff +package extractor import ( "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/statediff/builder" + "github.com/ethereum/go-ethereum/statediff/publisher" ) type Extractor interface { @@ -28,14 +30,14 @@ type Extractor interface { } type extractor struct { - Builder Builder // Interface for building state diff objects from two blocks - Publisher Publisher // Interface for publishing state diff objects to a datastore (e.g. IPFS) + Builder builder.Builder // Interface for building state diff objects from two blocks + Publisher publisher.Publisher // Interface for publishing state diff objects to a datastore (e.g. IPFS) } -func NewExtractor(builder Builder, publisher Publisher) (*extractor, error) { +func NewExtractor(b builder.Builder, p publisher.Publisher) (*extractor, error) { return &extractor{ - Builder: builder, - Publisher: publisher, + Builder: b, + Publisher: p, }, nil } diff --git a/statediff/extractor/extractor_suite_test.go b/statediff/extractor/extractor_suite_test.go new file mode 100644 index 000000000..fe7719032 --- /dev/null +++ b/statediff/extractor/extractor_suite_test.go @@ -0,0 +1,32 @@ +// Copyright 2015 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +// Contains a batch of utility type declarations used by the tests. As the node +// operates on unique types, a lot of them are needed to check various features. + +package extractor_test + +import ( + "testing" + + . "github.com/onsi/ginkgo" + . "github.com/onsi/gomega" +) + +func TestStateDiffExtractor(t *testing.T) { + RegisterFailHandler(Fail) + RunSpecs(t, "StateDiff Extractor Suite") +} \ No newline at end of file diff --git a/statediff/extractor_test.go b/statediff/extractor/extractor_test.go similarity index 87% rename from statediff/extractor_test.go rename to statediff/extractor/extractor_test.go index 9ccd250f6..d76329726 100644 --- a/statediff/extractor_test.go +++ b/statediff/extractor/extractor_test.go @@ -17,16 +17,19 @@ // Contains a batch of utility type declarations used by the tests. As the node // operates on unique types, a lot of them are needed to check various features. -package statediff_test +package extractor_test import ( - "github.com/onsi/ginkgo" - "github.com/ethereum/go-ethereum/statediff" - "github.com/onsi/gomega" - "github.com/ethereum/go-ethereum/core/types" - "math/rand" - "github.com/ethereum/go-ethereum/statediff/testhelpers" "math/big" + "math/rand" + + "github.com/onsi/ginkgo" + "github.com/onsi/gomega" + + "github.com/ethereum/go-ethereum/statediff/extractor" + statediff "github.com/ethereum/go-ethereum/statediff/builder" + "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/statediff/testhelpers" ) var _ = ginkgo.Describe("Extractor", func() { var publisher testhelpers.MockPublisher @@ -34,13 +37,13 @@ var _ = ginkgo.Describe("Extractor", func() { var currentBlockNumber *big.Int var parentBlock, currentBlock *types.Block var expectedStateDiff statediff.StateDiff - var extractor statediff.Extractor + var ex extractor.Extractor var err error ginkgo.BeforeEach(func() { publisher = testhelpers.MockPublisher{} builder = testhelpers.MockBuilder{} - extractor, err = statediff.NewExtractor(&builder, &publisher) + ex, err = extractor.NewExtractor(&builder, &publisher) gomega.Expect(err).NotTo(gomega.HaveOccurred()) blockNumber := rand.Int63() @@ -61,7 +64,7 @@ var _ = ginkgo.Describe("Extractor", func() { ginkgo.It("builds a state diff struct", func() { builder.SetStateDiffToBuild(&expectedStateDiff) - _, err = extractor.ExtractStateDiff(*parentBlock, *currentBlock) + _, err = ex.ExtractStateDiff(*parentBlock, *currentBlock) gomega.Expect(err).NotTo(gomega.HaveOccurred()) gomega.Expect(builder.OldStateRoot).To(gomega.Equal(parentBlock.Root())) @@ -73,7 +76,7 @@ var _ = ginkgo.Describe("Extractor", func() { ginkgo.It("returns an error if building the state diff fails", func() { builder.SetBuilderError(testhelpers.MockError) - _, err = extractor.ExtractStateDiff(*parentBlock, *currentBlock) + _, err = ex.ExtractStateDiff(*parentBlock, *currentBlock) gomega.Expect(err).To(gomega.HaveOccurred()) gomega.Expect(err).To(gomega.MatchError(testhelpers.MockError)) @@ -82,7 +85,7 @@ var _ = ginkgo.Describe("Extractor", func() { ginkgo.It("publishes the state diff struct", func() { builder.SetStateDiffToBuild(&expectedStateDiff) - _, err = extractor.ExtractStateDiff(*parentBlock, *currentBlock) + _, err = ex.ExtractStateDiff(*parentBlock, *currentBlock) gomega.Expect(err).NotTo(gomega.HaveOccurred()) gomega.Expect(publisher.StateDiff).To(gomega.Equal(&expectedStateDiff)) @@ -91,7 +94,7 @@ var _ = ginkgo.Describe("Extractor", func() { ginkgo.It("returns an error if publishing the diff fails", func() { publisher.SetPublisherError(testhelpers.MockError) - _, err = extractor.ExtractStateDiff(*parentBlock, *currentBlock) + _, err = ex.ExtractStateDiff(*parentBlock, *currentBlock) gomega.Expect(err).To(gomega.HaveOccurred()) gomega.Expect(err).To(gomega.MatchError(testhelpers.MockError)) diff --git a/statediff/ipfs/adder.go b/statediff/publisher/ipfs/adder.go similarity index 96% rename from statediff/ipfs/adder.go rename to statediff/publisher/ipfs/adder.go index 23eea1c1f..eb8e012c5 100644 --- a/statediff/ipfs/adder.go +++ b/statediff/publisher/ipfs/adder.go @@ -24,7 +24,7 @@ import ( "github.com/ipfs/go-ipfs/core" "github.com/ipfs/go-ipfs/repo/fsrepo" - ipld "gx/ipfs/QmWi2BYBL5gJ3CiAiQchg6rn1A8iBsrWy51EYxvHVjFvLb/go-ipld-format" + ipld "gx/ipfs/QmcKKBwfz6FyQdHR2jsXrrF6XeSBXYL86anmWNewpFpoF5/go-ipld-format" ) type Adder interface { diff --git a/statediff/ipfs/dag_putter.go b/statediff/publisher/ipfs/dag_putter.go similarity index 82% rename from statediff/ipfs/dag_putter.go rename to statediff/publisher/ipfs/dag_putter.go index 4f99bdae3..2f2ff02e2 100644 --- a/statediff/ipfs/dag_putter.go +++ b/statediff/publisher/ipfs/dag_putter.go @@ -23,9 +23,9 @@ import ( "bytes" "encoding/gob" - ipld "gx/ipfs/QmWi2BYBL5gJ3CiAiQchg6rn1A8iBsrWy51EYxvHVjFvLb/go-ipld-format" - "github.com/ethereum/go-ethereum/statediff" - "github.com/ethereum/go-ethereum/common" + ipld "gx/ipfs/QmcKKBwfz6FyQdHR2jsXrrF6XeSBXYL86anmWNewpFpoF5/go-ipld-format" + + "github.com/ethereum/go-ethereum/statediff/builder" ) const ( @@ -33,7 +33,7 @@ const ( ) type DagPutter interface { - DagPut(sd *statediff.StateDiff) (string, error) + DagPut(sd *builder.StateDiff) (string, error) } type dagPutter struct { @@ -44,7 +44,7 @@ func NewDagPutter(adder Adder) *dagPutter { return &dagPutter{Adder: adder} } -func (bhdp *dagPutter) DagPut(sd *statediff.StateDiff) (string, error) { +func (bhdp *dagPutter) DagPut(sd *builder.StateDiff) (string, error) { nd, err := bhdp.getNode(sd) if err != nil { return "", err @@ -56,7 +56,7 @@ func (bhdp *dagPutter) DagPut(sd *statediff.StateDiff) (string, error) { return nd.Cid().String(), nil } -func (bhdp *dagPutter) getNode(sd *statediff.StateDiff) (ipld.Node, error) { +func (bhdp *dagPutter) getNode(sd *builder.StateDiff) (ipld.Node, error) { var buff bytes.Buffer enc := gob.NewEncoder(&buff) diff --git a/statediff/ipfs/helpers.go b/statediff/publisher/ipfs/helpers.go similarity index 94% rename from statediff/ipfs/helpers.go rename to statediff/publisher/ipfs/helpers.go index a9904afa5..ea6ca5ab5 100644 --- a/statediff/ipfs/helpers.go +++ b/statediff/publisher/ipfs/helpers.go @@ -21,7 +21,7 @@ package ipfs import ( mh "gx/ipfs/QmZyZDi491cCNTLfAhwcaDii2Kg4pwKRkhqQzURGDvY6ua/go-multihash" - "gx/ipfs/QmapdYm1b22Frv3k17fqrBYTFRxwiaVJkB299Mfn33edeB/go-cid" + "gx/ipfs/QmR8BauakNcBa3RbE4nbQu76PDiJgoQgz8AJdhJuiU4TAw/go-cid" ) func RawToCid(codec uint64, raw []byte) (*cid.Cid, error) { @@ -34,5 +34,5 @@ func RawToCid(codec uint64, raw []byte) (*cid.Cid, error) { if err != nil { return nil, err } - return c, nil + return &c, nil } \ No newline at end of file diff --git a/statediff/ipfs/node.go b/statediff/publisher/ipfs/node.go similarity index 87% rename from statediff/ipfs/node.go rename to statediff/publisher/ipfs/node.go index dd7447a80..1d8daa5e8 100644 --- a/statediff/ipfs/node.go +++ b/statediff/publisher/ipfs/node.go @@ -20,14 +20,14 @@ package ipfs import ( - ipld "gx/ipfs/QmWi2BYBL5gJ3CiAiQchg6rn1A8iBsrWy51EYxvHVjFvLb/go-ipld-format" - "gx/ipfs/QmapdYm1b22Frv3k17fqrBYTFRxwiaVJkB299Mfn33edeB/go-cid" + "gx/ipfs/QmR8BauakNcBa3RbE4nbQu76PDiJgoQgz8AJdhJuiU4TAw/go-cid" + ipld "gx/ipfs/QmcKKBwfz6FyQdHR2jsXrrF6XeSBXYL86anmWNewpFpoF5/go-ipld-format" - "github.com/i-norden/go-ethereum/statediff" + "github.com/ethereum/go-ethereum/statediff/builder" ) type StateDiffNode struct { - *statediff.StateDiff + *builder.StateDiff cid *cid.Cid rawdata []byte @@ -37,8 +37,8 @@ func (sdn *StateDiffNode) RawData() []byte { return sdn.rawdata } -func (sdn *StateDiffNode) Cid() *cid.Cid { - return sdn.cid +func (sdn *StateDiffNode) Cid() cid.Cid { + return *sdn.cid } func (sdn StateDiffNode) String() string { diff --git a/statediff/publisher.go b/statediff/publisher/publisher.go similarity index 75% rename from statediff/publisher.go rename to statediff/publisher/publisher.go index 48ad7cf64..404871405 100644 --- a/statediff/publisher.go +++ b/statediff/publisher/publisher.go @@ -17,23 +17,26 @@ // Contains a batch of utility type declarations used by the tests. As the node // operates on unique types, a lot of them are needed to check various features. -package statediff +package publisher import ( "errors" - "github.com/ethereum/go-ethereum/statediff/ipfs" + + "github.com/ethereum/go-ethereum/statediff" + "github.com/ethereum/go-ethereum/statediff/builder" + "github.com/ethereum/go-ethereum/statediff/publisher/ipfs" ) type Publisher interface { - PublishStateDiff(sd *StateDiff) (string, error) + PublishStateDiff(sd *builder.StateDiff) (string, error) } type publisher struct { ipfs.DagPutter - Config + statediff.Config } -func NewPublisher(config Config) (*publisher, error) { +func NewPublisher(config statediff.Config) (*publisher, error) { adder, err := ipfs.NewAdder(config.Path) if err != nil { return nil, err @@ -45,17 +48,17 @@ func NewPublisher(config Config) (*publisher, error) { }, nil } -func (p *publisher) PublishStateDiff(sd *StateDiff) (string, error) { +func (p *publisher) PublishStateDiff(sd *builder.StateDiff) (string, error) { switch p.Mode { - case IPLD: + case statediff.IPLD: cidStr, err := p.DagPut(sd) if err != nil { return "", err } return cidStr, err - case LDB: - case SQL: + case statediff.LDB: + case statediff.SQL: default: } diff --git a/statediff/publisher/publisher_suite_test.go b/statediff/publisher/publisher_suite_test.go new file mode 100644 index 000000000..29d10e2a1 --- /dev/null +++ b/statediff/publisher/publisher_suite_test.go @@ -0,0 +1,32 @@ +// Copyright 2015 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +// Contains a batch of utility type declarations used by the tests. As the node +// operates on unique types, a lot of them are needed to check various features. + +package publisher_test + +import ( + "testing" + + . "github.com/onsi/ginkgo" + . "github.com/onsi/gomega" +) + +func TestStateDiffPublisher(t *testing.T) { + RegisterFailHandler(Fail) + RunSpecs(t, "StateDiff Publisher Suite") +} \ No newline at end of file diff --git a/statediff/publisher/publisher_test.go b/statediff/publisher/publisher_test.go new file mode 100644 index 000000000..520163ddc --- /dev/null +++ b/statediff/publisher/publisher_test.go @@ -0,0 +1,20 @@ +// Copyright 2015 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +// Contains a batch of utility type declarations used by the tests. As the node +// operates on unique types, a lot of them are needed to check various features. + +package publisher_test \ No newline at end of file diff --git a/statediff/service.go b/statediff/service.go deleted file mode 100644 index 86839cf3b..000000000 --- a/statediff/service.go +++ /dev/null @@ -1,70 +0,0 @@ -package statediff - -import ( - "github.com/ethereum/go-ethereum/core" - "github.com/ethereum/go-ethereum/ethdb" - "github.com/ethereum/go-ethereum/p2p" - "github.com/ethereum/go-ethereum/rpc" - "github.com/ethereum/go-ethereum/event" - "log" -) - -type StateDiffService struct { - builder *builder - extractor *extractor - blockchain *core.BlockChain -} - -func NewStateDiffService(db ethdb.Database, blockChain *core.BlockChain) (*StateDiffService, error) { - config := Config{} - builder := NewBuilder(db) - publisher, err := NewPublisher(config) - if err != nil { - return nil, nil - } - - extractor, _ := NewExtractor(builder, publisher) - return &StateDiffService{ - blockchain: blockChain, - extractor: extractor, - }, nil -} - -func (StateDiffService) Protocols() []p2p.Protocol { - return []p2p.Protocol{} - -} - -func (StateDiffService) APIs() []rpc.API { - return []rpc.API{} -} - -func (sds *StateDiffService) loop (sub event.Subscription, events chan core.ChainHeadEvent) { - defer sub.Unsubscribe() - - for { - select { - case ev, ok := <-events: - if !ok { - log.Fatalf("Error getting chain head event from subscription.") - } - log.Println("doing something with an event", ev) - previousBlock := ev.Block - //TODO: figure out the best way to get the previous block - currentBlock := ev.Block - sds.extractor.ExtractStateDiff(*previousBlock, *currentBlock) - } - } - -} -func (sds *StateDiffService) Start(server *p2p.Server) error { - events := make(chan core.ChainHeadEvent, 10) - sub := sds.blockchain.SubscribeChainHeadEvent(events) - - go sds.loop(sub, events) - - return nil -} -func (StateDiffService) Stop() error { - return nil -} diff --git a/statediff/service/service.go b/statediff/service/service.go new file mode 100644 index 000000000..b3da7532d --- /dev/null +++ b/statediff/service/service.go @@ -0,0 +1,94 @@ +// Copyright 2015 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +// Contains a batch of utility type declarations used by the tests. As the node +// operates on unique types, a lot of them are needed to check various features. + +package service + +import ( + "log" + + "github.com/ethereum/go-ethereum/core" + "github.com/ethereum/go-ethereum/ethdb" + "github.com/ethereum/go-ethereum/p2p" + "github.com/ethereum/go-ethereum/rpc" + "github.com/ethereum/go-ethereum/event" + "github.com/ethereum/go-ethereum/statediff" + "github.com/ethereum/go-ethereum/statediff/builder" + "github.com/ethereum/go-ethereum/statediff/extractor" + "github.com/ethereum/go-ethereum/statediff/publisher" +) + +type StateDiffService struct { + builder builder.Builder + extractor extractor.Extractor + blockchain *core.BlockChain +} + +func NewStateDiffService(db ethdb.Database, blockChain *core.BlockChain) (*StateDiffService, error) { + config := statediff.Config{} + b := builder.NewBuilder(db) + p, err := publisher.NewPublisher(config) + if err != nil { + return nil, nil + } + + e, _ := extractor.NewExtractor(b, p) + return &StateDiffService{ + blockchain: blockChain, + extractor: e, + }, nil +} + +func (StateDiffService) Protocols() []p2p.Protocol { + return []p2p.Protocol{} + +} + +func (StateDiffService) APIs() []rpc.API { + return []rpc.API{} +} + +func (sds *StateDiffService) loop (sub event.Subscription, events chan core.ChainHeadEvent) { + defer sub.Unsubscribe() + + for { + select { + case ev, ok := <-events: + if !ok { + log.Fatalf("Error getting chain head event from subscription.") + } + log.Println("doing something with an event", ev) + previousBlock := ev.Block + //TODO: figure out the best way to get the previous block + currentBlock := ev.Block + sds.extractor.ExtractStateDiff(*previousBlock, *currentBlock) + } + } + +} +func (sds *StateDiffService) Start(server *p2p.Server) error { + events := make(chan core.ChainHeadEvent, 10) + sub := sds.blockchain.SubscribeChainHeadEvent(events) + + go sds.loop(sub, events) + + return nil +} +func (StateDiffService) Stop() error { + return nil +} diff --git a/statediff/service/service_suite_test.go b/statediff/service/service_suite_test.go new file mode 100644 index 000000000..60212a8fd --- /dev/null +++ b/statediff/service/service_suite_test.go @@ -0,0 +1,32 @@ +// Copyright 2015 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +// Contains a batch of utility type declarations used by the tests. As the node +// operates on unique types, a lot of them are needed to check various features. + +package service_test + +import ( + "testing" + + . "github.com/onsi/ginkgo" + . "github.com/onsi/gomega" +) + +func TestStateDiffService(t *testing.T) { + RegisterFailHandler(Fail) + RunSpecs(t, "StateDiff Service Suite") +} \ No newline at end of file diff --git a/statediff/service/service_test.go b/statediff/service/service_test.go new file mode 100644 index 000000000..99e6cfd63 --- /dev/null +++ b/statediff/service/service_test.go @@ -0,0 +1,20 @@ +// Copyright 2015 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +// Contains a batch of utility type declarations used by the tests. As the node +// operates on unique types, a lot of them are needed to check various features. + +package service_test \ No newline at end of file diff --git a/statediff/publisher_test.go b/statediff/statediff_integration_test.go similarity index 100% rename from statediff/publisher_test.go rename to statediff/statediff_integration_test.go diff --git a/statediff/statediff_suite_test.go b/statediff/statediff_suite_test.go index 7118ea792..99834db48 100644 --- a/statediff/statediff_suite_test.go +++ b/statediff/statediff_suite_test.go @@ -1,3 +1,22 @@ +// Copyright 2015 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +// Contains a batch of utility type declarations used by the tests. As the node +// operates on unique types, a lot of them are needed to check various features. + package statediff_test import ( @@ -7,7 +26,7 @@ import ( . "github.com/onsi/gomega" ) -func TestStatediff(t *testing.T) { +func TestStateDiff(t *testing.T) { RegisterFailHandler(Fail) - RunSpecs(t, "Statediff Suite") + RunSpecs(t, "StateDiff Suite") } diff --git a/statediff/testhelpers/mocks.go b/statediff/testhelpers/mocks.go index 4bff3c028..b4608e455 100644 --- a/statediff/testhelpers/mocks.go +++ b/statediff/testhelpers/mocks.go @@ -1,9 +1,28 @@ +// Copyright 2015 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +// Contains a batch of utility type declarations used by the tests. As the node +// operates on unique types, a lot of them are needed to check various features. + package testhelpers import ( - "github.com/ethereum/go-ethereum/common" - "github.com/ethereum/go-ethereum/statediff" "errors" + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/statediff/builder" ) var MockError = errors.New("mock error") @@ -13,11 +32,11 @@ type MockBuilder struct { NewStateRoot common.Hash BlockNumber int64 BlockHash common.Hash - stateDiff *statediff.StateDiff + stateDiff *builder.StateDiff builderError error } -func (builder *MockBuilder) BuildStateDiff(oldStateRoot, newStateRoot common.Hash, blockNumber int64, blockHash common.Hash) (*statediff.StateDiff, error) { +func (builder *MockBuilder) BuildStateDiff(oldStateRoot, newStateRoot common.Hash, blockNumber int64, blockHash common.Hash) (*builder.StateDiff, error) { builder.OldStateRoot = oldStateRoot builder.NewStateRoot = newStateRoot builder.BlockNumber = blockNumber @@ -26,7 +45,7 @@ func (builder *MockBuilder) BuildStateDiff(oldStateRoot, newStateRoot common.Has return builder.stateDiff, builder.builderError } -func (builder *MockBuilder) SetStateDiffToBuild(stateDiff *statediff.StateDiff) { +func (builder *MockBuilder) SetStateDiffToBuild(stateDiff *builder.StateDiff) { builder.stateDiff = stateDiff } @@ -35,11 +54,11 @@ func (builder *MockBuilder) SetBuilderError(err error) { } type MockPublisher struct{ - StateDiff *statediff.StateDiff + StateDiff *builder.StateDiff publisherError error } -func (publisher *MockPublisher) PublishStateDiff(sd *statediff.StateDiff) (string, error) { +func (publisher *MockPublisher) PublishStateDiff(sd *builder.StateDiff) (string, error) { publisher.StateDiff = sd return "", publisher.publisherError } -- 2.45.2