[committee] Factor out committee membership, provide entry for alternative committee membership (#1818)

* [committee] Move core.ShardingSchedule to shard.Schedule

* [consensus] Remove redundant PublicKeys field of Consensus as Decider maintains that

* [committee] Use committee package to pick PublicKeys

* [committee] Use committee inplace of CalculateShardState

* [committee] Remove core/resharding.go, complete usage of committee as implementation replacement

* [committee] Address PR comments
pull/1821/head
Edgar Aroutiounian 5 years ago committed by GitHub
parent 4e628224e6
commit e6a4fbea4f
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 97
      api/service/resharding/service.go
  2. 17
      cmd/client/txgen/main.go
  3. 41
      cmd/harmony/main.go
  4. 3
      consensus/consensus.go
  5. 6
      consensus/consensus_leader_msg_test.go
  6. 39
      consensus/consensus_service.go
  7. 8
      consensus/consensus_service_test.go
  8. 4
      consensus/consensus_test.go
  9. 8
      consensus/consensus_v2.go
  10. 6
      consensus/consensus_validator_msg_test.go
  11. 4
      consensus/fbft_log_test.go
  12. 39
      core/blockchain.go
  13. 3
      core/core_test.go
  14. 5
      core/genesis.go
  15. 259
      core/resharding.go
  16. 12
      core/resharding.md
  17. 149
      core/resharding_test.go
  18. 3
      core/state_processor.go
  19. 10
      core/values/blockchain.go
  20. 6
      drand/drand_leader.go
  21. 7
      internal/chain/engine.go
  22. 13
      internal/configs/sharding/localnet.go
  23. 5
      internal/hmyapi/blockchain.go
  24. 2
      internal/params/config.go
  25. 41
      node/node.go
  26. 10
      node/node_cross_shard.go
  27. 3
      node/node_explorer.go
  28. 7
      node/node_genesis.go
  29. 72
      node/node_handler.go
  30. 6
      node/node_handler_test.go
  31. 21
      node/node_newblock.go
  32. 15
      node/node_resharding.go
  33. 8
      node/node_test.go
  34. 12
      node/worker/worker.go
  35. 261
      shard/committee/assignment.go
  36. 76
      shard/shard_state.go
  37. 36
      shard/shard_state_test.go
  38. 19
      shard/values.go

@ -1,97 +0,0 @@
package resharding
import (
"time"
"github.com/ethereum/go-ethereum/rpc"
msg_pb "github.com/harmony-one/harmony/api/proto/message"
"github.com/harmony-one/harmony/core"
"github.com/harmony-one/harmony/internal/utils"
)
// Constants for resharding service.
const (
ReshardingCheckTime = time.Second
)
// Service is the role conversion service.
type Service struct {
stopChan chan struct{}
stoppedChan chan struct{}
messageChan chan *msg_pb.Message
beaconChain *core.BlockChain
}
// New returns role conversion service.
func New(beaconChain *core.BlockChain) *Service {
return &Service{beaconChain: beaconChain}
}
// StartService starts role conversion service.
func (s *Service) StartService() {
s.stopChan = make(chan struct{})
s.stoppedChan = make(chan struct{})
s.Init()
s.Run(s.stopChan, s.stoppedChan)
}
// Init initializes role conversion service.
func (s *Service) Init() {
}
// Run runs role conversion.
func (s *Service) Run(stopChan chan struct{}, stoppedChan chan struct{}) {
go func() {
defer close(stoppedChan)
for {
select {
default:
utils.Logger().Info().Msg("Running role conversion")
// TODO: Write some logic here.
s.DoService()
case <-stopChan:
return
}
}
}()
}
// DoService does role conversion.
func (s *Service) DoService() {
tick := time.NewTicker(ReshardingCheckTime)
// Get current shard state hash.
currentShardStateHash := s.beaconChain.CurrentBlock().Header().ShardStateHash()
for {
select {
case <-tick.C:
LatestShardStateHash := s.beaconChain.CurrentBlock().Header().ShardStateHash()
if currentShardStateHash != LatestShardStateHash {
// TODO(minhdoan): Add resharding logic later after modifying the resharding func as it current doesn't calculate the role (leader/validator)
}
}
}
}
// StopService stops role conversion service.
func (s *Service) StopService() {
utils.Logger().Info().Msg("Stopping role conversion service")
s.stopChan <- struct{}{}
<-s.stoppedChan
utils.Logger().Info().Msg("Role conversion stopped")
}
// NotifyService notify service
func (s *Service) NotifyService(params map[string]interface{}) {
return
}
// SetMessageChan sets up message channel to service.
func (s *Service) SetMessageChan(messageChan chan *msg_pb.Message) {
s.messageChan = messageChan
}
// APIs for the services.
func (s *Service) APIs() []rpc.API {
return nil
}

@ -10,29 +10,28 @@ import (
"sync"
"time"
"github.com/harmony-one/harmony/consensus"
"github.com/harmony-one/harmony/consensus/quorum"
"github.com/harmony-one/harmony/core"
"github.com/harmony-one/harmony/internal/ctxerror"
"github.com/harmony-one/harmony/internal/shardchain"
"github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/log"
bls2 "github.com/harmony-one/bls/ffi/go/bls"
"github.com/harmony-one/harmony/internal/params"
"github.com/harmony-one/harmony/api/client"
proto_node "github.com/harmony-one/harmony/api/proto/node"
"github.com/harmony-one/harmony/common/denominations"
"github.com/harmony-one/harmony/consensus"
"github.com/harmony-one/harmony/consensus/quorum"
"github.com/harmony-one/harmony/core"
"github.com/harmony-one/harmony/core/types"
"github.com/harmony-one/harmony/crypto/bls"
nodeconfig "github.com/harmony-one/harmony/internal/configs/node"
"github.com/harmony-one/harmony/internal/ctxerror"
"github.com/harmony-one/harmony/internal/genesis"
"github.com/harmony-one/harmony/internal/params"
"github.com/harmony-one/harmony/internal/shardchain"
"github.com/harmony-one/harmony/internal/utils"
"github.com/harmony-one/harmony/node"
"github.com/harmony-one/harmony/p2p"
p2p_host "github.com/harmony-one/harmony/p2p/host"
"github.com/harmony-one/harmony/p2p/p2pimpl"
"github.com/harmony-one/harmony/shard"
)
var (
@ -105,7 +104,7 @@ func setUpTXGen() *node.Node {
txGen := node.New(myhost, consensusObj, chainDBFactory, false) //Changed it : no longer archival node.
txGen.Client = client.NewClient(txGen.GetHost(), uint32(shardID))
consensusObj.ChainReader = txGen.Blockchain()
genesisShardingConfig := core.ShardingSchedule.InstanceForEpoch(big.NewInt(core.GenesisEpoch))
genesisShardingConfig := shard.Schedule.InstanceForEpoch(big.NewInt(core.GenesisEpoch))
startIdx := 0
endIdx := startIdx + genesisShardingConfig.NumNodesPerShard()
pubs := []*bls2.PublicKey{}

@ -16,7 +16,6 @@ import (
"github.com/ethereum/go-ethereum/ethdb"
"github.com/ethereum/go-ethereum/log"
"github.com/harmony-one/bls/ffi/go/bls"
"github.com/harmony-one/harmony/api/service/syncing"
"github.com/harmony-one/harmony/consensus"
"github.com/harmony-one/harmony/consensus/quorum"
@ -34,6 +33,7 @@ import (
"github.com/harmony-one/harmony/node"
"github.com/harmony-one/harmony/p2p"
"github.com/harmony-one/harmony/p2p/p2pimpl"
"github.com/harmony-one/harmony/shard"
)
// Version string variables
@ -71,7 +71,6 @@ func printVersion() {
os.Exit(0)
}
// Flags
var (
ip = flag.String("ip", "127.0.0.1", "ip of the node")
port = flag.String("port", "9000", "port of the node.")
@ -102,7 +101,6 @@ var (
syncFreq = flag.Int("sync_freq", 60, "unit in seconds")
// beaconSyncFreq indicates beaconchain sync frequency
beaconSyncFreq = flag.Int("beacon_sync_freq", 60, "unit in seconds")
// blockPeriod indicates the how long the leader waits to propose a new block.
blockPeriod = flag.Int("block_period", 8, "how long in second the leader waits to propose a new block.")
leaderOverride = flag.Bool("leader_override", false, "true means override the default leader role and acts as validator")
@ -113,33 +111,24 @@ var (
blsKeyFile = flag.String("blskey_file", "", "The encrypted file of bls serialized private key by passphrase.")
blsPass = flag.String("blspass", "", "The file containing passphrase to decrypt the encrypted bls file.")
blsPassphrase string
// Sharding configuration parameters for devnet
devnetNumShards = flag.Uint("dn_num_shards", 2, "number of shards for -network_type=devnet (default: 2)")
devnetShardSize = flag.Int("dn_shard_size", 10, "number of nodes per shard for -network_type=devnet (default 10)")
devnetHarmonySize = flag.Int("dn_hmy_size", -1, "number of Harmony-operated nodes per shard for -network_type=devnet; negative (default) means equal to -dn_shard_size")
// logConn logs incoming/outgoing connections
logConn = flag.Bool("log_conn", false, "log incoming/outgoing connections")
keystoreDir = flag.String("keystore", hmykey.DefaultKeyStoreDir, "The default keystore directory")
initialAccount = &genesis.DeployAccount{}
// logging verbosity
verbosity = flag.Int("verbosity", 5, "Logging verbosity: 0=silent, 1=error, 2=warn, 3=info, 4=debug, 5=detail (default: 5)")
// dbDir is the database directory.
dbDir = flag.String("db_dir", "", "blockchain database directory")
// Disable view change.
disableViewChange = flag.Bool("disable_view_change", false, "Do not propose view change (testing only)")
// metrics flag to collct meetrics or not, pushgateway ip and port for metrics
metricsFlag = flag.Bool("metrics", false, "Collect and upload node metrics")
pushgatewayIP = flag.String("pushgateway_ip", "grafana.harmony.one", "Metrics view ip")
pushgatewayPort = flag.String("pushgateway_port", "9091", "Metrics view port")
publicRPC = flag.Bool("public_rpc", false, "Enable Public RPC Access (default: false)")
)
@ -203,13 +192,13 @@ func passphraseForBls() {
}
func setupInitialAccount() (isLeader bool) {
genesisShardingConfig := core.ShardingSchedule.InstanceForEpoch(big.NewInt(core.GenesisEpoch))
genesisShardingConfig := shard.Schedule.InstanceForEpoch(big.NewInt(core.GenesisEpoch))
pubKey := setupConsensusKey(nodeconfig.GetDefaultConfig())
reshardingEpoch := genesisShardingConfig.ReshardingEpoch()
if reshardingEpoch != nil && len(reshardingEpoch) > 0 {
for _, epoch := range reshardingEpoch {
config := core.ShardingSchedule.InstanceForEpoch(epoch)
config := shard.Schedule.InstanceForEpoch(epoch)
isLeader, initialAccount = config.FindAccount(pubKey.SerializeToHexStr())
if initialAccount != nil {
break
@ -323,7 +312,7 @@ func setupConsensusAndNode(nodeConfig *nodeconfig.ConfigType) *node.Node {
switch {
case *networkType == nodeconfig.Localnet:
epochConfig := core.ShardingSchedule.InstanceForEpoch(ethCommon.Big0)
epochConfig := shard.Schedule.InstanceForEpoch(ethCommon.Big0)
selfPort, err := strconv.ParseUint(*port, 10, 16)
if err != nil {
utils.Logger().Fatal().
@ -359,9 +348,9 @@ func setupConsensusAndNode(nodeConfig *nodeconfig.ConfigType) *node.Node {
currentNode.NodeConfig.SetClientGroupID(nodeconfig.NewClientGroupIDByShardID(nodeconfig.ShardID(*shardID)))
case "validator":
currentNode.NodeConfig.SetRole(nodeconfig.Validator)
if nodeConfig.ShardID == 0 {
currentNode.NodeConfig.SetShardGroupID(nodeconfig.NewGroupIDByShardID(0))
currentNode.NodeConfig.SetClientGroupID(nodeconfig.NewClientGroupIDByShardID(0))
if nodeConfig.ShardID == shard.BeaconChainShardID {
currentNode.NodeConfig.SetShardGroupID(nodeconfig.NewGroupIDByShardID(shard.BeaconChainShardID))
currentNode.NodeConfig.SetClientGroupID(nodeconfig.NewClientGroupIDByShardID(shard.BeaconChainShardID))
} else {
currentNode.NodeConfig.SetShardGroupID(nodeconfig.NewGroupIDByShardID(nodeconfig.ShardID(nodeConfig.ShardID)))
currentNode.NodeConfig.SetClientGroupID(nodeconfig.NewClientGroupIDByShardID(nodeconfig.ShardID(nodeConfig.ShardID)))
@ -381,8 +370,8 @@ func setupConsensusAndNode(nodeConfig *nodeconfig.ConfigType) *node.Node {
// currentNode.DRand = dRand
// This needs to be executed after consensus and drand are setup
if err := currentNode.CalculateInitShardState(); err != nil {
ctxerror.Crit(utils.GetLogger(), err, "CalculateInitShardState failed",
if err := currentNode.InitConsensusWithValidators(); err != nil {
ctxerror.Crit(utils.GetLogger(), err, "InitConsensusWithMembers failed",
"shardID", *shardID)
}
@ -429,13 +418,13 @@ func main() {
switch *networkType {
case nodeconfig.Mainnet:
core.ShardingSchedule = shardingconfig.MainnetSchedule
shard.Schedule = shardingconfig.MainnetSchedule
case nodeconfig.Testnet:
core.ShardingSchedule = shardingconfig.TestnetSchedule
shard.Schedule = shardingconfig.TestnetSchedule
case nodeconfig.Pangaea:
core.ShardingSchedule = shardingconfig.PangaeaSchedule
shard.Schedule = shardingconfig.PangaeaSchedule
case nodeconfig.Localnet:
core.ShardingSchedule = shardingconfig.LocalnetSchedule
shard.Schedule = shardingconfig.LocalnetSchedule
case nodeconfig.Devnet:
if *devnetHarmonySize < 0 {
*devnetHarmonySize = *devnetShardSize
@ -448,7 +437,7 @@ func main() {
err)
os.Exit(1)
}
core.ShardingSchedule = shardingconfig.NewFixedSchedule(devnetConfig)
shard.Schedule = shardingconfig.NewFixedSchedule(devnetConfig)
}
initSetup()
@ -476,7 +465,7 @@ func main() {
currentNode.SetSyncFreq(*syncFreq)
currentNode.SetBeaconSyncFreq(*beaconSyncFreq)
if nodeConfig.ShardID != 0 && currentNode.NodeConfig.Role() != nodeconfig.ExplorerNode {
if nodeConfig.ShardID != shard.BeaconChainShardID && currentNode.NodeConfig.Role() != nodeconfig.ExplorerNode {
utils.GetLogInstance().Info("SupportBeaconSyncing", "shardID", currentNode.Blockchain().ShardID(), "shardID", nodeConfig.ShardID)
go currentNode.SupportBeaconSyncing()
}

@ -79,8 +79,6 @@ type Consensus struct {
// If the number of validators is less than minPeers, the consensus won't start
MinPeers int
CommitteePublicKeys map[string]bool
pubKeyLock sync.Mutex
// private/public keys of current node
@ -216,7 +214,6 @@ func New(
consensus.current = State{mode: Normal}
// FBFT timeout
consensus.consensusTimeout = createTimeout()
consensus.CommitteePublicKeys = make(map[string]bool)
consensus.validators.Store(leader.ConsensusPubKey.SerializeToHexStr(), leader)
if blsPriKey != nil {

@ -7,12 +7,12 @@ import (
"github.com/harmony-one/harmony/api/proto"
msg_pb "github.com/harmony-one/harmony/api/proto/message"
"github.com/harmony-one/harmony/consensus/quorum"
"github.com/harmony-one/harmony/core/values"
"github.com/harmony-one/harmony/crypto/bls"
"github.com/harmony-one/harmony/internal/ctxerror"
"github.com/harmony-one/harmony/internal/utils"
"github.com/harmony-one/harmony/p2p"
"github.com/harmony-one/harmony/p2p/p2pimpl"
"github.com/harmony-one/harmony/shard"
)
func TestConstructAnnounceMessage(test *testing.T) {
@ -24,7 +24,7 @@ func TestConstructAnnounceMessage(test *testing.T) {
}
decider := quorum.NewDecider(quorum.SuperMajorityVote)
consensus, err := New(
host, values.BeaconChainShardID, leader, bls.RandPrivateKey(), decider,
host, shard.BeaconChainShardID, leader, bls.RandPrivateKey(), decider,
)
if err != nil {
test.Fatalf("Cannot create consensus: %v", err)
@ -57,7 +57,7 @@ func TestConstructPreparedMessage(test *testing.T) {
}
decider := quorum.NewDecider(quorum.SuperMajorityVote)
consensus, err := New(
host, values.BeaconChainShardID, leader, bls.RandPrivateKey(), decider,
host, shard.BeaconChainShardID, leader, bls.RandPrivateKey(), decider,
)
if err != nil {
test.Fatalf("Cannot craeate consensus: %v", err)

@ -14,7 +14,6 @@ import (
"github.com/harmony-one/harmony/block"
consensus_engine "github.com/harmony-one/harmony/consensus/engine"
"github.com/harmony-one/harmony/consensus/quorum"
"github.com/harmony-one/harmony/core"
"github.com/harmony-one/harmony/core/types"
bls_cosi "github.com/harmony-one/harmony/crypto/bls"
"github.com/harmony-one/harmony/crypto/hash"
@ -23,6 +22,8 @@ import (
"github.com/harmony-one/harmony/internal/profiler"
"github.com/harmony-one/harmony/internal/utils"
"github.com/harmony-one/harmony/p2p"
"github.com/harmony-one/harmony/shard"
"github.com/harmony-one/harmony/shard/committee"
libp2p_peer "github.com/libp2p/go-libp2p-peer"
"github.com/rs/zerolog"
)
@ -110,17 +111,14 @@ func (consensus *Consensus) DebugPrintPublicKeys() {
utils.Logger().Debug().Strs("PublicKeys", keys).Int("count", len(keys)).Msgf("Debug Public Keys")
}
// UpdatePublicKeys updates the PublicKeys variable, protected by a mutex
// UpdatePublicKeys updates the PublicKeys for quorum on current subcommittee, protected by a mutex
func (consensus *Consensus) UpdatePublicKeys(pubKeys []*bls.PublicKey) int64 {
consensus.pubKeyLock.Lock()
consensus.Decider.UpdateParticipants(pubKeys)
consensus.CommitteePublicKeys = map[string]bool{}
utils.Logger().Info().Msg("My Committee updated")
for i, pubKey := range consensus.Decider.DumpParticipants() {
utils.Logger().Info().Int("index", i).Str("BlsPubKey", pubKey).Msg("Member")
consensus.CommitteePublicKeys[pubKey] = true
for i := range pubKeys {
utils.Logger().Info().Int("index", i).Str("BLSPubKey", pubKeys[i].SerializeToHexStr()).Msg("Member")
}
consensus.LeaderPubKey = pubKeys[0]
utils.Logger().Info().
Str("info", consensus.LeaderPubKey.SerializeToHexStr()).Msg("My Leader")
@ -230,8 +228,7 @@ func (consensus *Consensus) ToggleConsensusCheck() {
// IsValidatorInCommittee returns whether the given validator BLS address is part of my committee
func (consensus *Consensus) IsValidatorInCommittee(pubKey *bls.PublicKey) bool {
_, ok := consensus.CommitteePublicKeys[pubKey.SerializeToHexStr()]
return ok
return consensus.Decider.IndexOf(pubKey) != -1
}
// Verify the signature of the message are valid from the signer's public key.
@ -458,22 +455,21 @@ func (consensus *Consensus) getLeaderPubKeyFromCoinbase(header *block.Header) (*
func (consensus *Consensus) UpdateConsensusInformation() Mode {
pubKeys := []*bls.PublicKey{}
hasError := false
header := consensus.ChainReader.CurrentHeader()
epoch := header.Epoch()
curPubKeys := core.CalculatePublicKeys(epoch, header.ShardID())
_, curPubKeys := committee.WithStakingEnabled.ComputePublicKeys(
epoch, consensus.ChainReader, int(header.ShardID()),
)
consensus.numPrevPubKeys = len(curPubKeys)
consensus.getLogger().Info().Msg("[UpdateConsensusInformation] Updating.....")
if core.IsEpochLastBlockByHeader(header) {
if shard.Schedule.IsLastBlock(header.Number().Uint64()) {
// increase epoch by one if it's the last block
consensus.SetEpochNum(epoch.Uint64() + 1)
consensus.getLogger().Info().Uint64("headerNum", header.Number().Uint64()).
Msg("[UpdateConsensusInformation] Epoch updated for next epoch")
nextEpoch := new(big.Int).Add(epoch, common.Big1)
pubKeys = core.CalculatePublicKeys(nextEpoch, header.ShardID())
_, pubKeys = committee.WithStakingEnabled.ComputePublicKeys(
new(big.Int).Add(epoch, common.Big1), consensus.ChainReader, int(header.ShardID()),
)
} else {
consensus.SetEpochNum(epoch.Uint64())
pubKeys = curPubKeys
@ -493,7 +489,8 @@ func (consensus *Consensus) UpdateConsensusInformation() Mode {
consensus.UpdatePublicKeys(pubKeys)
// take care of possible leader change during the epoch
if !core.IsEpochLastBlockByHeader(header) && header.Number().Uint64() != 0 {
if !shard.Schedule.IsLastBlock(header.Number().Uint64()) &&
header.Number().Uint64() != 0 {
leaderPubKey, err := consensus.getLeaderPubKeyFromCoinbase(header)
if err != nil || leaderPubKey == nil {
consensus.getLogger().Debug().Err(err).
@ -508,9 +505,9 @@ func (consensus *Consensus) UpdateConsensusInformation() Mode {
}
}
for _, key := range pubKeys {
for i := range pubKeys {
// in committee
if key.IsEqual(consensus.PubKey) {
if pubKeys[i].IsEqual(consensus.PubKey) {
if hasError {
return Syncing
}
@ -544,7 +541,7 @@ func (consensus *Consensus) IsLeader() bool {
// NeedsRandomNumberGeneration returns true if the current epoch needs random number generation
func (consensus *Consensus) NeedsRandomNumberGeneration(epoch *big.Int) bool {
if consensus.ShardID == 0 && epoch.Uint64() >= core.ShardingSchedule.RandomnessStartingEpoch() {
if consensus.ShardID == 0 && epoch.Uint64() >= shard.Schedule.RandomnessStartingEpoch() {
return true
}

@ -6,11 +6,11 @@ import (
msg_pb "github.com/harmony-one/harmony/api/proto/message"
"github.com/harmony-one/harmony/consensus/quorum"
"github.com/harmony-one/harmony/core/values"
"github.com/harmony-one/harmony/crypto/bls"
"github.com/harmony-one/harmony/internal/utils"
"github.com/harmony-one/harmony/p2p"
"github.com/harmony-one/harmony/p2p/p2pimpl"
"github.com/harmony-one/harmony/shard"
)
func TestPopulateMessageFields(t *testing.T) {
@ -23,7 +23,7 @@ func TestPopulateMessageFields(t *testing.T) {
blsPriKey := bls.RandPrivateKey()
decider := quorum.NewDecider(quorum.SuperMajorityVote)
consensus, err := New(
host, values.BeaconChainShardID, leader, blsPriKey, decider,
host, shard.BeaconChainShardID, leader, blsPriKey, decider,
)
if err != nil {
t.Fatalf("Cannot craeate consensus: %v", err)
@ -60,7 +60,7 @@ func TestSignAndMarshalConsensusMessage(t *testing.T) {
}
decider := quorum.NewDecider(quorum.SuperMajorityVote)
consensus, err := New(
host, values.BeaconChainShardID, leader, bls.RandPrivateKey(), decider,
host, shard.BeaconChainShardID, leader, bls.RandPrivateKey(), decider,
)
if err != nil {
t.Fatalf("Cannot craeate consensus: %v", err)
@ -88,7 +88,7 @@ func TestSetViewID(t *testing.T) {
}
decider := quorum.NewDecider(quorum.SuperMajorityVote)
consensus, err := New(
host, values.BeaconChainShardID, leader, bls.RandPrivateKey(), decider,
host, shard.BeaconChainShardID, leader, bls.RandPrivateKey(), decider,
)
if err != nil {
t.Fatalf("Cannot craeate consensus: %v", err)

@ -4,11 +4,11 @@ import (
"testing"
"github.com/harmony-one/harmony/consensus/quorum"
"github.com/harmony-one/harmony/core/values"
"github.com/harmony-one/harmony/crypto/bls"
"github.com/harmony-one/harmony/internal/utils"
"github.com/harmony-one/harmony/p2p"
"github.com/harmony-one/harmony/p2p/p2pimpl"
"github.com/harmony-one/harmony/shard"
)
func TestNew(test *testing.T) {
@ -20,7 +20,7 @@ func TestNew(test *testing.T) {
}
decider := quorum.NewDecider(quorum.SuperMajorityVote)
consensus, err := New(
host, values.BeaconChainShardID, leader, bls.RandPrivateKey(), decider,
host, shard.BeaconChainShardID, leader, bls.RandPrivateKey(), decider,
)
if err != nil {
test.Fatalf("Cannot craeate consensus: %v", err)

@ -14,7 +14,6 @@ import (
msg_pb "github.com/harmony-one/harmony/api/proto/message"
"github.com/harmony-one/harmony/block"
"github.com/harmony-one/harmony/consensus/quorum"
"github.com/harmony-one/harmony/core"
"github.com/harmony-one/harmony/core/types"
vrf_bls "github.com/harmony-one/harmony/crypto/vrf/bls"
"github.com/harmony-one/harmony/internal/chain"
@ -22,6 +21,7 @@ import (
"github.com/harmony-one/harmony/internal/ctxerror"
"github.com/harmony-one/harmony/internal/utils"
"github.com/harmony-one/harmony/p2p/host"
"github.com/harmony-one/harmony/shard"
"github.com/harmony-one/vdf/src/vdf_go"
)
@ -1187,7 +1187,7 @@ func (consensus *Consensus) Start(blockChannel chan *types.Block, stopChan chan
if err == nil {
vdfInProgress = false
// Verify the randomness
vdfObject := vdf_go.New(core.ShardingSchedule.VdfDifficulty(), seed)
vdfObject := vdf_go.New(shard.Schedule.VdfDifficulty(), seed)
if !vdfObject.Verify(vdfOutput) {
consensus.getLogger().Warn().
Uint64("MsgBlockNum", newBlock.NumberU64()).
@ -1323,7 +1323,7 @@ func (consensus *Consensus) GenerateVdfAndProof(newBlock *types.Block, vrfBlockN
// TODO ek – limit concurrency
go func() {
vdf := vdf_go.New(core.ShardingSchedule.VdfDifficulty(), seed)
vdf := vdf_go.New(shard.Schedule.VdfDifficulty(), seed)
outputChannel := vdf.GetOutputChannel()
start := time.Now()
vdf.Execute()
@ -1364,7 +1364,7 @@ func (consensus *Consensus) ValidateVdfAndProof(headerObj *block.Header) bool {
}
}
vdfObject := vdf_go.New(core.ShardingSchedule.VdfDifficulty(), seed)
vdfObject := vdf_go.New(shard.Schedule.VdfDifficulty(), seed)
vdfOutput := [516]byte{}
copy(vdfOutput[:], headerObj.Vdf())
if vdfObject.Verify(vdfOutput) {

@ -7,11 +7,11 @@ import (
"github.com/harmony-one/harmony/api/proto"
msg_pb "github.com/harmony-one/harmony/api/proto/message"
"github.com/harmony-one/harmony/consensus/quorum"
"github.com/harmony-one/harmony/core/values"
"github.com/harmony-one/harmony/crypto/bls"
"github.com/harmony-one/harmony/internal/utils"
"github.com/harmony-one/harmony/p2p"
"github.com/harmony-one/harmony/p2p/p2pimpl"
"github.com/harmony-one/harmony/shard"
)
func TestConstructPrepareMessage(test *testing.T) {
@ -23,7 +23,7 @@ func TestConstructPrepareMessage(test *testing.T) {
}
decider := quorum.NewDecider(quorum.SuperMajorityVote)
consensus, err := New(
host, values.BeaconChainShardID, leader, bls.RandPrivateKey(), decider,
host, shard.BeaconChainShardID, leader, bls.RandPrivateKey(), decider,
)
if err != nil {
test.Fatalf("Cannot craeate consensus: %v", err)
@ -54,7 +54,7 @@ func TestConstructCommitMessage(test *testing.T) {
}
decider := quorum.NewDecider(quorum.SuperMajorityVote)
consensus, err := New(
host, values.BeaconChainShardID, leader, bls.RandPrivateKey(), decider,
host, shard.BeaconChainShardID, leader, bls.RandPrivateKey(), decider,
)
if err != nil {
test.Fatalf("Cannot craeate consensus: %v", err)

@ -7,11 +7,11 @@ import (
"github.com/harmony-one/harmony/api/proto"
msg_pb "github.com/harmony-one/harmony/api/proto/message"
"github.com/harmony-one/harmony/consensus/quorum"
"github.com/harmony-one/harmony/core/values"
"github.com/harmony-one/harmony/crypto/bls"
"github.com/harmony-one/harmony/internal/utils"
"github.com/harmony-one/harmony/p2p"
"github.com/harmony-one/harmony/p2p/p2pimpl"
"github.com/harmony-one/harmony/shard"
)
func constructAnnounceMessage(t *testing.T) []byte {
@ -23,7 +23,7 @@ func constructAnnounceMessage(t *testing.T) []byte {
}
decider := quorum.NewDecider(quorum.SuperMajorityVote)
consensus, err := New(
host, values.BeaconChainShardID, leader, bls.RandPrivateKey(), decider,
host, shard.BeaconChainShardID, leader, bls.RandPrivateKey(), decider,
)
if err != nil {
t.Fatalf("Cannot create consensus: %v", err)

@ -45,6 +45,7 @@ import (
"github.com/harmony-one/harmony/internal/params"
"github.com/harmony-one/harmony/internal/utils"
"github.com/harmony-one/harmony/shard"
"github.com/harmony-one/harmony/shard/committee"
staking "github.com/harmony-one/harmony/staking/types"
lru "github.com/hashicorp/golang-lru"
)
@ -246,27 +247,16 @@ func IsEpochBlock(block *types.Block) bool {
// genesis block is the first epoch block
return true
}
return ShardingSchedule.IsLastBlock(block.NumberU64() - 1)
return shard.Schedule.IsLastBlock(block.NumberU64() - 1)
}
// EpochFirstBlock returns the block number of the first block of an epoch.
// TODO: instead of using fixed epoch schedules, determine the first block by epoch changes.
func EpochFirstBlock(epoch *big.Int) *big.Int {
if epoch.Cmp(big.NewInt(0)) == 0 {
return big.NewInt(0)
if epoch.Cmp(big.NewInt(GenesisEpoch)) == 0 {
return big.NewInt(GenesisEpoch)
}
return big.NewInt(int64(ShardingSchedule.EpochLastBlock(epoch.Uint64()-1) + 1))
}
// IsEpochLastBlock returns whether this block is the last block of an epoch.
func IsEpochLastBlock(block *types.Block) bool {
return ShardingSchedule.IsLastBlock(block.NumberU64())
}
// IsEpochLastBlockByHeader returns whether this block is the last block of an epoch
// given block header
func IsEpochLastBlockByHeader(header *block.Header) bool {
return ShardingSchedule.IsLastBlock(header.Number().Uint64())
return big.NewInt(int64(shard.Schedule.EpochLastBlock(epoch.Uint64()-1) + 1))
}
func (bc *BlockChain) getProcInterrupt() bool {
@ -1083,7 +1073,7 @@ func (bc *BlockChain) WriteBlockWithState(block *types.Block, receipts []*types.
epoch := block.Header().Epoch()
if bc.chainConfig.IsCrossTx(block.Epoch()) {
shardingConfig := ShardingSchedule.InstanceForEpoch(epoch)
shardingConfig := shard.Schedule.InstanceForEpoch(epoch)
shardNum := int(shardingConfig.NumShards())
for i := 0; i < shardNum; i++ {
if i == int(block.ShardID()) {
@ -1943,7 +1933,18 @@ func (bc *BlockChain) GetShardState(epoch *big.Int) (shard.State, error) {
if err == nil { // TODO ek – distinguish ErrNotFound
return shardState, err
}
shardState, err = CalculateNewShardState(bc, epoch)
if epoch.Cmp(big.NewInt(GenesisEpoch)) == 0 {
shardState, err = committee.WithStakingEnabled.ReadFromComputation(
big.NewInt(GenesisEpoch), *bc.Config(), nil,
)
} else {
prevEpoch := new(big.Int).Sub(epoch, common.Big1)
shardState, err = committee.WithStakingEnabled.ReadFromChain(
prevEpoch, bc,
)
}
if err != nil {
return nil, err
}
@ -2164,7 +2165,7 @@ func (bc *BlockChain) CXMerkleProof(shardID uint32, block *types.Block) (*types.
}
epoch := block.Header().Epoch()
shardingConfig := ShardingSchedule.InstanceForEpoch(epoch)
shardingConfig := shard.Schedule.InstanceForEpoch(epoch)
shardNum := int(shardingConfig.NumShards())
for i := 0; i < shardNum; i++ {
@ -2384,7 +2385,7 @@ func (bc *BlockChain) CurrentValidatorAddresses() []common.Address {
if err != nil {
continue
}
epoch := ShardingSchedule.CalcEpochNumber(val.CreationHeight.Uint64())
epoch := shard.Schedule.CalcEpochNumber(val.CreationHeight.Uint64())
if epoch.Cmp(currentEpoch) >= 0 {
// wait for next epoch
continue

@ -7,6 +7,7 @@ import (
blockfactory "github.com/harmony-one/harmony/block/factory"
"github.com/harmony-one/harmony/core/types"
shardingconfig "github.com/harmony-one/harmony/internal/configs/sharding"
"github.com/harmony-one/harmony/shard"
)
func TestIsEpochBlock(t *testing.T) {
@ -58,7 +59,7 @@ func TestIsEpochBlock(t *testing.T) {
},
}
for i, test := range tests {
ShardingSchedule = test.schedule
shard.Schedule = test.schedule
r := IsEpochBlock(test.block)
if r != test.expected {
t.Errorf("index: %v, expected: %v, got: %v\n", i, test.expected, r)

@ -46,6 +46,11 @@ import (
var errGenesisNoConfig = errors.New("genesis has no chain configuration")
const (
// GenesisEpoch is the number of the genesis epoch.
GenesisEpoch = 0
)
// Genesis specifies the header fields, state of a genesis block. It also defines hard
// fork switch-over blocks through the chain configuration.
type Genesis struct {

@ -1,259 +0,0 @@
package core
import (
"encoding/hex"
"errors"
"math/big"
"math/rand"
"sort"
"github.com/ethereum/go-ethereum/common"
"github.com/harmony-one/bls/ffi/go/bls"
common2 "github.com/harmony-one/harmony/internal/common"
shardingconfig "github.com/harmony-one/harmony/internal/configs/sharding"
"github.com/harmony-one/harmony/internal/ctxerror"
"github.com/harmony-one/harmony/internal/utils"
"github.com/harmony-one/harmony/shard"
)
const (
// GenesisEpoch is the number of the genesis epoch.
GenesisEpoch = 0
// CuckooRate is the percentage of nodes getting reshuffled in the second step of cuckoo resharding.
CuckooRate = 0.1
)
// ShardingState is data structure hold the sharding state
type ShardingState struct {
epoch uint64 // current epoch
rnd uint64 // random seed for resharding
numShards int // TODO ek – equal to len(shardState); remove this
shardState shard.State
}
// sortedCommitteeBySize will sort shards by size
// Suppose there are N shards, the first N/2 larger shards are called active committees
// the rest N/2 smaller committees are called inactive committees
// actually they are all just normal shards
// TODO: sort the committee weighted by total staking instead of shard size
func (ss *ShardingState) sortCommitteeBySize() {
sort.Slice(ss.shardState, func(i, j int) bool {
return len(ss.shardState[i].NodeList) > len(ss.shardState[j].NodeList)
})
}
// assignNewNodes add new nodes into the N/2 active committees evenly
func (ss *ShardingState) assignNewNodes(newNodeList []shard.NodeID) {
ss.sortCommitteeBySize()
numActiveShards := ss.numShards / 2
Shuffle(newNodeList)
for i, nid := range newNodeList {
id := 0
if numActiveShards > 0 {
id = i % numActiveShards
}
if id < len(ss.shardState) {
ss.shardState[id].NodeList = append(ss.shardState[id].NodeList, nid)
} else {
utils.Logger().Error().Int("id", id).Int("shardState Count", len(ss.shardState)).Msg("assignNewNodes index out of range")
}
}
}
// cuckooResharding uses cuckoo rule to reshard X% of active committee(shards) into inactive committee(shards)
func (ss *ShardingState) cuckooResharding(percent float64) {
numActiveShards := ss.numShards / 2
kickedNodes := []shard.NodeID{}
for i := range ss.shardState {
if i >= numActiveShards {
break
}
numKicked := int(percent * float64(len(ss.shardState[i].NodeList)))
if numKicked == 0 {
numKicked++ // At least kick one node out
}
length := len(ss.shardState[i].NodeList)
if length-numKicked <= 0 {
continue // Never empty a shard
}
tmp := ss.shardState[i].NodeList[length-numKicked:]
kickedNodes = append(kickedNodes, tmp...)
ss.shardState[i].NodeList = ss.shardState[i].NodeList[:length-numKicked]
}
Shuffle(kickedNodes)
numInactiveShards := ss.numShards - numActiveShards
for i, nid := range kickedNodes {
id := numActiveShards
if numInactiveShards > 0 {
id += i % numInactiveShards
}
ss.shardState[id].NodeList = append(ss.shardState[id].NodeList, nid)
}
}
// Reshard will first add new nodes into shards, then use cuckoo rule to reshard to get new shard state
func (ss *ShardingState) Reshard(newNodeList []shard.NodeID, percent float64) {
rand.Seed(int64(ss.rnd))
ss.sortCommitteeBySize()
// Take out and preserve leaders
leaders := []shard.NodeID{}
for i := 0; i < ss.numShards; i++ {
if len(ss.shardState[i].NodeList) > 0 {
leaders = append(leaders, ss.shardState[i].NodeList[0])
ss.shardState[i].NodeList = ss.shardState[i].NodeList[1:]
// Also shuffle the rest of the nodes
Shuffle(ss.shardState[i].NodeList)
}
}
ss.assignNewNodes(newNodeList)
ss.cuckooResharding(percent)
// Put leader back
if len(leaders) < ss.numShards {
utils.Logger().Error().Msg("Not enough leaders to assign to shards")
}
for i := 0; i < ss.numShards; i++ {
ss.shardState[i].NodeList = append([]shard.NodeID{leaders[i]}, ss.shardState[i].NodeList...)
}
}
// Shuffle will shuffle the list with result uniquely determined by seed, assuming there is no repeat items in the list
func Shuffle(list []shard.NodeID) {
// Sort to make sure everyone will generate the same with the same rand seed.
sort.Slice(list, func(i, j int) bool {
return shard.CompareNodeIDByBLSKey(list[i], list[j]) == -1
})
rand.Shuffle(len(list), func(i, j int) {
list[i], list[j] = list[j], list[i]
})
}
// GetEpochFromBlockNumber calculates the epoch number the block belongs to
func GetEpochFromBlockNumber(blockNumber uint64) uint64 {
return ShardingSchedule.CalcEpochNumber(blockNumber).Uint64()
}
// GetShardingStateFromBlockChain will retrieve random seed and shard map from beacon chain for given a epoch
func GetShardingStateFromBlockChain(bc *BlockChain, epoch *big.Int) (*ShardingState, error) {
if bc == nil {
return nil, errors.New("no blockchain is supplied to get shard state")
}
shardState, err := bc.ReadShardState(epoch)
if err != nil {
return nil, err
}
shardState = shardState.DeepCopy()
// TODO(RJ,HB): use real randomness for resharding
//blockNumber := GetBlockNumberFromEpoch(epoch.Uint64())
//rndSeedBytes := bc.GetVdfByNumber(blockNumber)
rndSeed := uint64(0)
return &ShardingState{epoch: epoch.Uint64(), rnd: rndSeed, shardState: shardState, numShards: len(shardState)}, nil
}
// CalculateNewShardState get sharding state from previous epoch and calculate sharding state for new epoch
func CalculateNewShardState(bc *BlockChain, epoch *big.Int) (shard.State, error) {
if epoch.Cmp(big.NewInt(GenesisEpoch)) == 0 {
return CalculateInitShardState(), nil
}
prevEpoch := new(big.Int).Sub(epoch, common.Big1)
ss, err := GetShardingStateFromBlockChain(bc, prevEpoch)
if err != nil {
return nil, ctxerror.New("cannot retrieve previous sharding state").
WithCause(err)
}
utils.Logger().Info().Float64("percentage", CuckooRate).Msg("Cuckoo Rate")
return ss.shardState, nil
}
// TODO ek – shardingSchedule should really be part of a general-purpose network
// configuration. We are OK for the time being,
// until the day we should let one node process join multiple networks.
// ShardingSchedule is the sharding configuration schedule.
// Depends on the type of the network. Defaults to the mainnet schedule.
var ShardingSchedule shardingconfig.Schedule = shardingconfig.MainnetSchedule
// CalculateInitShardState returns the initial shard state at genesis.
func CalculateInitShardState() shard.State {
return CalculateShardState(big.NewInt(GenesisEpoch))
}
// CalculateShardState returns the shard state based on epoch number
// This api for getting shard state is what should be used to get shard state regardless of
// current chain dependency (ex. getting shard state from block header received during cross-shard transaction)
func CalculateShardState(epoch *big.Int) shard.State {
utils.Logger().Info().Int64("epoch", epoch.Int64()).Msg("Get Shard State of Epoch.")
shardingConfig := ShardingSchedule.InstanceForEpoch(epoch)
shardNum := int(shardingConfig.NumShards())
shardHarmonyNodes := shardingConfig.NumHarmonyOperatedNodesPerShard()
shardSize := shardingConfig.NumNodesPerShard()
hmyAccounts := shardingConfig.HmyAccounts()
fnAccounts := shardingConfig.FnAccounts()
shardState := shard.State{}
for i := 0; i < shardNum; i++ {
com := shard.Committee{ShardID: uint32(i)}
for j := 0; j < shardHarmonyNodes; j++ {
index := i + j*shardNum // The initial account to use for genesis nodes
pub := &bls.PublicKey{}
pub.DeserializeHexStr(hmyAccounts[index].BlsPublicKey)
pubKey := shard.BlsPublicKey{}
pubKey.FromLibBLSPublicKey(pub)
// TODO: directly read address for bls too
curNodeID := shard.NodeID{
EcdsaAddress: common2.ParseAddr(hmyAccounts[index].Address),
BlsPublicKey: pubKey,
}
com.NodeList = append(com.NodeList, curNodeID)
}
// add FN runner's key
for j := shardHarmonyNodes; j < shardSize; j++ {
index := i + (j-shardHarmonyNodes)*shardNum
pub := &bls.PublicKey{}
pub.DeserializeHexStr(fnAccounts[index].BlsPublicKey)
pubKey := shard.BlsPublicKey{}
pubKey.FromLibBLSPublicKey(pub)
// TODO: directly read address for bls too
curNodeID := shard.NodeID{
EcdsaAddress: common2.ParseAddr(fnAccounts[index].Address),
BlsPublicKey: pubKey,
}
com.NodeList = append(com.NodeList, curNodeID)
}
shardState = append(shardState, com)
}
return shardState
}
// CalculatePublicKeys returns the publickeys given epoch and shardID
func CalculatePublicKeys(epoch *big.Int, shardID uint32) []*bls.PublicKey {
shardState := CalculateShardState(epoch)
// Update validator public keys
committee := shardState.FindCommitteeByID(shardID)
if committee == nil {
utils.Logger().Warn().Uint32("shardID", shardID).Uint64("epoch", epoch.Uint64()).Msg("Cannot find committee")
return nil
}
pubKeys := []*bls.PublicKey{}
for _, node := range committee.NodeList {
pubKey := &bls.PublicKey{}
pubKeyBytes := node.BlsPublicKey[:]
err := pubKey.Deserialize(pubKeyBytes)
if err != nil {
utils.Logger().Warn().Str("pubKeyBytes", hex.EncodeToString(pubKeyBytes)).Msg("Cannot Deserialize pubKey")
return nil
}
pubKeys = append(pubKeys, pubKey)
}
return pubKeys
}

@ -1,12 +0,0 @@
## Resharding
In current design, the epoch is defined to be fixed length, the epoch length is a constant parameter BlocksPerEpoch. In future, it will be dynamically adjustable according to security parameter. During the epoch transition, suppose there are N shards, we sort the shards according to the size of active nodes (that had staking for next epoch). The first N/2 larger shards will be called active committees, and the last N/2 smaller shards will be called inactive committees. Don't be confused by
the name, they are all normal shards with same function.
All the information about sharding will be stored in BeaconChain. A sharding state is defined as a map which maps each NodeID to the ShardID the node belongs to. Every node will have a unique NodeID and be mapped to one ShardID. At the beginning of a new epoch, the BeaconChain leader will propose a new block containing the new sharding state, the new sharding state is uniquely determined by the randomness generated by distributed randomness protocol. During the consensus process, all the validators will perform the same calculation and verify the proposed sharding state is valid. After consensus is reached, each node will write the new sharding state into the block. This block is called epoch block. In current code, it's the first block of each epoch in BeaconChain.
The main function of resharding is CalculcateNewShardState. It will take 3 inputs: newNodeList, oldShardState, randomSeed and output newShardState.
The newNodeList will be retrieved from BeaconChain staking transaction during the previous epoch. The randomSeed and oldShardState is stored in previous epoch block. It should be noticed that the randomSeed generation currently is mocked. After the distributed randomness protocol(drand) is ready, the drand service will generate the random seed for resharding.
The resharding process is as follows: we first get newNodeList from staking transactions from previous epoch and assign the new nodes evenly into the N/2 active committees. Then, we kick out X% of nodes from each active committees and put these kicked out nodes into inactive committees evenly. The percentage X roughly equals to the percentage of new nodes into active committee in order to balance the committee size.

@ -1,149 +0,0 @@
package core
import (
"fmt"
"math/rand"
"strconv"
"testing"
"github.com/ethereum/go-ethereum/common"
"github.com/harmony-one/harmony/shard"
"github.com/stretchr/testify/assert"
)
var (
blsPubKey1 = [48]byte{}
blsPubKey2 = [48]byte{}
blsPubKey3 = [48]byte{}
blsPubKey4 = [48]byte{}
blsPubKey5 = [48]byte{}
blsPubKey6 = [48]byte{}
blsPubKey7 = [48]byte{}
blsPubKey8 = [48]byte{}
blsPubKey9 = [48]byte{}
blsPubKey10 = [48]byte{}
)
func init() {
copy(blsPubKey1[:], []byte("random key 1"))
copy(blsPubKey2[:], []byte("random key 2"))
copy(blsPubKey3[:], []byte("random key 3"))
copy(blsPubKey4[:], []byte("random key 4"))
copy(blsPubKey5[:], []byte("random key 5"))
copy(blsPubKey6[:], []byte("random key 6"))
copy(blsPubKey7[:], []byte("random key 7"))
copy(blsPubKey8[:], []byte("random key 8"))
copy(blsPubKey9[:], []byte("random key 9"))
copy(blsPubKey10[:], []byte("random key 10"))
}
func fakeGetInitShardState(numberOfShards, numOfNodes int) shard.State {
rand.Seed(int64(42))
shardState := shard.State{}
for i := 0; i < numberOfShards; i++ {
sid := uint32(i)
com := shard.Committee{ShardID: sid}
for j := 0; j < numOfNodes; j++ {
nid := strconv.Itoa(int(rand.Int63()))
blsPubKey := [48]byte{}
copy(blsPubKey1[:], []byte(nid))
com.NodeList = append(com.NodeList, shard.NodeID{
EcdsaAddress: common.BytesToAddress([]byte(nid)),
BlsPublicKey: blsPubKey,
})
}
shardState = append(shardState, com)
}
return shardState
}
func fakeNewNodeList(seed int64) []shard.NodeID {
rand.Seed(seed)
numNewNodes := rand.Intn(10)
nodeList := []shard.NodeID{}
for i := 0; i < numNewNodes; i++ {
nid := strconv.Itoa(int(rand.Int63()))
blsPubKey := [48]byte{}
copy(blsPubKey1[:], []byte(nid))
nodeList = append(nodeList, shard.NodeID{
EcdsaAddress: common.BytesToAddress([]byte(nid)),
BlsPublicKey: blsPubKey,
})
}
return nodeList
}
func TestFakeNewNodeList(t *testing.T) {
nodeList := fakeNewNodeList(42)
fmt.Println("newNodeList: ", nodeList)
}
func TestShuffle(t *testing.T) {
nodeList := []shard.NodeID{
{EcdsaAddress: common.Address{0x12}, BlsPublicKey: blsPubKey1},
{EcdsaAddress: common.Address{0x22}, BlsPublicKey: blsPubKey2},
{EcdsaAddress: common.Address{0x32}, BlsPublicKey: blsPubKey3},
{EcdsaAddress: common.Address{0x42}, BlsPublicKey: blsPubKey4},
{EcdsaAddress: common.Address{0x52}, BlsPublicKey: blsPubKey5},
{EcdsaAddress: common.Address{0x62}, BlsPublicKey: blsPubKey6},
{EcdsaAddress: common.Address{0x72}, BlsPublicKey: blsPubKey7},
{EcdsaAddress: common.Address{0x82}, BlsPublicKey: blsPubKey8},
{EcdsaAddress: common.Address{0x92}, BlsPublicKey: blsPubKey9},
{EcdsaAddress: common.Address{0x02}, BlsPublicKey: blsPubKey10},
}
cpList := []shard.NodeID{}
cpList = append(cpList, nodeList...)
Shuffle(nodeList)
cnt := 0
for i := 0; i < 10; i++ {
if cpList[i] == nodeList[i] {
cnt++
}
}
if cnt == 10 {
t.Error("Shuffle list is the same as original list")
}
return
}
func TestSortCommitteeBySize(t *testing.T) {
shardState := fakeGetInitShardState(6, 10)
ss := &ShardingState{epoch: 1, rnd: 42, shardState: shardState, numShards: len(shardState)}
ss.sortCommitteeBySize()
for i := 0; i < ss.numShards-1; i++ {
assert.Equal(t, true, len(ss.shardState[i].NodeList) >= len(ss.shardState[i+1].NodeList))
}
}
func TestUpdateShardState(t *testing.T) {
shardState := fakeGetInitShardState(6, 10)
ss := &ShardingState{epoch: 1, rnd: 42, shardState: shardState, numShards: len(shardState)}
newNodeList := []shard.NodeID{
{EcdsaAddress: common.Address{0x12}, BlsPublicKey: blsPubKey1},
{EcdsaAddress: common.Address{0x22}, BlsPublicKey: blsPubKey2},
{EcdsaAddress: common.Address{0x32}, BlsPublicKey: blsPubKey3},
{EcdsaAddress: common.Address{0x42}, BlsPublicKey: blsPubKey4},
{EcdsaAddress: common.Address{0x52}, BlsPublicKey: blsPubKey5},
{EcdsaAddress: common.Address{0x62}, BlsPublicKey: blsPubKey6},
}
ss.Reshard(newNodeList, 0.2)
assert.Equal(t, 6, ss.numShards)
}
func TestAssignNewNodes(t *testing.T) {
shardState := fakeGetInitShardState(2, 2)
ss := &ShardingState{epoch: 1, rnd: 42, shardState: shardState, numShards: len(shardState)}
newNodes := []shard.NodeID{
{EcdsaAddress: common.Address{0x12}, BlsPublicKey: blsPubKey1},
{EcdsaAddress: common.Address{0x22}, BlsPublicKey: blsPubKey2},
{EcdsaAddress: common.Address{0x32}, BlsPublicKey: blsPubKey3},
}
ss.assignNewNodes(newNodes)
assert.Equal(t, 2, ss.numShards)
assert.Equal(t, 5, len(ss.shardState[0].NodeList))
}

@ -30,6 +30,7 @@ import (
"github.com/harmony-one/harmony/internal/ctxerror"
"github.com/harmony-one/harmony/internal/params"
"github.com/harmony-one/harmony/internal/utils"
"github.com/harmony-one/harmony/shard"
staking "github.com/harmony-one/harmony/staking/types"
)
@ -122,7 +123,7 @@ func getTransactionType(config *params.ChainConfig, header *block.Header, tx *ty
if header.ShardID() == tx.ShardID() && (!config.IsCrossTx(header.Epoch()) || tx.ShardID() == tx.ToShardID()) {
return types.SameShardTx
}
numShards := ShardingSchedule.InstanceForEpoch(header.Epoch()).NumShards()
numShards := shard.Schedule.InstanceForEpoch(header.Epoch()).NumShards()
// Assuming here all the shards are consecutive from 0 to n-1, n is total number of shards
if tx.ShardID() != tx.ToShardID() && header.ShardID() == tx.ShardID() && tx.ToShardID() < numShards {
return types.SubtractionOnly

@ -1,10 +0,0 @@
package values
const (
// BeaconChainShardID is the ShardID of the BeaconChain
BeaconChainShardID = 0
// VotingPowerReduceBlockThreshold roughly corresponds to 3 hours
VotingPowerReduceBlockThreshold = 1350
// VotingPowerFullReduce roughly corresponds to 12 hours
VotingPowerFullReduce = 4 * VotingPowerReduceBlockThreshold
)

@ -4,17 +4,17 @@ import (
"bytes"
"time"
"github.com/harmony-one/harmony/crypto/bls"
protobuf "github.com/golang/protobuf/proto"
msg_pb "github.com/harmony-one/harmony/api/proto/message"
"github.com/harmony-one/harmony/core"
"github.com/harmony-one/harmony/core/types"
"github.com/harmony-one/harmony/crypto/bls"
"github.com/harmony-one/harmony/crypto/vdf"
"github.com/harmony-one/harmony/crypto/vrf/p256"
nodeconfig "github.com/harmony-one/harmony/internal/configs/node"
"github.com/harmony-one/harmony/internal/utils"
"github.com/harmony-one/harmony/p2p/host"
"github.com/harmony-one/harmony/shard"
)
const (
@ -30,7 +30,7 @@ func (dRand *DRand) WaitForEpochBlock(blockChannel chan *types.Block, stopChan c
default:
// keep waiting for epoch block
newBlock := <-blockChannel
if core.IsEpochLastBlock(newBlock) {
if shard.Schedule.IsLastBlock(newBlock.Number().Uint64()) {
dRand.init(newBlock)
}
// TODO: use real vrf

@ -8,12 +8,12 @@ import (
"github.com/harmony-one/bls/ffi/go/bls"
"github.com/harmony-one/harmony/block"
"github.com/harmony-one/harmony/consensus/engine"
"github.com/harmony-one/harmony/core"
"github.com/harmony-one/harmony/core/state"
"github.com/harmony-one/harmony/core/types"
"github.com/harmony-one/harmony/internal/ctxerror"
"github.com/harmony-one/harmony/internal/utils"
"github.com/harmony-one/harmony/shard"
"github.com/harmony-one/harmony/shard/committee"
staking "github.com/harmony-one/harmony/staking/types"
"github.com/pkg/errors"
"golang.org/x/crypto/sha3"
@ -116,6 +116,7 @@ func (e *engineImpl) VerifySeal(chain engine.ChainReader, header *block.Header)
return nil
}
publicKeys, err := ReadPublicKeysFromLastBlock(chain, header)
if err != nil {
return ctxerror.New("[VerifySeal] Cannot retrieve publickeys from last block").WithCause(err)
}
@ -166,7 +167,7 @@ func (e *engineImpl) Finalize(
func QuorumForBlock(chain engine.ChainReader, h *block.Header, reCalculate bool) (quorum int, err error) {
var ss shard.State
if reCalculate {
ss = core.CalculateShardState(h.Epoch())
ss, _ = committee.WithStakingEnabled.ReadFromComputation(h.Epoch(), *chain.Config(), nil)
} else {
ss, err = chain.ReadShardState(h.Epoch())
if err != nil {
@ -225,7 +226,7 @@ func GetPublicKeys(chain engine.ChainReader, header *block.Header, reCalculate b
var shardState shard.State
var err error
if reCalculate {
shardState = core.CalculateShardState(header.Epoch())
shardState, _ = committee.WithStakingEnabled.ReadFromComputation(header.Epoch(), *chain.Config(), nil)
} else {
shardState, err = chain.ReadShardState(header.Epoch())
if err != nil {

@ -152,8 +152,11 @@ func (ls localnetSchedule) GetShardingStructure(numShard, shardID int) []map[str
return res
}
var localnetReshardingEpoch = []*big.Int{big.NewInt(0), big.NewInt(localnetV1Epoch), big.NewInt(localnetV2Epoch)}
var localnetV0 = MustNewInstance(2, 7, 5, genesis.LocalHarmonyAccounts, genesis.LocalFnAccounts, localnetReshardingEpoch)
var localnetV1 = MustNewInstance(2, 8, 5, genesis.LocalHarmonyAccountsV1, genesis.LocalFnAccountsV1, localnetReshardingEpoch)
var localnetV2 = MustNewInstance(2, 9, 6, genesis.LocalHarmonyAccountsV2, genesis.LocalFnAccountsV2, localnetReshardingEpoch)
var (
localnetReshardingEpoch = []*big.Int{
big.NewInt(0), big.NewInt(localnetV1Epoch), big.NewInt(localnetV2Epoch),
}
localnetV0 = MustNewInstance(2, 7, 5, genesis.LocalHarmonyAccounts, genesis.LocalFnAccounts, localnetReshardingEpoch)
localnetV1 = MustNewInstance(2, 8, 5, genesis.LocalHarmonyAccountsV1, genesis.LocalFnAccountsV1, localnetReshardingEpoch)
localnetV2 = MustNewInstance(2, 9, 6, genesis.LocalHarmonyAccountsV2, genesis.LocalFnAccountsV2, localnetReshardingEpoch)
)

@ -20,6 +20,7 @@ import (
internal_bls "github.com/harmony-one/harmony/crypto/bls"
internal_common "github.com/harmony-one/harmony/internal/common"
"github.com/harmony-one/harmony/internal/utils"
"github.com/harmony-one/harmony/shard"
)
const (
@ -359,10 +360,10 @@ func (s *PublicBlockChainAPI) GetDelegatorsInformation(ctx context.Context, addr
func (s *PublicBlockChainAPI) GetShardingStructure(ctx context.Context) ([]map[string]interface{}, error) {
// Get header and number of shards.
epoch := s.GetEpoch(ctx)
numShard := core.ShardingSchedule.InstanceForEpoch(big.NewInt(int64(epoch))).NumShards()
numShard := shard.Schedule.InstanceForEpoch(big.NewInt(int64(epoch))).NumShards()
// Return shareding structure for each case.
return core.ShardingSchedule.GetShardingStructure(int(numShard), int(s.b.GetShardID())), nil
return shard.Schedule.GetShardingStructure(int(numShard), int(s.b.GetShardID())), nil
}
// GetShardID returns shard ID of the requested node.

@ -36,7 +36,7 @@ var (
ChainID: TestnetChainID,
CrossTxEpoch: big.NewInt(0),
CrossLinkEpoch: big.NewInt(0),
StakingEpoch: big.NewInt(0),
StakingEpoch: EpochTBD,
EIP155Epoch: big.NewInt(0),
S3Epoch: big.NewInt(0),
}

@ -20,7 +20,6 @@ import (
"github.com/harmony-one/harmony/contracts"
"github.com/harmony-one/harmony/core"
"github.com/harmony-one/harmony/core/types"
"github.com/harmony-one/harmony/core/values"
"github.com/harmony-one/harmony/drand"
"github.com/harmony-one/harmony/internal/chain"
nodeconfig "github.com/harmony-one/harmony/internal/configs/node"
@ -33,6 +32,7 @@ import (
"github.com/harmony-one/harmony/p2p"
p2p_host "github.com/harmony-one/harmony/p2p/host"
"github.com/harmony-one/harmony/shard"
"github.com/harmony-one/harmony/shard/committee"
staking "github.com/harmony-one/harmony/staking/types"
)
@ -277,7 +277,7 @@ func (node *Node) tryBroadcast(tx *types.Transaction) {
// Add new transactions to the pending transaction list.
func (node *Node) addPendingTransactions(newTxs types.Transactions) {
txPoolLimit := core.ShardingSchedule.MaxTxPoolSizeLimit()
txPoolLimit := shard.Schedule.MaxTxPoolSizeLimit()
node.pendingTxMutex.Lock()
for _, tx := range newTxs {
if _, ok := node.pendingTransactions[tx.Hash()]; !ok {
@ -293,7 +293,7 @@ func (node *Node) addPendingTransactions(newTxs types.Transactions) {
// Add new staking transactions to the pending staking transaction list.
func (node *Node) addPendingStakingTransactions(newStakingTxs staking.StakingTransactions) {
txPoolLimit := core.ShardingSchedule.MaxTxPoolSizeLimit()
txPoolLimit := shard.Schedule.MaxTxPoolSizeLimit()
node.pendingStakingTxMutex.Lock()
for _, tx := range newStakingTxs {
if _, ok := node.pendingStakingTransactions[tx.Hash()]; !ok {
@ -350,7 +350,7 @@ func (node *Node) AddPendingReceipts(receipts *types.CXReceiptsProof) {
// Take out a subset of valid transactions from the pending transaction list
// Note the pending transaction list will then contain the rest of the txs
func (node *Node) getTransactionsForNewBlock(coinbase common.Address) (types.Transactions, staking.StakingTransactions) {
txsThrottleConfig := core.ShardingSchedule.TxsThrottleConfig()
txsThrottleConfig := shard.Schedule.TxsThrottleConfig()
// the next block number to be added in consensus protocol, which is always one more than current chain header block
newBlockNum := node.Blockchain().CurrentBlock().NumberU64() + 1
@ -483,7 +483,8 @@ func New(host p2p.Host, consensusObj *consensus.Consensus, chainDBFactory shardc
node.chainConfig = chainConfig
collection := shardchain.NewCollection(
chainDBFactory, &genesisInitializer{&node}, chain.Engine, &chainConfig)
chainDBFactory, &genesisInitializer{&node}, chain.Engine, &chainConfig,
)
if isArchival {
collection.DisableCache()
}
@ -505,7 +506,7 @@ func New(host p2p.Host, consensusObj *consensus.Consensus, chainDBFactory shardc
node.CxPool = core.NewCxPool(core.CxPoolSize)
node.Worker = worker.New(node.Blockchain().Config(), blockchain, chain.Engine)
if node.Blockchain().ShardID() != values.BeaconChainShardID {
if node.Blockchain().ShardID() != shard.BeaconChainShardID {
node.BeaconWorker = worker.New(node.Beaconchain().Config(), beaconChain, chain.Engine)
}
@ -545,44 +546,42 @@ func New(host p2p.Host, consensusObj *consensus.Consensus, chainDBFactory shardc
// Setup initial state of syncing.
node.peerRegistrationRecord = make(map[string]*syncConfig)
node.startConsensus = make(chan struct{})
go node.bootstrapConsensus()
return &node
}
// CalculateInitShardState initialize shard state from latest epoch and update committee pub keys for consensus and drand
func (node *Node) CalculateInitShardState() (err error) {
// InitConsensusWithValidators initialize shard state from latest epoch and update committee pub
// keys for consensus and drand
func (node *Node) InitConsensusWithValidators() (err error) {
if node.Consensus == nil {
return ctxerror.New("[CalculateInitShardState] consenus is nil; Cannot figure out shardID")
return ctxerror.New("[InitConsensusWithValidators] consenus is nil; Cannot figure out shardID")
}
shardID := node.Consensus.ShardID
// Get genesis epoch shard state from chain
blockNum := node.Blockchain().CurrentBlock().NumberU64()
node.Consensus.SetMode(consensus.Listening)
epoch := core.ShardingSchedule.CalcEpochNumber(blockNum)
epoch := shard.Schedule.CalcEpochNumber(blockNum)
utils.Logger().Info().
Uint64("blockNum", blockNum).
Uint32("shardID", shardID).
Uint64("epoch", epoch.Uint64()).
Msg("[CalculateInitShardState] Try To Get PublicKeys from database")
pubKeys := core.CalculatePublicKeys(epoch, shardID)
Msg("[InitConsensusWithValidators] Try To Get PublicKeys")
_, pubKeys := committee.WithStakingEnabled.ComputePublicKeys(
epoch, node.Consensus.ChainReader, int(shardID),
)
if len(pubKeys) == 0 {
return ctxerror.New(
"[CalculateInitShardState] PublicKeys is Empty, Cannot update public keys",
"[InitConsensusWithValidators] PublicKeys is Empty, Cannot update public keys",
"shardID", shardID,
"blockNum", blockNum)
}
for _, key := range pubKeys {
if key.IsEqual(node.Consensus.PubKey) {
for i := range pubKeys {
if pubKeys[i].IsEqual(node.Consensus.PubKey) {
utils.Logger().Info().
Uint64("blockNum", blockNum).
Int("numPubKeys", len(pubKeys)).
Msg("[CalculateInitShardState] Successfully updated public keys")
Msg("[InitConsensusWithValidators] Successfully updated public keys")
node.Consensus.UpdatePublicKeys(pubKeys)
node.Consensus.SetMode(consensus.Normal)
return nil

@ -4,13 +4,9 @@ import (
"encoding/binary"
"errors"
"github.com/harmony-one/harmony/p2p/host"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/rlp"
"github.com/harmony-one/bls/ffi/go/bls"
proto_node "github.com/harmony-one/harmony/api/proto/node"
"github.com/harmony-one/harmony/block"
"github.com/harmony-one/harmony/core"
@ -19,6 +15,8 @@ import (
nodeconfig "github.com/harmony-one/harmony/internal/configs/node"
"github.com/harmony-one/harmony/internal/ctxerror"
"github.com/harmony-one/harmony/internal/utils"
"github.com/harmony-one/harmony/p2p/host"
"github.com/harmony-one/harmony/shard"
)
// BroadcastCXReceipts broadcasts cross shard receipts to correspoding
@ -38,7 +36,7 @@ func (node *Node) BroadcastCXReceipts(newBlock *types.Block, lastCommits []byte)
//#### END Read payload data from committed msg
epoch := newBlock.Header().Epoch()
shardingConfig := core.ShardingSchedule.InstanceForEpoch(epoch)
shardingConfig := shard.Schedule.InstanceForEpoch(epoch)
shardNum := int(shardingConfig.NumShards())
myShardID := node.Consensus.ShardID
utils.Logger().Info().Int("shardNum", shardNum).Uint32("myShardID", myShardID).Uint64("blockNum", newBlock.NumberU64()).Msg("[BroadcastCXReceipts]")
@ -345,7 +343,7 @@ func (node *Node) ProposeCrossLinkDataForBeaconchain() (types.CrossLinks, error)
Uint64("blockNum", node.Blockchain().CurrentBlock().NumberU64()+1).
Msg("Proposing cross links ...")
curBlock := node.Blockchain().CurrentBlock()
numShards := core.ShardingSchedule.InstanceForEpoch(curBlock.Header().Epoch()).NumShards()
numShards := shard.Schedule.InstanceForEpoch(curBlock.Header().Epoch()).NumShards()
shardCrossLinks := make([]types.CrossLinks, numShards)

@ -14,6 +14,7 @@ import (
"github.com/harmony-one/harmony/core"
"github.com/harmony-one/harmony/core/types"
"github.com/harmony-one/harmony/internal/utils"
"github.com/harmony-one/harmony/shard"
)
var once sync.Once
@ -107,7 +108,7 @@ func (node *Node) ExplorerMessageHandler(payload []byte) {
func (node *Node) AddNewBlockForExplorer(block *types.Block) {
utils.Logger().Debug().Uint64("blockHeight", block.NumberU64()).Msg("[Explorer] Adding new block for explorer node")
if err := node.AddNewBlock(block); err == nil {
if core.IsEpochLastBlock(block) {
if shard.Schedule.IsLastBlock(block.Number().Uint64()) {
node.Consensus.UpdateConsensusInformation()
}
// Clean up the blocks to avoid OOM.

@ -21,6 +21,7 @@ import (
"github.com/harmony-one/harmony/internal/params"
"github.com/harmony-one/harmony/internal/utils"
"github.com/harmony-one/harmony/shard"
"github.com/harmony-one/harmony/shard/committee"
)
const (
@ -41,8 +42,10 @@ type genesisInitializer struct {
// InitChainDB sets up a new genesis block in the database for the given shard.
func (gi *genesisInitializer) InitChainDB(db ethdb.Database, shardID uint32) error {
shardState := core.CalculateInitShardState()
if shardID != 0 {
shardState, _ := committee.WithStakingEnabled.ReadFromComputation(
big.NewInt(core.GenesisEpoch), gi.node.chainConfig, nil,
)
if shardID != shard.BeaconChainShardID {
// store only the local shard for shard chains
c := shardState.FindCommitteeByID(shardID)
if c == nil {

@ -5,11 +5,9 @@ import (
"context"
"math/big"
"math/rand"
"sync"
"sync/atomic"
"time"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/rlp"
"github.com/harmony-one/bls/ffi/go/bls"
@ -19,7 +17,6 @@ import (
proto_discovery "github.com/harmony-one/harmony/api/proto/discovery"
proto_node "github.com/harmony-one/harmony/api/proto/node"
"github.com/harmony-one/harmony/block"
"github.com/harmony-one/harmony/core"
"github.com/harmony-one/harmony/core/types"
nodeconfig "github.com/harmony-one/harmony/internal/configs/node"
"github.com/harmony-one/harmony/internal/ctxerror"
@ -325,7 +322,7 @@ func (node *Node) PostConsensusProcessing(newBlock *types.Block, commitSigAndBit
if node.NodeConfig.ShardID == 0 {
node.BroadcastNewBlock(newBlock)
}
if node.NodeConfig.ShardID != 0 && newBlock.Epoch().Cmp(node.Blockchain().Config().CrossLinkEpoch) >= 0 {
if node.NodeConfig.ShardID != shard.BeaconChainShardID && newBlock.Epoch().Cmp(node.Blockchain().Config().CrossLinkEpoch) >= 0 {
node.BroadcastCrossLinkHeader(newBlock)
}
node.BroadcastCXReceipts(newBlock, commitSigAndBitmap)
@ -348,7 +345,7 @@ func (node *Node) PostConsensusProcessing(newBlock *types.Block, commitSigAndBit
node.BroadcastMissingCXReceipts()
// Update consensus keys at last so the change of leader status doesn't mess up normal flow
if core.IsEpochLastBlock(newBlock) {
if shard.Schedule.IsLastBlock(newBlock.Number().Uint64()) {
node.Consensus.UpdateConsensusInformation()
}
@ -380,33 +377,7 @@ func (node *Node) PostConsensusProcessing(newBlock *types.Block, commitSigAndBit
// node.ConfirmedBlockChannel <- newBlock
// }()
//}
// TODO: enable staking
// TODO: update staking information once per epoch.
//node.UpdateStakingList(node.QueryStakeInfo())
//node.printStakingList()
}
// TODO: enable shard state update
//newBlockHeader := newBlock.Header()
//if newBlockHeader.ShardStateHash != (common.Hash{}) {
// if node.Consensus.ShardID == 0 {
// // TODO ek – this is a temp hack until beacon chain sync is fixed
// // End-of-epoch block on beacon chain; block's EpochState is the
// // master resharding table. Broadcast it to the network.
// if err := node.broadcastEpochShardState(newBlock); err != nil {
// e := ctxerror.New("cannot broadcast shard state").WithCause(err)
// ctxerror.Log15(utils.Logger().Error, e)
// }
// }
// shardState, err := newBlockHeader.CalculateShardState()
// if err != nil {
// e := ctxerror.New("cannot get shard state from header").WithCause(err)
// ctxerror.Log15(utils.Logger().Error, e)
// } else {
// node.transitionIntoNextEpoch(shardState)
// }
//}
}
}
}
@ -448,43 +419,6 @@ func (node *Node) AddNewBlock(newBlock *types.Block) error {
return err
}
type genesisNode struct {
ShardID uint32
MemberIndex int
NodeID shard.NodeID
}
var (
genesisCatalogOnce sync.Once
genesisNodeByStakingAddress = make(map[common.Address]*genesisNode)
genesisNodeByConsensusKey = make(map[shard.BlsPublicKey]*genesisNode)
)
func initGenesisCatalog() {
genesisShardState := core.CalculateInitShardState()
for _, committee := range genesisShardState {
for i, nodeID := range committee.NodeList {
genesisNode := &genesisNode{
ShardID: committee.ShardID,
MemberIndex: i,
NodeID: nodeID,
}
genesisNodeByStakingAddress[nodeID.EcdsaAddress] = genesisNode
genesisNodeByConsensusKey[nodeID.BlsPublicKey] = genesisNode
}
}
}
func getGenesisNodeByStakingAddress(address common.Address) *genesisNode {
genesisCatalogOnce.Do(initGenesisCatalog)
return genesisNodeByStakingAddress[address]
}
func getGenesisNodeByConsensusKey(key shard.BlsPublicKey) *genesisNode {
genesisCatalogOnce.Do(initGenesisCatalog)
return genesisNodeByConsensusKey[key]
}
func (node *Node) pingMessageHandler(msgPayload []byte, sender libp2p_peer.ID) int {
ping, err := proto_discovery.GetPingMessage(msgPayload)
if err != nil {

@ -6,12 +6,12 @@ import (
"github.com/ethereum/go-ethereum/common"
"github.com/harmony-one/harmony/consensus"
"github.com/harmony-one/harmony/consensus/quorum"
"github.com/harmony-one/harmony/core/values"
"github.com/harmony-one/harmony/crypto/bls"
nodeconfig "github.com/harmony-one/harmony/internal/configs/node"
"github.com/harmony-one/harmony/internal/utils"
"github.com/harmony-one/harmony/p2p"
"github.com/harmony-one/harmony/p2p/p2pimpl"
"github.com/harmony-one/harmony/shard"
)
func TestAddNewBlock(t *testing.T) {
@ -25,7 +25,7 @@ func TestAddNewBlock(t *testing.T) {
}
decider := quorum.NewDecider(quorum.SuperMajorityVote)
consensus, err := consensus.New(
host, values.BeaconChainShardID, leader, blsKey, decider,
host, shard.BeaconChainShardID, leader, blsKey, decider,
)
if err != nil {
t.Fatalf("Cannot craeate consensus: %v", err)
@ -58,7 +58,7 @@ func TestVerifyNewBlock(t *testing.T) {
}
decider := quorum.NewDecider(quorum.SuperMajorityVote)
consensus, err := consensus.New(
host, values.BeaconChainShardID, leader, blsKey, decider,
host, shard.BeaconChainShardID, leader, blsKey, decider,
)
if err != nil {
t.Fatalf("Cannot craeate consensus: %v", err)

@ -6,11 +6,11 @@ import (
"time"
"github.com/ethereum/go-ethereum/common"
"github.com/harmony-one/harmony/core"
"github.com/harmony-one/harmony/core/types"
"github.com/harmony-one/harmony/internal/ctxerror"
"github.com/harmony-one/harmony/internal/utils"
"github.com/harmony-one/harmony/shard"
"github.com/harmony-one/harmony/shard/committee"
)
// Constants of proposing a new block
@ -124,12 +124,13 @@ func (node *Node) proposeNewBlock() (*types.Block, error) {
}
func (node *Node) proposeShardStateWithoutBeaconSync(block *types.Block) shard.State {
if block == nil || !core.IsEpochLastBlock(block) {
if block == nil || !shard.Schedule.IsLastBlock(block.Number().Uint64()) {
return nil
}
nextEpoch := new(big.Int).Add(block.Header().Epoch(), common.Big1)
return core.CalculateShardState(nextEpoch)
shardState, _ := committee.WithStakingEnabled.ReadFromComputation(
new(big.Int).Add(block.Header().Epoch(), common.Big1), node.chainConfig, nil,
)
return shardState
}
func (node *Node) proposeShardState(block *types.Block) error {
@ -144,13 +145,15 @@ func (node *Node) proposeShardState(block *types.Block) error {
func (node *Node) proposeBeaconShardState(block *types.Block) error {
// TODO ek - replace this with variable epoch logic.
if !core.IsEpochLastBlock(block) {
if !shard.Schedule.IsLastBlock(block.Number().Uint64()) {
// We haven't reached the end of this epoch; don't propose yet.
return nil
}
nextEpoch := new(big.Int).Add(block.Header().Epoch(), common.Big1)
// TODO: add logic for EPoS
shardState, err := core.CalculateNewShardState(node.Blockchain(), nextEpoch)
// TODO Use ReadFromComputation
prevEpoch := new(big.Int).Sub(block.Header().Epoch(), common.Big1)
shardState, err := committee.WithStakingEnabled.ReadFromChain(
prevEpoch, node.Blockchain(),
)
if err != nil {
return err
}

@ -15,13 +15,13 @@ import (
"github.com/ethereum/go-ethereum/rlp"
"github.com/harmony-one/bls/ffi/go/bls"
proto_node "github.com/harmony-one/harmony/api/proto/node"
"github.com/harmony-one/harmony/core"
"github.com/harmony-one/harmony/core/types"
nodeconfig "github.com/harmony-one/harmony/internal/configs/node"
"github.com/harmony-one/harmony/internal/ctxerror"
"github.com/harmony-one/harmony/internal/utils"
"github.com/harmony-one/harmony/p2p/host"
"github.com/harmony-one/harmony/shard"
"github.com/harmony-one/harmony/shard/committee"
)
// validateNewShardState validate whether the new shard state root matches
@ -30,8 +30,8 @@ func (node *Node) validateNewShardState(block *types.Block) error {
header := block.Header()
if header.ShardStateHash() == (common.Hash{}) {
// No new shard state was proposed
if block.ShardID() == 0 {
if core.IsEpochLastBlock(block) {
if block.ShardID() == shard.BeaconChainShardID {
if shard.Schedule.IsLastBlock(block.Number().Uint64()) {
// TODO ek - invoke view change
return errors.New("beacon leader did not propose resharding")
}
@ -51,14 +51,17 @@ func (node *Node) validateNewShardState(block *types.Block) error {
return err
}
proposed := *shardState
if block.ShardID() == 0 {
if block.ShardID() == shard.BeaconChainShardID {
// Beacon validators independently recalculate the master state and
// compare it against the proposed copy.
nextEpoch := new(big.Int).Add(block.Header().Epoch(), common.Big1)
// TODO ek – this may be called from regular shards,
// for vetting beacon chain blocks received during block syncing.
// DRand may or or may not get in the way. Test this out.
expected, err := core.CalculateNewShardState(node.Blockchain(), nextEpoch)
expected, err := committee.WithStakingEnabled.ReadFromChain(
new(big.Int).Sub(block.Header().Epoch(), common.Big1),
node.Beaconchain(),
)
if err != nil {
return ctxerror.New("cannot calculate expected shard state").
WithCause(err)

@ -11,7 +11,6 @@ import (
proto_discovery "github.com/harmony-one/harmony/api/proto/discovery"
"github.com/harmony-one/harmony/consensus"
"github.com/harmony-one/harmony/consensus/quorum"
"github.com/harmony-one/harmony/core/values"
bls2 "github.com/harmony-one/harmony/crypto/bls"
"github.com/harmony-one/harmony/crypto/pki"
"github.com/harmony-one/harmony/drand"
@ -19,6 +18,7 @@ import (
"github.com/harmony-one/harmony/internal/utils"
"github.com/harmony-one/harmony/p2p"
"github.com/harmony-one/harmony/p2p/p2pimpl"
"github.com/harmony-one/harmony/shard"
"github.com/stretchr/testify/assert"
)
@ -35,7 +35,7 @@ func TestNewNode(t *testing.T) {
}
decider := quorum.NewDecider(quorum.SuperMajorityVote)
consensus, err := consensus.New(
host, values.BeaconChainShardID, leader, blsKey, decider,
host, shard.BeaconChainShardID, leader, blsKey, decider,
)
if err != nil {
t.Fatalf("Cannot craeate consensus: %v", err)
@ -202,7 +202,7 @@ func TestAddPeers(t *testing.T) {
}
decider := quorum.NewDecider(quorum.SuperMajorityVote)
consensus, err := consensus.New(
host, values.BeaconChainShardID, leader, blsKey, decider,
host, shard.BeaconChainShardID, leader, blsKey, decider,
)
if err != nil {
t.Fatalf("Cannot craeate consensus: %v", err)
@ -252,7 +252,7 @@ func TestAddBeaconPeer(t *testing.T) {
}
decider := quorum.NewDecider(quorum.SuperMajorityVote)
consensus, err := consensus.New(
host, values.BeaconChainShardID, leader, blsKey, decider,
host, shard.BeaconChainShardID, leader, blsKey, decider,
)
if err != nil {
t.Fatalf("Cannot craeate consensus: %v", err)

@ -13,13 +13,13 @@ import (
"github.com/harmony-one/harmony/core"
"github.com/harmony-one/harmony/core/state"
"github.com/harmony-one/harmony/core/types"
"github.com/harmony-one/harmony/core/values"
"github.com/harmony-one/harmony/core/vm"
shardingconfig "github.com/harmony-one/harmony/internal/configs/sharding"
"github.com/harmony-one/harmony/internal/ctxerror"
"github.com/harmony-one/harmony/internal/params"
"github.com/harmony-one/harmony/internal/utils"
"github.com/harmony-one/harmony/shard"
"github.com/harmony-one/harmony/shard/committee"
staking "github.com/harmony-one/harmony/staking/types"
)
@ -162,7 +162,7 @@ func (w *Worker) SelectStakingTransactionsForNewBlock(
coinbase common.Address) (staking.StakingTransactions, staking.StakingTransactions, staking.StakingTransactions) {
// only beaconchain process staking transaction
if w.chain.ShardID() != values.BeaconChainShardID {
if w.chain.ShardID() != shard.BeaconChainShardID {
utils.Logger().Warn().Msgf("Invalid shardID: %v", w.chain.ShardID())
return nil, nil, nil
}
@ -365,11 +365,13 @@ func (w *Worker) IncomingReceipts() []*types.CXReceiptsProof {
// ProposeShardStateWithoutBeaconSync proposes the next shard state for next epoch.
func (w *Worker) ProposeShardStateWithoutBeaconSync() shard.State {
if !core.ShardingSchedule.IsLastBlock(w.current.header.Number().Uint64()) {
if !shard.Schedule.IsLastBlock(w.current.header.Number().Uint64()) {
return nil
}
nextEpoch := new(big.Int).Add(w.current.header.Epoch(), common.Big1)
return core.CalculateShardState(nextEpoch)
shardState, _ := committee.WithStakingEnabled.ReadFromComputation(
new(big.Int).Add(w.current.header.Epoch(), common.Big1), *w.config, nil,
)
return shardState
}
// FinalizeNewBlock generate a new block for the next consensus round.

@ -0,0 +1,261 @@
package committee
import (
"math/big"
"sort"
"github.com/ethereum/go-ethereum/common"
"github.com/harmony-one/bls/ffi/go/bls"
"github.com/harmony-one/harmony/block"
common2 "github.com/harmony-one/harmony/internal/common"
shardingconfig "github.com/harmony-one/harmony/internal/configs/sharding"
"github.com/harmony-one/harmony/internal/ctxerror"
"github.com/harmony-one/harmony/internal/params"
"github.com/harmony-one/harmony/internal/utils"
"github.com/harmony-one/harmony/numeric"
"github.com/harmony-one/harmony/shard"
staking "github.com/harmony-one/harmony/staking/types"
)
// StateID means reading off whole network when using calls that accept
// a shardID parameter
const StateID = -1
// MembershipList ..
type MembershipList interface {
ReadFromComputation(
epoch *big.Int, config params.ChainConfig, reader StakingCandidatesReader,
) (shard.State, error)
ReadFromChain(epoch *big.Int, reader ChainReader) (shard.State, error)
}
// PublicKeys per epoch
type PublicKeys interface {
// If call shardID with StateID then only superCommittee is non-nil,
// otherwise get back the shardSpecific slice as well.
ComputePublicKeys(
epoch *big.Int, reader ChainReader, shardID int,
) (superCommittee, shardSpecific []*bls.PublicKey)
ReadPublicKeysFromDB(
hash common.Hash, reader ChainReader,
) ([]*bls.PublicKey, error)
}
// Reader ..
type Reader interface {
PublicKeys
MembershipList
}
// StakingCandidatesReader ..
type StakingCandidatesReader interface {
ValidatorInformation(addr common.Address) (*staking.Validator, error)
ValidatorStakingWithDelegation(addr common.Address) numeric.Dec
ValidatorCandidates() []common.Address
}
// ChainReader is a subset of Engine.ChainReader, just enough to do assignment
type ChainReader interface {
// ReadShardState retrieves sharding state given the epoch number.
// This api reads the shard state cached or saved on the chaindb.
// Thus, only should be used to read the shard state of the current chain.
ReadShardState(epoch *big.Int) (shard.State, error)
// GetHeader retrieves a block header from the database by hash and number.
GetHeaderByHash(common.Hash) *block.Header
// Config retrieves the blockchain's chain configuration.
Config() *params.ChainConfig
}
type partialStakingEnabled struct{}
var (
// WithStakingEnabled ..
WithStakingEnabled Reader = partialStakingEnabled{}
)
func preStakingEnabledCommittee(s shardingconfig.Instance) shard.State {
shardNum := int(s.NumShards())
shardHarmonyNodes := s.NumHarmonyOperatedNodesPerShard()
shardSize := s.NumNodesPerShard()
hmyAccounts := s.HmyAccounts()
fnAccounts := s.FnAccounts()
shardState := shard.State{}
for i := 0; i < shardNum; i++ {
com := shard.Committee{ShardID: uint32(i)}
for j := 0; j < shardHarmonyNodes; j++ {
index := i + j*shardNum // The initial account to use for genesis nodes
pub := &bls.PublicKey{}
pub.DeserializeHexStr(hmyAccounts[index].BlsPublicKey)
pubKey := shard.BlsPublicKey{}
pubKey.FromLibBLSPublicKey(pub)
// TODO: directly read address for bls too
curNodeID := shard.NodeID{
common2.ParseAddr(hmyAccounts[index].Address),
pubKey,
nil,
}
com.NodeList = append(com.NodeList, curNodeID)
}
// add FN runner's key
for j := shardHarmonyNodes; j < shardSize; j++ {
index := i + (j-shardHarmonyNodes)*shardNum
pub := &bls.PublicKey{}
pub.DeserializeHexStr(fnAccounts[index].BlsPublicKey)
pubKey := shard.BlsPublicKey{}
pubKey.FromLibBLSPublicKey(pub)
// TODO: directly read address for bls too
curNodeID := shard.NodeID{
common2.ParseAddr(fnAccounts[index].Address),
pubKey,
nil,
}
com.NodeList = append(com.NodeList, curNodeID)
}
shardState = append(shardState, com)
}
return shardState
}
func with400Stakers(
s shardingconfig.Instance, stakerReader StakingCandidatesReader,
) (shard.State, error) {
// TODO Nervous about this because overtime the list will become quite large
candidates := stakerReader.ValidatorCandidates()
stakers := make([]*staking.Validator, len(candidates))
for i := range candidates {
// TODO Should be using .ValidatorStakingWithDelegation, not implemented yet
validator, err := stakerReader.ValidatorInformation(candidates[i])
if err != nil {
return nil, err
}
stakers[i] = validator
}
sort.SliceStable(
stakers,
func(i, j int) bool { return stakers[i].Stake.Cmp(stakers[j].Stake) >= 0 },
)
const sCount = 401
top := stakers[:sCount]
shardCount := int(s.NumShards())
superComm := make(shard.State, shardCount)
fillCount := make([]int, shardCount)
// TODO Finish this logic, not correct, need to operate EPoS on slot level,
// not validator level
for i := 0; i < shardCount; i++ {
superComm[i] = shard.Committee{}
superComm[i].NodeList = make(shard.NodeIDList, s.NumNodesPerShard())
}
scratchPad := &bls.PublicKey{}
for i := range top {
spot := int(top[i].Address.Big().Int64()) % shardCount
fillCount[spot]++
// scratchPad.DeserializeHexStr()
pubKey := shard.BlsPublicKey{}
pubKey.FromLibBLSPublicKey(scratchPad)
superComm[spot].NodeList = append(
superComm[spot].NodeList,
shard.NodeID{
top[i].Address,
pubKey,
&shard.StakedMember{big.NewInt(0)},
},
)
}
utils.Logger().Info().Ints("distribution of Stakers in Shards", fillCount)
return superComm, nil
}
func (def partialStakingEnabled) ReadPublicKeysFromDB(
h common.Hash, reader ChainReader,
) ([]*bls.PublicKey, error) {
header := reader.GetHeaderByHash(h)
shardID := header.ShardID()
superCommittee, err := reader.ReadShardState(header.Epoch())
if err != nil {
return nil, err
}
subCommittee := superCommittee.FindCommitteeByID(shardID)
if subCommittee == nil {
return nil, ctxerror.New("cannot find shard in the shard state",
"blockNumber", header.Number(),
"shardID", header.ShardID(),
)
}
committerKeys := []*bls.PublicKey{}
for i := range subCommittee.NodeList {
committerKey := new(bls.PublicKey)
err := subCommittee.NodeList[i].BlsPublicKey.ToLibBLSPublicKey(committerKey)
if err != nil {
return nil, ctxerror.New("cannot convert BLS public key",
"blsPublicKey", subCommittee.NodeList[i].BlsPublicKey).WithCause(err)
}
committerKeys = append(committerKeys, committerKey)
}
return committerKeys, nil
return nil, nil
}
// ReadPublicKeysFromChain produces publicKeys of entire supercommittee per epoch, optionally providing a
// shard specific subcommittee
func (def partialStakingEnabled) ComputePublicKeys(
epoch *big.Int, reader ChainReader, shardID int,
) ([]*bls.PublicKey, []*bls.PublicKey) {
config := reader.Config()
instance := shard.Schedule.InstanceForEpoch(epoch)
if !config.IsStaking(epoch) {
superComm := preStakingEnabledCommittee(instance)
spot := 0
allIdentities := make([]*bls.PublicKey, int(instance.NumShards())*instance.NumNodesPerShard())
for i := range superComm {
for j := range superComm[i].NodeList {
identity := &bls.PublicKey{}
superComm[i].NodeList[j].BlsPublicKey.ToLibBLSPublicKey(identity)
allIdentities[spot] = identity
spot++
}
}
if shardID == StateID {
return allIdentities, nil
}
subCommittee := superComm.FindCommitteeByID(uint32(shardID))
subCommitteeIdentities := make([]*bls.PublicKey, len(subCommittee.NodeList))
spot = 0
for i := range subCommittee.NodeList {
identity := &bls.PublicKey{}
subCommittee.NodeList[i].BlsPublicKey.ToLibBLSPublicKey(identity)
subCommitteeIdentities[spot] = identity
spot++
}
return allIdentities, subCommitteeIdentities
}
// TODO Implement for the staked case
return nil, nil
}
func (def partialStakingEnabled) ReadFromChain(
epoch *big.Int, reader ChainReader,
) (newSuperComm shard.State, err error) {
return reader.ReadShardState(epoch)
}
// ReadFromComputation is single entry point for reading the State of the network
func (def partialStakingEnabled) ReadFromComputation(
epoch *big.Int, config params.ChainConfig, stakerReader StakingCandidatesReader,
) (newSuperComm shard.State, err error) {
instance := shard.Schedule.InstanceForEpoch(epoch)
if !config.IsStaking(epoch) {
return preStakingEnabledCommittee(instance), nil
}
return with400Stakers(instance, stakerReader)
}

@ -3,6 +3,8 @@ package shard
import (
"bytes"
"encoding/hex"
"encoding/json"
"math/big"
"sort"
"github.com/ethereum/go-ethereum/common"
@ -16,15 +18,71 @@ var (
emptyBlsPubKey = BlsPublicKey{}
)
// PublicKeySizeInBytes ..
const PublicKeySizeInBytes = 48
// EpochShardState is the shard state of an epoch
type EpochShardState struct {
Epoch uint64
ShardState State
}
// StakedMember is a committee member with stake
type StakedMember struct {
// nil means not active, 0 means our node, >= 0 means staked node
WithDelegationApplied *big.Int `json:"with-delegation-applied,omitempty"`
}
// State is the collection of all committees
type State []Committee
// BlsPublicKey defines the bls public key
type BlsPublicKey [PublicKeySizeInBytes]byte
// NodeID represents node id (BLS address)
type NodeID struct {
EcdsaAddress common.Address `json:"ecdsa_address"`
BlsPublicKey BlsPublicKey `json:"bls_pubkey"`
Validator *StakedMember `json:"staked-validator,omitempty" rlp:"nil"`
}
// NodeIDList is a list of NodeIDList.
type NodeIDList []NodeID
// Committee contains the active nodes in one shard
type Committee struct {
ShardID uint32 `json:"shard_id"`
NodeList NodeIDList `json:"node_list"`
}
// JSON produces a non-pretty printed JSON string of the SuperCommittee
func (ss State) JSON() string {
type V struct {
ECDSAAddress common.Address `json:"ecdsa_address"`
BLSPublicKey string `json:"bls-public-key"`
}
type T struct {
ShardID uint32 `json:"shard_id"`
Total int `json:"count"`
NodeList []V `json:"entries"`
}
t := []T{}
for i := range ss {
sub := ss[i]
subList := []V{}
for j := range sub.NodeList {
subList = append(subList, V{
sub.NodeList[j].EcdsaAddress,
sub.NodeList[j].BlsPublicKey.Hex(),
})
}
t = append(t, T{sub.ShardID, len(sub.NodeList), subList})
}
buf, _ := json.Marshal(t)
return string(buf)
}
// FindCommitteeByID returns the committee configuration for the given shard,
// or nil if the given shard is not found.
func (ss State) FindCommitteeByID(shardID uint32) *Committee {
@ -65,9 +123,6 @@ func CompareShardState(s1, s2 State) int {
return 0
}
// BlsPublicKey defines the bls public key
type BlsPublicKey [48]byte
// IsEmpty returns whether the bls public key is empty 0 bytes
func (pk BlsPublicKey) IsEmpty() bool {
return bytes.Compare(pk[:], emptyBlsPubKey[:]) == 0
@ -100,12 +155,6 @@ func CompareBlsPublicKey(k1, k2 BlsPublicKey) int {
return bytes.Compare(k1[:], k2[:])
}
// NodeID represents node id (BLS address)
type NodeID struct {
EcdsaAddress common.Address `json:"ecdsa_address"`
BlsPublicKey BlsPublicKey `json:"bls_pubkey"`
}
// CompareNodeID compares two node IDs.
func CompareNodeID(id1, id2 *NodeID) int {
if c := bytes.Compare(id1.EcdsaAddress[:], id2.EcdsaAddress[:]); c != 0 {
@ -117,9 +166,6 @@ func CompareNodeID(id1, id2 *NodeID) int {
return 0
}
// NodeIDList is a list of NodeIDList.
type NodeIDList []NodeID
// DeepCopy returns a deep copy of the receiver.
func (l NodeIDList) DeepCopy() NodeIDList {
return append(l[:0:0], l...)
@ -145,12 +191,6 @@ func CompareNodeIDList(l1, l2 NodeIDList) int {
return 0
}
// Committee contains the active nodes in one shard
type Committee struct {
ShardID uint32 `json:"shard_id"`
NodeList NodeIDList `json:"node_list"`
}
// DeepCopy returns a deep copy of the receiver.
func (c Committee) DeepCopy() Committee {
r := Committee{}

@ -31,14 +31,14 @@ func init() {
func TestGetHashFromNodeList(t *testing.T) {
l1 := []NodeID{
{common.Address{0x11}, blsPubKey1},
{common.Address{0x22}, blsPubKey2},
{common.Address{0x33}, blsPubKey3},
{common.Address{0x11}, blsPubKey1, nil},
{common.Address{0x22}, blsPubKey2, nil},
{common.Address{0x33}, blsPubKey3, nil},
}
l2 := []NodeID{
{common.Address{0x22}, blsPubKey2},
{common.Address{0x11}, blsPubKey1},
{common.Address{0x33}, blsPubKey3},
{common.Address{0x22}, blsPubKey2, nil},
{common.Address{0x11}, blsPubKey1, nil},
{common.Address{0x33}, blsPubKey3, nil},
}
h1 := GetHashFromNodeList(l1)
h2 := GetHashFromNodeList(l2)
@ -52,17 +52,17 @@ func TestHash(t *testing.T) {
com1 := Committee{
ShardID: 22,
NodeList: []NodeID{
{common.Address{0x12}, blsPubKey11},
{common.Address{0x23}, blsPubKey22},
{common.Address{0x11}, blsPubKey1},
{common.Address{0x12}, blsPubKey11, nil},
{common.Address{0x23}, blsPubKey22, nil},
{common.Address{0x11}, blsPubKey1, nil},
},
}
com2 := Committee{
ShardID: 2,
NodeList: []NodeID{
{common.Address{0x44}, blsPubKey4},
{common.Address{0x55}, blsPubKey5},
{common.Address{0x66}, blsPubKey6},
{common.Address{0x44}, blsPubKey4, nil},
{common.Address{0x55}, blsPubKey5, nil},
{common.Address{0x66}, blsPubKey6, nil},
},
}
shardState1 := State{com1, com2}
@ -71,17 +71,17 @@ func TestHash(t *testing.T) {
com3 := Committee{
ShardID: 2,
NodeList: []NodeID{
{common.Address{0x44}, blsPubKey4},
{common.Address{0x55}, blsPubKey5},
{common.Address{0x66}, blsPubKey6},
{common.Address{0x44}, blsPubKey4, nil},
{common.Address{0x55}, blsPubKey5, nil},
{common.Address{0x66}, blsPubKey6, nil},
},
}
com4 := Committee{
ShardID: 22,
NodeList: []NodeID{
{common.Address{0x12}, blsPubKey11},
{common.Address{0x23}, blsPubKey22},
{common.Address{0x11}, blsPubKey1},
{common.Address{0x12}, blsPubKey11, nil},
{common.Address{0x23}, blsPubKey22, nil},
{common.Address{0x11}, blsPubKey1, nil},
},
}

@ -0,0 +1,19 @@
package shard
import (
shardingconfig "github.com/harmony-one/harmony/internal/configs/sharding"
)
const (
// BeaconChainShardID is the ShardID of the BeaconChain
BeaconChainShardID = 0
)
// TODO ek – Schedule should really be part of a general-purpose network
// configuration. We are OK for the time being,
// until the day we should let one node process join multiple networks.
var (
// Schedule is the sharding configuration schedule.
// Depends on the type of the network. Defaults to the mainnet schedule.
Schedule shardingconfig.Schedule = shardingconfig.MainnetSchedule
)
Loading…
Cancel
Save