cmd, consensus, eth: split ethash related config to it own (#15520)

* cmd, consensus, eth: split ethash related config to it own

* eth, consensus: minor polish

* eth, consenus, console: compress pow testing config field to single one

* consensus, eth: document pow mode
pull/15514/head
gary rong 7 years ago committed by Péter Szilágyi
parent b0056f5bd0
commit f14047dae5
  1. 34
      cmd/utils/flags.go
  2. 3
      consensus/ethash/algorithm_test.go
  3. 10
      consensus/ethash/consensus.go
  4. 117
      consensus/ethash/ethash.go
  5. 2
      consensus/ethash/sealer.go
  6. 5
      console/console_test.go
  7. 20
      eth/backend.go
  8. 39
      eth/config.go
  9. 51
      eth/gen_config.go
  10. 2
      les/backend.go

@ -217,27 +217,27 @@ var (
EthashCachesInMemoryFlag = cli.IntFlag{ EthashCachesInMemoryFlag = cli.IntFlag{
Name: "ethash.cachesinmem", Name: "ethash.cachesinmem",
Usage: "Number of recent ethash caches to keep in memory (16MB each)", Usage: "Number of recent ethash caches to keep in memory (16MB each)",
Value: eth.DefaultConfig.EthashCachesInMem, Value: eth.DefaultConfig.Ethash.CachesInMem,
} }
EthashCachesOnDiskFlag = cli.IntFlag{ EthashCachesOnDiskFlag = cli.IntFlag{
Name: "ethash.cachesondisk", Name: "ethash.cachesondisk",
Usage: "Number of recent ethash caches to keep on disk (16MB each)", Usage: "Number of recent ethash caches to keep on disk (16MB each)",
Value: eth.DefaultConfig.EthashCachesOnDisk, Value: eth.DefaultConfig.Ethash.CachesOnDisk,
} }
EthashDatasetDirFlag = DirectoryFlag{ EthashDatasetDirFlag = DirectoryFlag{
Name: "ethash.dagdir", Name: "ethash.dagdir",
Usage: "Directory to store the ethash mining DAGs (default = inside home folder)", Usage: "Directory to store the ethash mining DAGs (default = inside home folder)",
Value: DirectoryString{eth.DefaultConfig.EthashDatasetDir}, Value: DirectoryString{eth.DefaultConfig.Ethash.DatasetDir},
} }
EthashDatasetsInMemoryFlag = cli.IntFlag{ EthashDatasetsInMemoryFlag = cli.IntFlag{
Name: "ethash.dagsinmem", Name: "ethash.dagsinmem",
Usage: "Number of recent ethash mining DAGs to keep in memory (1+GB each)", Usage: "Number of recent ethash mining DAGs to keep in memory (1+GB each)",
Value: eth.DefaultConfig.EthashDatasetsInMem, Value: eth.DefaultConfig.Ethash.DatasetsInMem,
} }
EthashDatasetsOnDiskFlag = cli.IntFlag{ EthashDatasetsOnDiskFlag = cli.IntFlag{
Name: "ethash.dagsondisk", Name: "ethash.dagsondisk",
Usage: "Number of recent ethash mining DAGs to keep on disk (1+GB each)", Usage: "Number of recent ethash mining DAGs to keep on disk (1+GB each)",
Value: eth.DefaultConfig.EthashDatasetsOnDisk, Value: eth.DefaultConfig.Ethash.DatasetsOnDisk,
} }
// Transaction pool settings // Transaction pool settings
TxPoolNoLocalsFlag = cli.BoolFlag{ TxPoolNoLocalsFlag = cli.BoolFlag{
@ -910,22 +910,22 @@ func setTxPool(ctx *cli.Context, cfg *core.TxPoolConfig) {
func setEthash(ctx *cli.Context, cfg *eth.Config) { func setEthash(ctx *cli.Context, cfg *eth.Config) {
if ctx.GlobalIsSet(EthashCacheDirFlag.Name) { if ctx.GlobalIsSet(EthashCacheDirFlag.Name) {
cfg.EthashCacheDir = ctx.GlobalString(EthashCacheDirFlag.Name) cfg.Ethash.CacheDir = ctx.GlobalString(EthashCacheDirFlag.Name)
} }
if ctx.GlobalIsSet(EthashDatasetDirFlag.Name) { if ctx.GlobalIsSet(EthashDatasetDirFlag.Name) {
cfg.EthashDatasetDir = ctx.GlobalString(EthashDatasetDirFlag.Name) cfg.Ethash.DatasetDir = ctx.GlobalString(EthashDatasetDirFlag.Name)
} }
if ctx.GlobalIsSet(EthashCachesInMemoryFlag.Name) { if ctx.GlobalIsSet(EthashCachesInMemoryFlag.Name) {
cfg.EthashCachesInMem = ctx.GlobalInt(EthashCachesInMemoryFlag.Name) cfg.Ethash.CachesInMem = ctx.GlobalInt(EthashCachesInMemoryFlag.Name)
} }
if ctx.GlobalIsSet(EthashCachesOnDiskFlag.Name) { if ctx.GlobalIsSet(EthashCachesOnDiskFlag.Name) {
cfg.EthashCachesOnDisk = ctx.GlobalInt(EthashCachesOnDiskFlag.Name) cfg.Ethash.CachesOnDisk = ctx.GlobalInt(EthashCachesOnDiskFlag.Name)
} }
if ctx.GlobalIsSet(EthashDatasetsInMemoryFlag.Name) { if ctx.GlobalIsSet(EthashDatasetsInMemoryFlag.Name) {
cfg.EthashDatasetsInMem = ctx.GlobalInt(EthashDatasetsInMemoryFlag.Name) cfg.Ethash.DatasetsInMem = ctx.GlobalInt(EthashDatasetsInMemoryFlag.Name)
} }
if ctx.GlobalIsSet(EthashDatasetsOnDiskFlag.Name) { if ctx.GlobalIsSet(EthashDatasetsOnDiskFlag.Name) {
cfg.EthashDatasetsOnDisk = ctx.GlobalInt(EthashDatasetsOnDiskFlag.Name) cfg.Ethash.DatasetsOnDisk = ctx.GlobalInt(EthashDatasetsOnDiskFlag.Name)
} }
} }
@ -1159,10 +1159,14 @@ func MakeChain(ctx *cli.Context, stack *node.Node) (chain *core.BlockChain, chai
} else { } else {
engine = ethash.NewFaker() engine = ethash.NewFaker()
if !ctx.GlobalBool(FakePoWFlag.Name) { if !ctx.GlobalBool(FakePoWFlag.Name) {
engine = ethash.New( engine = ethash.New(ethash.Config{
stack.ResolvePath(eth.DefaultConfig.EthashCacheDir), eth.DefaultConfig.EthashCachesInMem, eth.DefaultConfig.EthashCachesOnDisk, CacheDir: stack.ResolvePath(eth.DefaultConfig.Ethash.CacheDir),
stack.ResolvePath(eth.DefaultConfig.EthashDatasetDir), eth.DefaultConfig.EthashDatasetsInMem, eth.DefaultConfig.EthashDatasetsOnDisk, CachesInMem: eth.DefaultConfig.Ethash.CachesInMem,
) CachesOnDisk: eth.DefaultConfig.Ethash.CachesOnDisk,
DatasetDir: stack.ResolvePath(eth.DefaultConfig.Ethash.DatasetDir),
DatasetsInMem: eth.DefaultConfig.Ethash.DatasetsInMem,
DatasetsOnDisk: eth.DefaultConfig.Ethash.DatasetsOnDisk,
})
} }
} }
vmcfg := vm.Config{EnablePreimageRecording: ctx.GlobalBool(VMEnableDebugFlag.Name)} vmcfg := vm.Config{EnablePreimageRecording: ctx.GlobalBool(VMEnableDebugFlag.Name)}

@ -703,8 +703,7 @@ func TestConcurrentDiskCacheGeneration(t *testing.T) {
go func(idx int) { go func(idx int) {
defer pend.Done() defer pend.Done()
ethash := New(Config{cachedir, 0, 1, "", 0, 0, ModeNormal})
ethash := New(cachedir, 0, 1, "", 0, 0)
if err := ethash.VerifySeal(nil, block.Header()); err != nil { if err := ethash.VerifySeal(nil, block.Header()); err != nil {
t.Errorf("proc %d: block verification failed: %v", idx, err) t.Errorf("proc %d: block verification failed: %v", idx, err)
} }

@ -68,7 +68,7 @@ func (ethash *Ethash) Author(header *types.Header) (common.Address, error) {
// stock Ethereum ethash engine. // stock Ethereum ethash engine.
func (ethash *Ethash) VerifyHeader(chain consensus.ChainReader, header *types.Header, seal bool) error { func (ethash *Ethash) VerifyHeader(chain consensus.ChainReader, header *types.Header, seal bool) error {
// If we're running a full engine faking, accept any input as valid // If we're running a full engine faking, accept any input as valid
if ethash.fakeFull { if ethash.config.PowMode == ModeFullFake {
return nil return nil
} }
// Short circuit if the header is known, or it's parent not // Short circuit if the header is known, or it's parent not
@ -89,7 +89,7 @@ func (ethash *Ethash) VerifyHeader(chain consensus.ChainReader, header *types.He
// a results channel to retrieve the async verifications. // a results channel to retrieve the async verifications.
func (ethash *Ethash) VerifyHeaders(chain consensus.ChainReader, headers []*types.Header, seals []bool) (chan<- struct{}, <-chan error) { func (ethash *Ethash) VerifyHeaders(chain consensus.ChainReader, headers []*types.Header, seals []bool) (chan<- struct{}, <-chan error) {
// If we're running a full engine faking, accept any input as valid // If we're running a full engine faking, accept any input as valid
if ethash.fakeFull || len(headers) == 0 { if ethash.config.PowMode == ModeFullFake || len(headers) == 0 {
abort, results := make(chan struct{}), make(chan error, len(headers)) abort, results := make(chan struct{}), make(chan error, len(headers))
for i := 0; i < len(headers); i++ { for i := 0; i < len(headers); i++ {
results <- nil results <- nil
@ -169,7 +169,7 @@ func (ethash *Ethash) verifyHeaderWorker(chain consensus.ChainReader, headers []
// rules of the stock Ethereum ethash engine. // rules of the stock Ethereum ethash engine.
func (ethash *Ethash) VerifyUncles(chain consensus.ChainReader, block *types.Block) error { func (ethash *Ethash) VerifyUncles(chain consensus.ChainReader, block *types.Block) error {
// If we're running a full engine faking, accept any input as valid // If we're running a full engine faking, accept any input as valid
if ethash.fakeFull { if ethash.config.PowMode == ModeFullFake {
return nil return nil
} }
// Verify that there are at most 2 uncles included in this block // Verify that there are at most 2 uncles included in this block
@ -455,7 +455,7 @@ func calcDifficultyFrontier(time uint64, parent *types.Header) *big.Int {
// the PoW difficulty requirements. // the PoW difficulty requirements.
func (ethash *Ethash) VerifySeal(chain consensus.ChainReader, header *types.Header) error { func (ethash *Ethash) VerifySeal(chain consensus.ChainReader, header *types.Header) error {
// If we're running a fake PoW, accept any seal as valid // If we're running a fake PoW, accept any seal as valid
if ethash.fakeMode { if ethash.config.PowMode == ModeFake || ethash.config.PowMode == ModeFullFake {
time.Sleep(ethash.fakeDelay) time.Sleep(ethash.fakeDelay)
if ethash.fakeFail == header.Number.Uint64() { if ethash.fakeFail == header.Number.Uint64() {
return errInvalidPoW return errInvalidPoW
@ -480,7 +480,7 @@ func (ethash *Ethash) VerifySeal(chain consensus.ChainReader, header *types.Head
cache := ethash.cache(number) cache := ethash.cache(number)
size := datasetSize(number) size := datasetSize(number)
if ethash.tester { if ethash.config.PowMode == ModeTest {
size = 32 * 1024 size = 32 * 1024
} }
digest, result := hashimotoLight(size, cache, header.HashNoNonce().Bytes(), header.Nonce.Uint64()) digest, result := hashimotoLight(size, cache, header.HashNoNonce().Bytes(), header.Nonce.Uint64())

@ -45,7 +45,7 @@ var (
maxUint256 = new(big.Int).Exp(big.NewInt(2), big.NewInt(256), big.NewInt(0)) maxUint256 = new(big.Int).Exp(big.NewInt(2), big.NewInt(256), big.NewInt(0))
// sharedEthash is a full instance that can be shared between multiple users. // sharedEthash is a full instance that can be shared between multiple users.
sharedEthash = New("", 3, 0, "", 1, 0) sharedEthash = New(Config{"", 3, 0, "", 1, 0, ModeNormal})
// algorithmRevision is the data structure version used for file naming. // algorithmRevision is the data structure version used for file naming.
algorithmRevision = 23 algorithmRevision = 23
@ -320,15 +320,32 @@ func MakeDataset(block uint64, dir string) {
d.release() d.release()
} }
// Mode defines the type and amount of PoW verification an ethash engine makes.
type Mode uint
const (
ModeNormal Mode = iota
ModeShared
ModeTest
ModeFake
ModeFullFake
)
// Config are the configuration parameters of the ethash.
type Config struct {
CacheDir string
CachesInMem int
CachesOnDisk int
DatasetDir string
DatasetsInMem int
DatasetsOnDisk int
PowMode Mode
}
// Ethash is a consensus engine based on proot-of-work implementing the ethash // Ethash is a consensus engine based on proot-of-work implementing the ethash
// algorithm. // algorithm.
type Ethash struct { type Ethash struct {
cachedir string // Data directory to store the verification caches config Config
cachesinmem int // Number of caches to keep in memory
cachesondisk int // Number of caches to keep on disk
dagdir string // Data directory to store full mining datasets
dagsinmem int // Number of mining datasets to keep in memory
dagsondisk int // Number of mining datasets to keep on disk
caches map[uint64]*cache // In memory caches to avoid regenerating too often caches map[uint64]*cache // In memory caches to avoid regenerating too often
fcache *cache // Pre-generated cache for the estimated future epoch fcache *cache // Pre-generated cache for the estimated future epoch
@ -342,10 +359,7 @@ type Ethash struct {
hashrate metrics.Meter // Meter tracking the average hashrate hashrate metrics.Meter // Meter tracking the average hashrate
// The fields below are hooks for testing // The fields below are hooks for testing
tester bool // Flag whether to use a smaller test dataset
shared *Ethash // Shared PoW verifier to avoid cache regeneration shared *Ethash // Shared PoW verifier to avoid cache regeneration
fakeMode bool // Flag whether to disable PoW checking
fakeFull bool // Flag whether to disable all consensus rules
fakeFail uint64 // Block number which fails PoW check even in fake mode fakeFail uint64 // Block number which fails PoW check even in fake mode
fakeDelay time.Duration // Time delay to sleep for before returning from verify fakeDelay time.Duration // Time delay to sleep for before returning from verify
@ -353,28 +367,23 @@ type Ethash struct {
} }
// New creates a full sized ethash PoW scheme. // New creates a full sized ethash PoW scheme.
func New(cachedir string, cachesinmem, cachesondisk int, dagdir string, dagsinmem, dagsondisk int) *Ethash { func New(config Config) *Ethash {
if cachesinmem <= 0 { if config.CachesInMem <= 0 {
log.Warn("One ethash cache must always be in memory", "requested", cachesinmem) log.Warn("One ethash cache must always be in memory", "requested", config.CachesInMem)
cachesinmem = 1 config.CachesInMem = 1
} }
if cachedir != "" && cachesondisk > 0 { if config.CacheDir != "" && config.CachesOnDisk > 0 {
log.Info("Disk storage enabled for ethash caches", "dir", cachedir, "count", cachesondisk) log.Info("Disk storage enabled for ethash caches", "dir", config.CacheDir, "count", config.CachesOnDisk)
} }
if dagdir != "" && dagsondisk > 0 { if config.DatasetDir != "" && config.DatasetsOnDisk > 0 {
log.Info("Disk storage enabled for ethash DAGs", "dir", dagdir, "count", dagsondisk) log.Info("Disk storage enabled for ethash DAGs", "dir", config.DatasetDir, "count", config.DatasetsOnDisk)
} }
return &Ethash{ return &Ethash{
cachedir: cachedir, config: config,
cachesinmem: cachesinmem, caches: make(map[uint64]*cache),
cachesondisk: cachesondisk, datasets: make(map[uint64]*dataset),
dagdir: dagdir, update: make(chan struct{}),
dagsinmem: dagsinmem, hashrate: metrics.NewMeter(),
dagsondisk: dagsondisk,
caches: make(map[uint64]*cache),
datasets: make(map[uint64]*dataset),
update: make(chan struct{}),
hashrate: metrics.NewMeter(),
} }
} }
@ -382,12 +391,14 @@ func New(cachedir string, cachesinmem, cachesondisk int, dagdir string, dagsinme
// purposes. // purposes.
func NewTester() *Ethash { func NewTester() *Ethash {
return &Ethash{ return &Ethash{
cachesinmem: 1, config: Config{
caches: make(map[uint64]*cache), CachesInMem: 1,
datasets: make(map[uint64]*dataset), PowMode: ModeTest,
tester: true, },
update: make(chan struct{}), caches: make(map[uint64]*cache),
hashrate: metrics.NewMeter(), datasets: make(map[uint64]*dataset),
update: make(chan struct{}),
hashrate: metrics.NewMeter(),
} }
} }
@ -395,27 +406,45 @@ func NewTester() *Ethash {
// all blocks' seal as valid, though they still have to conform to the Ethereum // all blocks' seal as valid, though they still have to conform to the Ethereum
// consensus rules. // consensus rules.
func NewFaker() *Ethash { func NewFaker() *Ethash {
return &Ethash{fakeMode: true} return &Ethash{
config: Config{
PowMode: ModeFake,
},
}
} }
// NewFakeFailer creates a ethash consensus engine with a fake PoW scheme that // NewFakeFailer creates a ethash consensus engine with a fake PoW scheme that
// accepts all blocks as valid apart from the single one specified, though they // accepts all blocks as valid apart from the single one specified, though they
// still have to conform to the Ethereum consensus rules. // still have to conform to the Ethereum consensus rules.
func NewFakeFailer(fail uint64) *Ethash { func NewFakeFailer(fail uint64) *Ethash {
return &Ethash{fakeMode: true, fakeFail: fail} return &Ethash{
config: Config{
PowMode: ModeFake,
},
fakeFail: fail,
}
} }
// NewFakeDelayer creates a ethash consensus engine with a fake PoW scheme that // NewFakeDelayer creates a ethash consensus engine with a fake PoW scheme that
// accepts all blocks as valid, but delays verifications by some time, though // accepts all blocks as valid, but delays verifications by some time, though
// they still have to conform to the Ethereum consensus rules. // they still have to conform to the Ethereum consensus rules.
func NewFakeDelayer(delay time.Duration) *Ethash { func NewFakeDelayer(delay time.Duration) *Ethash {
return &Ethash{fakeMode: true, fakeDelay: delay} return &Ethash{
config: Config{
PowMode: ModeFake,
},
fakeDelay: delay,
}
} }
// NewFullFaker creates an ethash consensus engine with a full fake scheme that // NewFullFaker creates an ethash consensus engine with a full fake scheme that
// accepts all blocks as valid, without checking any consensus rules whatsoever. // accepts all blocks as valid, without checking any consensus rules whatsoever.
func NewFullFaker() *Ethash { func NewFullFaker() *Ethash {
return &Ethash{fakeMode: true, fakeFull: true} return &Ethash{
config: Config{
PowMode: ModeFullFake,
},
}
} }
// NewShared creates a full sized ethash PoW shared between all requesters running // NewShared creates a full sized ethash PoW shared between all requesters running
@ -436,7 +465,7 @@ func (ethash *Ethash) cache(block uint64) []uint32 {
current, future := ethash.caches[epoch], (*cache)(nil) current, future := ethash.caches[epoch], (*cache)(nil)
if current == nil { if current == nil {
// No in-memory cache, evict the oldest if the cache limit was reached // No in-memory cache, evict the oldest if the cache limit was reached
for len(ethash.caches) > 0 && len(ethash.caches) >= ethash.cachesinmem { for len(ethash.caches) > 0 && len(ethash.caches) >= ethash.config.CachesInMem {
var evict *cache var evict *cache
for _, cache := range ethash.caches { for _, cache := range ethash.caches {
if evict == nil || evict.used.After(cache.used) { if evict == nil || evict.used.After(cache.used) {
@ -473,7 +502,7 @@ func (ethash *Ethash) cache(block uint64) []uint32 {
ethash.lock.Unlock() ethash.lock.Unlock()
// Wait for generation finish, bump the timestamp and finalize the cache // Wait for generation finish, bump the timestamp and finalize the cache
current.generate(ethash.cachedir, ethash.cachesondisk, ethash.tester) current.generate(ethash.config.CacheDir, ethash.config.CachesOnDisk, ethash.config.PowMode == ModeTest)
current.lock.Lock() current.lock.Lock()
current.used = time.Now() current.used = time.Now()
@ -481,7 +510,7 @@ func (ethash *Ethash) cache(block uint64) []uint32 {
// If we exhausted the future cache, now's a good time to regenerate it // If we exhausted the future cache, now's a good time to regenerate it
if future != nil { if future != nil {
go future.generate(ethash.cachedir, ethash.cachesondisk, ethash.tester) go future.generate(ethash.config.CacheDir, ethash.config.CachesOnDisk, ethash.config.PowMode == ModeTest)
} }
return current.cache return current.cache
} }
@ -498,7 +527,7 @@ func (ethash *Ethash) dataset(block uint64) []uint32 {
current, future := ethash.datasets[epoch], (*dataset)(nil) current, future := ethash.datasets[epoch], (*dataset)(nil)
if current == nil { if current == nil {
// No in-memory dataset, evict the oldest if the dataset limit was reached // No in-memory dataset, evict the oldest if the dataset limit was reached
for len(ethash.datasets) > 0 && len(ethash.datasets) >= ethash.dagsinmem { for len(ethash.datasets) > 0 && len(ethash.datasets) >= ethash.config.DatasetsInMem {
var evict *dataset var evict *dataset
for _, dataset := range ethash.datasets { for _, dataset := range ethash.datasets {
if evict == nil || evict.used.After(dataset.used) { if evict == nil || evict.used.After(dataset.used) {
@ -536,7 +565,7 @@ func (ethash *Ethash) dataset(block uint64) []uint32 {
ethash.lock.Unlock() ethash.lock.Unlock()
// Wait for generation finish, bump the timestamp and finalize the cache // Wait for generation finish, bump the timestamp and finalize the cache
current.generate(ethash.dagdir, ethash.dagsondisk, ethash.tester) current.generate(ethash.config.DatasetDir, ethash.config.DatasetsOnDisk, ethash.config.PowMode == ModeTest)
current.lock.Lock() current.lock.Lock()
current.used = time.Now() current.used = time.Now()
@ -544,7 +573,7 @@ func (ethash *Ethash) dataset(block uint64) []uint32 {
// If we exhausted the future dataset, now's a good time to regenerate it // If we exhausted the future dataset, now's a good time to regenerate it
if future != nil { if future != nil {
go future.generate(ethash.dagdir, ethash.dagsondisk, ethash.tester) go future.generate(ethash.config.DatasetDir, ethash.config.DatasetsOnDisk, ethash.config.PowMode == ModeTest)
} }
return current.dataset return current.dataset
} }

@ -34,7 +34,7 @@ import (
// the block's difficulty requirements. // the block's difficulty requirements.
func (ethash *Ethash) Seal(chain consensus.ChainReader, block *types.Block, stop <-chan struct{}) (*types.Block, error) { func (ethash *Ethash) Seal(chain consensus.ChainReader, block *types.Block, stop <-chan struct{}) (*types.Block, error) {
// If we're running a fake PoW, simply return a 0 nonce immediately // If we're running a fake PoW, simply return a 0 nonce immediately
if ethash.fakeMode { if ethash.config.PowMode == ModeFake || ethash.config.PowMode == ModeFullFake {
header := block.Header() header := block.Header()
header.Nonce, header.MixDigest = types.BlockNonce{}, common.Hash{} header.Nonce, header.MixDigest = types.BlockNonce{}, common.Hash{}
return block.WithSeal(header), nil return block.WithSeal(header), nil

@ -27,6 +27,7 @@ import (
"time" "time"
"github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/consensus/ethash"
"github.com/ethereum/go-ethereum/core" "github.com/ethereum/go-ethereum/core"
"github.com/ethereum/go-ethereum/eth" "github.com/ethereum/go-ethereum/eth"
"github.com/ethereum/go-ethereum/internal/jsre" "github.com/ethereum/go-ethereum/internal/jsre"
@ -96,7 +97,9 @@ func newTester(t *testing.T, confOverride func(*eth.Config)) *tester {
ethConf := &eth.Config{ ethConf := &eth.Config{
Genesis: core.DeveloperGenesisBlock(15, common.Address{}), Genesis: core.DeveloperGenesisBlock(15, common.Address{}),
Etherbase: common.HexToAddress(testAddress), Etherbase: common.HexToAddress(testAddress),
PowTest: true, Ethash: ethash.Config{
PowMode: ethash.ModeTest,
},
} }
if confOverride != nil { if confOverride != nil {
confOverride(ethConf) confOverride(ethConf)

@ -125,7 +125,7 @@ func New(ctx *node.ServiceContext, config *Config) (*Ethereum, error) {
chainConfig: chainConfig, chainConfig: chainConfig,
eventMux: ctx.EventMux, eventMux: ctx.EventMux,
accountManager: ctx.AccountManager, accountManager: ctx.AccountManager,
engine: CreateConsensusEngine(ctx, config, chainConfig, chainDb), engine: CreateConsensusEngine(ctx, &config.Ethash, chainConfig, chainDb),
shutdownChan: make(chan bool), shutdownChan: make(chan bool),
stopDbUpgrade: stopDbUpgrade, stopDbUpgrade: stopDbUpgrade,
networkId: config.NetworkId, networkId: config.NetworkId,
@ -209,25 +209,31 @@ func CreateDB(ctx *node.ServiceContext, config *Config, name string) (ethdb.Data
} }
// CreateConsensusEngine creates the required type of consensus engine instance for an Ethereum service // CreateConsensusEngine creates the required type of consensus engine instance for an Ethereum service
func CreateConsensusEngine(ctx *node.ServiceContext, config *Config, chainConfig *params.ChainConfig, db ethdb.Database) consensus.Engine { func CreateConsensusEngine(ctx *node.ServiceContext, config *ethash.Config, chainConfig *params.ChainConfig, db ethdb.Database) consensus.Engine {
// If proof-of-authority is requested, set it up // If proof-of-authority is requested, set it up
if chainConfig.Clique != nil { if chainConfig.Clique != nil {
return clique.New(chainConfig.Clique, db) return clique.New(chainConfig.Clique, db)
} }
// Otherwise assume proof-of-work // Otherwise assume proof-of-work
switch { switch {
case config.PowFake: case config.PowMode == ethash.ModeFake:
log.Warn("Ethash used in fake mode") log.Warn("Ethash used in fake mode")
return ethash.NewFaker() return ethash.NewFaker()
case config.PowTest: case config.PowMode == ethash.ModeTest:
log.Warn("Ethash used in test mode") log.Warn("Ethash used in test mode")
return ethash.NewTester() return ethash.NewTester()
case config.PowShared: case config.PowMode == ethash.ModeShared:
log.Warn("Ethash used in shared mode") log.Warn("Ethash used in shared mode")
return ethash.NewShared() return ethash.NewShared()
default: default:
engine := ethash.New(ctx.ResolvePath(config.EthashCacheDir), config.EthashCachesInMem, config.EthashCachesOnDisk, engine := ethash.New(ethash.Config{
config.EthashDatasetDir, config.EthashDatasetsInMem, config.EthashDatasetsOnDisk) CacheDir: ctx.ResolvePath(config.CacheDir),
CachesInMem: config.CachesInMem,
CachesOnDisk: config.CachesOnDisk,
DatasetDir: config.DatasetDir,
DatasetsInMem: config.DatasetsInMem,
DatasetsOnDisk: config.DatasetsOnDisk,
})
engine.SetThreads(-1) // Disable CPU mining engine.SetThreads(-1) // Disable CPU mining
return engine return engine
} }

@ -25,6 +25,7 @@ import (
"github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/common/hexutil" "github.com/ethereum/go-ethereum/common/hexutil"
"github.com/ethereum/go-ethereum/consensus/ethash"
"github.com/ethereum/go-ethereum/core" "github.com/ethereum/go-ethereum/core"
"github.com/ethereum/go-ethereum/eth/downloader" "github.com/ethereum/go-ethereum/eth/downloader"
"github.com/ethereum/go-ethereum/eth/gasprice" "github.com/ethereum/go-ethereum/eth/gasprice"
@ -33,16 +34,18 @@ import (
// DefaultConfig contains default settings for use on the Ethereum main net. // DefaultConfig contains default settings for use on the Ethereum main net.
var DefaultConfig = Config{ var DefaultConfig = Config{
SyncMode: downloader.FastSync, SyncMode: downloader.FastSync,
EthashCacheDir: "ethash", Ethash: ethash.Config{
EthashCachesInMem: 2, CacheDir: "ethash",
EthashCachesOnDisk: 3, CachesInMem: 2,
EthashDatasetsInMem: 1, CachesOnDisk: 3,
EthashDatasetsOnDisk: 2, DatasetsInMem: 1,
NetworkId: 1, DatasetsOnDisk: 2,
LightPeers: 20, },
DatabaseCache: 128, NetworkId: 1,
GasPrice: big.NewInt(18 * params.Shannon), LightPeers: 20,
DatabaseCache: 128,
GasPrice: big.NewInt(18 * params.Shannon),
TxPool: core.DefaultTxPoolConfig, TxPool: core.DefaultTxPoolConfig,
GPO: gasprice.Config{ GPO: gasprice.Config{
@ -59,9 +62,9 @@ func init() {
} }
} }
if runtime.GOOS == "windows" { if runtime.GOOS == "windows" {
DefaultConfig.EthashDatasetDir = filepath.Join(home, "AppData", "Ethash") DefaultConfig.Ethash.DatasetDir = filepath.Join(home, "AppData", "Ethash")
} else { } else {
DefaultConfig.EthashDatasetDir = filepath.Join(home, ".ethash") DefaultConfig.Ethash.DatasetDir = filepath.Join(home, ".ethash")
} }
} }
@ -92,12 +95,7 @@ type Config struct {
GasPrice *big.Int GasPrice *big.Int
// Ethash options // Ethash options
EthashCacheDir string Ethash ethash.Config
EthashCachesInMem int
EthashCachesOnDisk int
EthashDatasetDir string
EthashDatasetsInMem int
EthashDatasetsOnDisk int
// Transaction pool options // Transaction pool options
TxPool core.TxPoolConfig TxPool core.TxPoolConfig
@ -109,10 +107,7 @@ type Config struct {
EnablePreimageRecording bool EnablePreimageRecording bool
// Miscellaneous options // Miscellaneous options
DocRoot string `toml:"-"` DocRoot string `toml:"-"`
PowFake bool `toml:"-"`
PowTest bool `toml:"-"`
PowShared bool `toml:"-"`
} }
type configMarshaling struct { type configMarshaling struct {

@ -7,6 +7,7 @@ import (
"github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/common/hexutil" "github.com/ethereum/go-ethereum/common/hexutil"
"github.com/ethereum/go-ethereum/consensus/ethash"
"github.com/ethereum/go-ethereum/core" "github.com/ethereum/go-ethereum/core"
"github.com/ethereum/go-ethereum/eth/downloader" "github.com/ethereum/go-ethereum/eth/downloader"
"github.com/ethereum/go-ethereum/eth/gasprice" "github.com/ethereum/go-ethereum/eth/gasprice"
@ -36,10 +37,8 @@ func (c Config) MarshalTOML() (interface{}, error) {
TxPool core.TxPoolConfig TxPool core.TxPoolConfig
GPO gasprice.Config GPO gasprice.Config
EnablePreimageRecording bool EnablePreimageRecording bool
DocRoot string `toml:"-"` DocRoot string `toml:"-"`
PowFake bool `toml:"-"` PowMode ethash.Mode `toml:"-"`
PowTest bool `toml:"-"`
PowShared bool `toml:"-"`
} }
var enc Config var enc Config
enc.Genesis = c.Genesis enc.Genesis = c.Genesis
@ -54,19 +53,17 @@ func (c Config) MarshalTOML() (interface{}, error) {
enc.MinerThreads = c.MinerThreads enc.MinerThreads = c.MinerThreads
enc.ExtraData = c.ExtraData enc.ExtraData = c.ExtraData
enc.GasPrice = c.GasPrice enc.GasPrice = c.GasPrice
enc.EthashCacheDir = c.EthashCacheDir enc.EthashCacheDir = c.Ethash.CacheDir
enc.EthashCachesInMem = c.EthashCachesInMem enc.EthashCachesInMem = c.Ethash.CachesInMem
enc.EthashCachesOnDisk = c.EthashCachesOnDisk enc.EthashCachesOnDisk = c.Ethash.CachesOnDisk
enc.EthashDatasetDir = c.EthashDatasetDir enc.EthashDatasetDir = c.Ethash.DatasetDir
enc.EthashDatasetsInMem = c.EthashDatasetsInMem enc.EthashDatasetsInMem = c.Ethash.DatasetsInMem
enc.EthashDatasetsOnDisk = c.EthashDatasetsOnDisk enc.EthashDatasetsOnDisk = c.Ethash.DatasetsOnDisk
enc.TxPool = c.TxPool enc.TxPool = c.TxPool
enc.GPO = c.GPO enc.GPO = c.GPO
enc.EnablePreimageRecording = c.EnablePreimageRecording enc.EnablePreimageRecording = c.EnablePreimageRecording
enc.DocRoot = c.DocRoot enc.DocRoot = c.DocRoot
enc.PowFake = c.PowFake enc.PowMode = c.Ethash.PowMode
enc.PowTest = c.PowTest
enc.PowShared = c.PowShared
return &enc, nil return &enc, nil
} }
@ -94,10 +91,8 @@ func (c *Config) UnmarshalTOML(unmarshal func(interface{}) error) error {
TxPool *core.TxPoolConfig TxPool *core.TxPoolConfig
GPO *gasprice.Config GPO *gasprice.Config
EnablePreimageRecording *bool EnablePreimageRecording *bool
DocRoot *string `toml:"-"` DocRoot *string `toml:"-"`
PowFake *bool `toml:"-"` PowMode *ethash.Mode `toml:"-"`
PowTest *bool `toml:"-"`
PowShared *bool `toml:"-"`
} }
var dec Config var dec Config
if err := unmarshal(&dec); err != nil { if err := unmarshal(&dec); err != nil {
@ -140,22 +135,22 @@ func (c *Config) UnmarshalTOML(unmarshal func(interface{}) error) error {
c.GasPrice = dec.GasPrice c.GasPrice = dec.GasPrice
} }
if dec.EthashCacheDir != nil { if dec.EthashCacheDir != nil {
c.EthashCacheDir = *dec.EthashCacheDir c.Ethash.CacheDir = *dec.EthashCacheDir
} }
if dec.EthashCachesInMem != nil { if dec.EthashCachesInMem != nil {
c.EthashCachesInMem = *dec.EthashCachesInMem c.Ethash.CachesInMem = *dec.EthashCachesInMem
} }
if dec.EthashCachesOnDisk != nil { if dec.EthashCachesOnDisk != nil {
c.EthashCachesOnDisk = *dec.EthashCachesOnDisk c.Ethash.CachesOnDisk = *dec.EthashCachesOnDisk
} }
if dec.EthashDatasetDir != nil { if dec.EthashDatasetDir != nil {
c.EthashDatasetDir = *dec.EthashDatasetDir c.Ethash.DatasetDir = *dec.EthashDatasetDir
} }
if dec.EthashDatasetsInMem != nil { if dec.EthashDatasetsInMem != nil {
c.EthashDatasetsInMem = *dec.EthashDatasetsInMem c.Ethash.DatasetsInMem = *dec.EthashDatasetsInMem
} }
if dec.EthashDatasetsOnDisk != nil { if dec.EthashDatasetsOnDisk != nil {
c.EthashDatasetsOnDisk = *dec.EthashDatasetsOnDisk c.Ethash.DatasetsOnDisk = *dec.EthashDatasetsOnDisk
} }
if dec.TxPool != nil { if dec.TxPool != nil {
c.TxPool = *dec.TxPool c.TxPool = *dec.TxPool
@ -169,14 +164,8 @@ func (c *Config) UnmarshalTOML(unmarshal func(interface{}) error) error {
if dec.DocRoot != nil { if dec.DocRoot != nil {
c.DocRoot = *dec.DocRoot c.DocRoot = *dec.DocRoot
} }
if dec.PowFake != nil { if dec.PowMode != nil {
c.PowFake = *dec.PowFake c.Ethash.PowMode = *dec.PowMode
}
if dec.PowTest != nil {
c.PowTest = *dec.PowTest
}
if dec.PowShared != nil {
c.PowShared = *dec.PowShared
} }
return nil return nil
} }

@ -98,7 +98,7 @@ func New(ctx *node.ServiceContext, config *eth.Config) (*LightEthereum, error) {
peers: peers, peers: peers,
reqDist: newRequestDistributor(peers, quitSync), reqDist: newRequestDistributor(peers, quitSync),
accountManager: ctx.AccountManager, accountManager: ctx.AccountManager,
engine: eth.CreateConsensusEngine(ctx, config, chainConfig, chainDb), engine: eth.CreateConsensusEngine(ctx, &config.Ethash, chainConfig, chainDb),
shutdownChan: make(chan bool), shutdownChan: make(chan bool),
networkId: config.NetworkId, networkId: config.NetworkId,
bloomRequests: make(chan chan *bloombits.Retrieval), bloomRequests: make(chan chan *bloombits.Retrieval),

Loading…
Cancel
Save