Make block proposal async and receiving last commit sig through channel

pull/3355/head
Rongjian Lan 4 years ago
parent 4089ea0695
commit 6845775078
  1. 2
      consensus/consensus.go
  2. 17
      consensus/consensus_v2.go
  3. 2
      consensus/engine/consensus_engine.go
  4. 5
      consensus/leader.go
  5. 2
      core/chain_makers.go
  6. 6
      core/state_processor.go
  7. 15
      crypto/bls/bls.go
  8. 6
      internal/chain/engine.go
  9. 182
      internal/chain/reward.go
  10. 12
      node/node_handler_test.go
  11. 35
      node/node_newblock.go
  12. 41
      node/worker/worker.go
  13. 86
      test/chain/bls/main.go
  14. 12
      test/chain/main.go

@ -97,6 +97,8 @@ type Consensus struct {
OnConsensusDone func(*types.Block)
// The verifier func passed from Node object
BlockVerifier func(*types.Block) error
// Block Proposer proposes new block at the specific block number
BlockProposer func(blockNum uint64) (*types.Block, error)
// verified block to state sync broadcast
VerifiedNewBlock chan *types.Block
// will trigger state syncing when blockNum is low

@ -191,9 +191,9 @@ func (consensus *Consensus) finalizeCommits() {
// BlockCommitSig returns the byte array of aggregated
// commit signature and bitmap signed on the block
func (consensus *Consensus) BlockCommitSig(blockNum uint64) ([]byte, []byte, error) {
func (consensus *Consensus) BlockCommitSigs(blockNum uint64) ([]byte, error) {
if consensus.blockNum <= 1 {
return nil, nil, nil
return nil, nil
}
lastCommits, err := consensus.ChainReader.ReadCommitSig(blockNum)
if err != nil ||
@ -205,21 +205,14 @@ func (consensus *Consensus) BlockCommitSig(blockNum uint64) ([]byte, []byte, err
consensus.getLogger().Error().
Int("numCommittedMsg", len(msgs)).
Msg("GetLastCommitSig failed with wrong number of committed message")
return nil, nil, errors.Errorf(
return nil, errors.Errorf(
"GetLastCommitSig failed with wrong number of committed message %d", len(msgs),
)
}
lastCommits = msgs[0].Payload
}
//#### Read payload data from committed msg
aggSig := make([]byte, bls.BLSSignatureSizeInBytes)
bitmap := make([]byte, len(lastCommits)-bls.BLSSignatureSizeInBytes)
offset := 0
copy(aggSig[:], lastCommits[offset:offset+bls.BLSSignatureSizeInBytes])
offset += bls.BLSSignatureSizeInBytes
copy(bitmap[:], lastCommits[offset:])
//#### END Read payload data from committed msg
return aggSig, bitmap, nil
return lastCommits, nil
}
// try to catch up if fall behind

@ -115,6 +115,6 @@ type Engine interface {
state *state.DB, txs []*types.Transaction,
receipts []*types.Receipt, outcxs []*types.CXReceipt,
incxs []*types.CXReceiptsProof, stks staking.StakingTransactions,
doubleSigners slash.Records,
doubleSigners slash.Records, sigsReady chan bool,
) (*types.Block, reward.Reader, error)
}

@ -205,6 +205,9 @@ func (consensus *Consensus) onPrepare(msg *msg_pb.Message) {
}
func (consensus *Consensus) onCommit(msg *msg_pb.Message) {
if consensus.viewID == 10 {
return
}
recvMsg, err := consensus.ParseFBFTMessage(msg)
if err != nil {
consensus.getLogger().Debug().Err(err).Msg("[OnCommit] Parse pbft message failed")
@ -308,7 +311,7 @@ func (consensus *Consensus) onCommit(msg *msg_pb.Message) {
consensus.getLogger().Info().Msg("[OnCommit] Starting Grace Period")
go func(viewID uint64) {
time.Sleep(2500 * time.Millisecond)
time.Sleep(1500 * time.Millisecond)
logger.Info().Msg("[OnCommit] Commit Grace Period Ended")
consensus.commitFinishChan <- viewID
}(viewID)

@ -196,7 +196,7 @@ func GenerateChain(
if b.engine != nil {
// Finalize and seal the block
block, _, err := b.engine.Finalize(
chainreader, b.header, statedb, b.txs, b.receipts, nil, nil, nil, nil,
chainreader, b.header, statedb, b.txs, b.receipts, nil, nil, nil, nil, nil,
)
if err != nil {
panic(err)

@ -135,9 +135,13 @@ func (p *StateProcessor) Process(
}
// Finalize the block, applying any consensus engine specific extras (e.g. block rewards)
sigsReady := make(chan bool)
go func() {
sigsReady <- true
}()
_, payout, err := p.engine.Finalize(
p.bc, header, statedb, block.Transactions(),
receipts, outcxs, incxs, block.StakingTransactions(), slashes,
receipts, outcxs, incxs, block.StakingTransactions(), slashes, sigsReady,
)
if err != nil {
return nil, nil, nil, 0, nil, errors.New("[Process] Cannot finalize block")

@ -95,3 +95,18 @@ func (pk *SerializedPublicKey) FromLibBLSPublicKey(key *bls.PublicKey) error {
copy(pk[:], bytes)
return nil
}
func SeparateSigAndMask(commitSigs []byte) ([]byte, []byte, error) {
if len(commitSigs) < BLSSignatureSizeInBytes {
return nil, nil, errors.New("no mask data found in commit sigs")
}
//#### Read payload data from committed msg
aggSig := make([]byte, BLSSignatureSizeInBytes)
bitmap := make([]byte, len(commitSigs)-BLSSignatureSizeInBytes)
offset := 0
copy(aggSig[:], commitSigs[offset:offset+BLSSignatureSizeInBytes])
offset += BLSSignatureSizeInBytes
copy(bitmap[:], commitSigs[offset:])
//#### END Read payload data from committed msg
return aggSig, bitmap, nil
}

@ -208,7 +208,7 @@ func (e *engineImpl) Finalize(
state *state.DB, txs []*types.Transaction,
receipts []*types.Receipt, outcxs []*types.CXReceipt,
incxs []*types.CXReceiptsProof, stks staking.StakingTransactions,
doubleSigners slash.Records,
doubleSigners slash.Records, sigsReady chan bool,
) (*types.Block, reward.Reader, error) {
isBeaconChain := header.ShardID() == shard.BeaconChainShardID
@ -249,10 +249,10 @@ func (e *engineImpl) Finalize(
// Accumulate block rewards and commit the final state root
// Header seems complete, assemble into a block and return
payout, err := AccumulateRewardsAndCountSigs(
chain, state, header, e.Beaconchain(),
chain, state, header, e.Beaconchain(), sigsReady,
)
if err != nil {
return nil, nil, errors.New("cannot pay block reward")
return nil, nil, err
}
// Apply slashes

@ -4,6 +4,7 @@ import (
"fmt"
"math/big"
"sort"
"time"
"github.com/harmony-one/harmony/internal/params"
@ -116,27 +117,25 @@ func lookupDelegatorShares(
// This func also do IncrementValidatorSigningCounts for validators
func AccumulateRewardsAndCountSigs(
bc engine.ChainReader, state *state.DB,
header *block.Header, beaconChain engine.ChainReader,
header *block.Header, beaconChain engine.ChainReader, sigsReady chan bool,
) (reward.Reader, error) {
blockNum := header.Number().Uint64()
currentHeader := beaconChain.CurrentHeader()
nowEpoch, blockNow := currentHeader.Epoch(), currentHeader.Number()
if blockNum == 0 {
// genesis block has no parent to reward.
return network.EmptyPayout, nil
}
if bc.Config().IsStaking(header.Epoch()) &&
bc.CurrentHeader().ShardID() != shard.BeaconChainShardID {
if blockNum == 0 || (bc.Config().IsStaking(header.Epoch()) &&
bc.CurrentHeader().ShardID() != shard.BeaconChainShardID) {
// Block here until the commit sigs are ready or timeout.
// sigsReady signal indicates that the commit sigs are already populated in the header object.
if err := waitForCommitSigs(sigsReady); err != nil {
return network.EmptyPayout, err
}
return network.EmptyPayout, nil
}
// After staking
if headerE := header.Epoch(); bc.Config().IsStaking(headerE) &&
bc.CurrentHeader().ShardID() == shard.BeaconChainShardID {
utils.AnalysisStart("accumulateRewardBeaconchainSelfPayout", nowEpoch, blockNow)
defaultReward := network.BaseStakedReward
// After block time is reduced to 5 seconds, the block reward is adjusted accordingly
@ -182,72 +181,9 @@ func AccumulateRewardsAndCountSigs(
newRewards, beaconP, shardP :=
big.NewInt(0), []reward.Payout{}, []reward.Payout{}
// Take care of my own beacon chain committee, _ is missing, for slashing
parentE, members, payable, missing, err := ballotResultBeaconchain(beaconChain, header)
if err != nil {
return network.EmptyPayout, err
}
subComm := shard.Committee{shard.BeaconChainShardID, members}
if err := availability.IncrementValidatorSigningCounts(
beaconChain,
subComm.StakedValidators(),
state,
payable,
missing,
); err != nil {
return network.EmptyPayout, err
}
votingPower, err := lookupVotingPower(
parentE, &subComm,
)
if err != nil {
return network.EmptyPayout, err
}
allSignersShare := numeric.ZeroDec()
for j := range payable {
voter := votingPower.Voters[payable[j].BLSPublicKey]
if !voter.IsHarmonyNode {
voterShare := voter.OverallPercent
allSignersShare = allSignersShare.Add(voterShare)
}
}
for beaconMember := range payable {
// TODO Give out whatever leftover to the last voter/handle
// what to do about share of those that didn't sign
blsKey := payable[beaconMember].BLSPublicKey
voter := votingPower.Voters[blsKey]
if !voter.IsHarmonyNode {
snapshot, err := bc.ReadValidatorSnapshot(voter.EarningAccount)
if err != nil {
return network.EmptyPayout, err
}
due := defaultReward.Mul(
voter.OverallPercent.Quo(allSignersShare),
).RoundInt()
newRewards.Add(newRewards, due)
shares, err := lookupDelegatorShares(snapshot)
if err != nil {
return network.EmptyPayout, err
}
if err := state.AddReward(snapshot.Validator, due, shares); err != nil {
return network.EmptyPayout, err
}
beaconP = append(beaconP, reward.Payout{
ShardID: shard.BeaconChainShardID,
Addr: voter.EarningAccount,
NewlyEarned: due,
EarningKey: voter.Identity,
})
}
}
utils.AnalysisEnd("accumulateRewardBeaconchainSelfPayout", nowEpoch, blockNow)
utils.AnalysisStart("accumulateRewardShardchainPayout", nowEpoch, blockNow)
// Handle rewards for shardchain
if cxLinks := header.CrossLinks(); len(cxLinks) > 0 {
utils.AnalysisStart("accumulateRewardShardchainPayout", nowEpoch, blockNow)
crossLinks := types.CrossLinks{}
if err := rlp.DecodeBytes(cxLinks, &crossLinks); err != nil {
return network.EmptyPayout, err
@ -391,11 +327,81 @@ func AccumulateRewardsAndCountSigs(
}
}
utils.AnalysisEnd("accumulateRewardShardchainPayout", nowEpoch, blockNow)
return network.NewStakingEraRewardForRound(
newRewards, missing, beaconP, shardP,
), nil
}
return network.EmptyPayout, nil
// Block here until the commit sigs are ready or timeout.
// sigsReady signal indicates that the commit sigs are already populated in the header object.
if err := waitForCommitSigs(sigsReady); err != nil {
return network.EmptyPayout, err
}
utils.AnalysisStart("accumulateRewardBeaconchainSelfPayout", nowEpoch, blockNow)
// Take care of my own beacon chain committee, _ is missing, for slashing
parentE, members, payable, missing, err := ballotResultBeaconchain(beaconChain, header)
if err != nil {
return network.EmptyPayout, err
}
subComm := shard.Committee{shard.BeaconChainShardID, members}
if err := availability.IncrementValidatorSigningCounts(
beaconChain,
subComm.StakedValidators(),
state,
payable,
missing,
); err != nil {
return network.EmptyPayout, err
}
votingPower, err := lookupVotingPower(
parentE, &subComm,
)
if err != nil {
return network.EmptyPayout, err
}
allSignersShare := numeric.ZeroDec()
for j := range payable {
voter := votingPower.Voters[payable[j].BLSPublicKey]
if !voter.IsHarmonyNode {
voterShare := voter.OverallPercent
allSignersShare = allSignersShare.Add(voterShare)
}
}
for beaconMember := range payable {
// TODO Give out whatever leftover to the last voter/handle
// what to do about share of those that didn't sign
blsKey := payable[beaconMember].BLSPublicKey
voter := votingPower.Voters[blsKey]
if !voter.IsHarmonyNode {
snapshot, err := bc.ReadValidatorSnapshot(voter.EarningAccount)
if err != nil {
return network.EmptyPayout, err
}
due := defaultReward.Mul(
voter.OverallPercent.Quo(allSignersShare),
).RoundInt()
newRewards.Add(newRewards, due)
shares, err := lookupDelegatorShares(snapshot)
if err != nil {
return network.EmptyPayout, err
}
if err := state.AddReward(snapshot.Validator, due, shares); err != nil {
return network.EmptyPayout, err
}
beaconP = append(beaconP, reward.Payout{
ShardID: shard.BeaconChainShardID,
Addr: voter.EarningAccount,
NewlyEarned: due,
EarningKey: voter.Identity,
})
}
}
utils.AnalysisEnd("accumulateRewardBeaconchainSelfPayout", nowEpoch, blockNow)
return network.NewStakingEraRewardForRound(
newRewards, missing, beaconP, shardP,
), nil
}
// Before staking
@ -417,6 +423,13 @@ func AccumulateRewardsAndCountSigs(
err, "cannot read shard state at epoch %v", parentHeader.Epoch(),
)
}
// Block here until the commit sigs are ready or timeout.
// sigsReady signal indicates that the commit sigs are already populated in the header object.
if err := waitForCommitSigs(sigsReady); err != nil {
return network.EmptyPayout, err
}
_, signers, _, err := availability.BallotResult(
parentHeader, header, parentShardState, header.ShardID(),
)
@ -452,3 +465,16 @@ func AccumulateRewardsAndCountSigs(
return network.NewPreStakingEraRewarded(totalAmount), nil
}
func waitForCommitSigs(sigsReady chan bool) error {
select {
case success := <-sigsReady:
if !success {
return errors.New("Failed to get commit sigs")
}
utils.Logger().Info().Msg("Commit sigs are ready")
case <-time.After(10 * time.Second):
return errors.New("Timeout waiting for commit sigs for reward calculation")
}
return nil
}

@ -42,8 +42,12 @@ func TestAddNewBlock(t *testing.T) {
node.Worker.CommitTransactions(
txs, stks, common.Address{},
)
commitSigs := make(chan []byte)
go func() {
commitSigs <- []byte{}
}()
block, _ := node.Worker.FinalizeNewBlock(
[]byte{}, []byte{}, 0, common.Address{}, nil, nil,
commitSigs, 0, common.Address{}, nil, nil,
)
_, err = node.Blockchain().InsertChain([]*types.Block{block}, true)
@ -81,8 +85,12 @@ func TestVerifyNewBlock(t *testing.T) {
node.Worker.CommitTransactions(
txs, stks, common.Address{},
)
commitSigs := make(chan []byte)
go func() {
commitSigs <- []byte{}
}()
block, _ := node.Worker.FinalizeNewBlock(
[]byte{}, []byte{}, 0, common.Address{}, nil, nil,
commitSigs, 0, common.Address{}, nil, nil,
)
if err := node.VerifyNewBlock(block); err != nil {

@ -47,12 +47,18 @@ func (node *Node) WaitForConsensusReadyV2(readySignal chan struct{}, stopChan ch
Uint64("blockNum", node.Blockchain().CurrentBlock().NumberU64()+1).
Msg("PROPOSING NEW BLOCK ------------------------------------------------")
newBlock, err := node.proposeNewBlock()
// Prepare last commit signatures
commitSigs := make(chan []byte)
sigs, err := node.Consensus.BlockCommitSigs(node.Blockchain().CurrentBlock().NumberU64())
if err != nil {
utils.Logger().Err(err).Msg("!!!!!!!!!Failed Proposing New Block!!!!!!!!!")
utils.Logger().Error().Err(err).Msg("[proposeNewBlock] Cannot get commit signatures from last block")
break
}
go func() {
commitSigs <- sigs
}()
newBlock, err := node.ProposeNewBlock(commitSigs)
err = node.Blockchain().Validator().ValidateHeader(newBlock, true)
if err == nil {
utils.Logger().Info().
Uint64("blockNum", newBlock.NumberU64()).
@ -67,7 +73,7 @@ func (node *Node) WaitForConsensusReadyV2(readySignal chan struct{}, stopChan ch
node.BlockChannel <- newBlock
break
} else {
utils.Logger().Err(err).Msg("!!!!!!!!!Failed Verifying New Block Header!!!!!!!!!")
utils.Logger().Err(err).Msg("!!!!!!!!!Failed Proposing New Block!!!!!!!!!")
}
}
}
@ -75,7 +81,7 @@ func (node *Node) WaitForConsensusReadyV2(readySignal chan struct{}, stopChan ch
}()
}
func (node *Node) proposeNewBlock() (*types.Block, error) {
func (node *Node) ProposeNewBlock(commitSigs chan []byte) (*types.Block, error) {
currentHeader := node.Blockchain().CurrentHeader()
nowEpoch, blockNow := currentHeader.Epoch(), currentHeader.Number()
utils.AnalysisStart("proposeNewBlock", nowEpoch, blockNow)
@ -224,17 +230,22 @@ func (node *Node) proposeNewBlock() (*types.Block, error) {
return nil, err
}
// Prepare last commit signatures
sig, mask, err := node.Consensus.BlockCommitSig(header.Number().Uint64() - 1)
finalizedBlock, err := node.Worker.FinalizeNewBlock(
commitSigs, node.Consensus.GetViewID(),
coinbase, crossLinksToPropose, shardState,
)
if err != nil {
utils.Logger().Error().Err(err).Msg("[proposeNewBlock] Cannot get commit signatures from last block")
utils.Logger().Error().Err(err).Msg("[proposeNewBlock] Failed finalizing the new block")
return nil, err
}
utils.Logger().Info().Msg("[proposeNewBlock] verifying the new block header")
err = node.Blockchain().Validator().ValidateHeader(finalizedBlock, true)
return node.Worker.FinalizeNewBlock(
sig, mask, node.Consensus.GetViewID(),
coinbase, crossLinksToPropose, shardState,
)
if err != nil {
utils.Logger().Error().Err(err).Msg("[proposeNewBlock] Failed verifying the new block header")
return nil, err
}
return finalizedBlock, nil
}
func (node *Node) proposeReceiptsProof() []*types.CXReceiptsProof {

@ -7,6 +7,8 @@ import (
"sort"
"time"
"github.com/harmony-one/harmony/crypto/bls"
"github.com/harmony-one/harmony/crypto/hash"
"github.com/ethereum/go-ethereum/common"
@ -422,18 +424,9 @@ func (w *Worker) verifySlashes(
// FinalizeNewBlock generate a new block for the next consensus round.
func (w *Worker) FinalizeNewBlock(
sig []byte, signers []byte, viewID uint64, coinbase common.Address,
commitSigs chan []byte, viewID uint64, coinbase common.Address,
crossLinks types.CrossLinks, shardState *shard.State,
) (*types.Block, error) {
// Put sig, signers, viewID, coinbase into header
if len(sig) > 0 && len(signers) > 0 {
// TODO: directly set signature into lastCommitSignature
sig2 := w.current.header.LastCommitSignature()
copy(sig2[:], sig[:])
w.current.header.SetLastCommitSignature(sig2)
w.current.header.SetLastCommitBitmap(signers)
}
w.current.header.SetCoinbase(coinbase)
w.current.header.SetViewID(new(big.Int).SetUint64(viewID))
@ -491,15 +484,39 @@ func (w *Worker) FinalizeNewBlock(
}
state := w.current.state.Copy()
copyHeader := types.CopyHeader(w.current.header)
sigsReady := make(chan bool)
go func() {
select {
case sigs := <-commitSigs:
sig, signers, err := bls.SeparateSigAndMask(sigs)
if err != nil {
utils.Logger().Error().Err(err).Msg("Failed to parse commit sigs")
sigsReady <- false
}
// Put sig, signers, viewID, coinbase into header
if len(sig) > 0 && len(signers) > 0 {
sig2 := copyHeader.LastCommitSignature()
copy(sig2[:], sig[:])
utils.Logger().Info().Hex("sigs", sig).Hex("bitmap", signers).Msg("Setting commit sigs")
copyHeader.SetLastCommitSignature(sig2)
copyHeader.SetLastCommitBitmap(signers)
}
sigsReady <- true
case <-time.After(10 * time.Second):
// Exit goroutine
utils.Logger().Warn().Msg("Timeout waiting for commit sigs")
}
}()
block, _, err := w.engine.Finalize(
w.chain, copyHeader, state, w.current.txs, w.current.receipts,
w.current.outcxs, w.current.incxs, w.current.stakingTxs,
w.current.slashes,
w.current.slashes, sigsReady,
)
if err != nil {
return nil, errors.Wrapf(err, "cannot finalize block")
}
return block, nil
}

@ -0,0 +1,86 @@
package main
import (
"fmt"
"github.com/harmony-one/harmony/crypto/bls"
"math/big"
common2 "github.com/ethereum/go-ethereum/common"
bls_core "github.com/harmony-one/bls/ffi/go/bls"
"github.com/harmony-one/harmony/crypto/hash"
"github.com/harmony-one/harmony/internal/common"
staking "github.com/harmony-one/harmony/staking/types"
)
var (
validatorAddress = common2.Address(common.MustBech32ToAddress("one1pdv9lrdwl0rg5vglh4xtyrv3wjk3wsqket7zxy"))
testBLSPubKey = "30b2c38b1316da91e068ac3bd8751c0901ef6c02a1d58bc712104918302c6ed03d5894671d0c816dad2b4d303320f202"
testBLSPrvKey = "c6d7603520311f7a4e6aac0b26701fc433b75b38df504cd416ef2b900cd66205"
postStakingEpoch = big.NewInt(200)
)
func init() {
bls_core.Init(bls_core.BLS12_381)
}
func generateBLSKeySigPair() (bls.SerializedPublicKey, bls.SerializedSignature) {
p := &bls_core.PublicKey{}
p.DeserializeHexStr(testBLSPubKey)
pub := bls.SerializedPublicKey{}
pub.FromLibBLSPublicKey(p)
messageBytes := []byte(staking.BLSVerificationStr)
privateKey := &bls_core.SecretKey{}
privateKey.DeserializeHexStr(testBLSPrvKey)
msgHash := hash.Keccak256(messageBytes)
signature := privateKey.SignHash(msgHash[:])
var sig bls.SerializedSignature
copy(sig[:], signature.Serialize())
return pub, sig
}
func main() {
pri1 := bls.RandPrivateKey()
pri2 := bls.RandPrivateKey()
pri3 := bls.RandPrivateKey()
pri4 := bls.RandPrivateKey()
pub1 := pri1.GetPublicKey()
pub2 := pri2.GetPublicKey()
pub3 := pri3.GetPublicKey()
pub4 := pri4.GetPublicKey()
w1 := bls.PublicKeyWrapper{Object: pub1}
w1.Bytes.FromLibBLSPublicKey(w1.Object)
w2 := bls.PublicKeyWrapper{Object: pub2}
w2.Bytes.FromLibBLSPublicKey(w2.Object)
w3 := bls.PublicKeyWrapper{Object: pub3}
w3.Bytes.FromLibBLSPublicKey(w3.Object)
w4 := bls.PublicKeyWrapper{Object: pub4}
w4.Bytes.FromLibBLSPublicKey(w4.Object)
wrappers := []bls.PublicKeyWrapper{w1, w2, w3, w4}
bls.NewMask(wrappers, nil)
messageBytes := []byte(staking.BLSVerificationStr)
msgHash := hash.Keccak256(messageBytes)
s1 := pri1.SignHash(msgHash)
//s2 := pri2.SignHash(msgHash)
//s3 := pri3.SignHash(msgHash)
//s4 := pri4.SignHash(msgHash)
var aggregatedSig bls_core.Sign
aggregatedSig.Add(s1)
aggPub := &bls_core.PublicKey{}
aggPub.Add(pub1)
fmt.Println(aggregatedSig.VerifyHash(pub1, msgHash))
}

@ -126,8 +126,12 @@ func fundFaucetContract(chain *core.BlockChain) {
if err != nil {
fmt.Println(err)
}
commitSigs := make(chan []byte)
go func() {
commitSigs <- []byte{}
}()
block, _ := contractworker.
FinalizeNewBlock([]byte{}, []byte{}, 0, common.Address{}, nil, nil)
FinalizeNewBlock(commitSigs, 0, common.Address{}, nil, nil)
_, err = chain.InsertChain(types.Blocks{block}, true /* verifyHeaders */)
if err != nil {
fmt.Println(err)
@ -171,8 +175,12 @@ func callFaucetContractToFundAnAddress(chain *core.BlockChain) {
if err != nil {
fmt.Println(err)
}
commitSigs := make(chan []byte)
go func() {
commitSigs <- []byte{}
}()
block, _ := contractworker.FinalizeNewBlock(
[]byte{}, []byte{}, 0, common.Address{}, nil, nil,
commitSigs, 0, common.Address{}, nil, nil,
)
_, err = chain.InsertChain(types.Blocks{block}, true /* verifyHeaders */)
if err != nil {

Loading…
Cancel
Save