The core protocol of WoopChain
You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
woop/node/node_syncing.go

674 lines
20 KiB

package node
import (
"fmt"
"math/rand"
"net"
"sync"
"time"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/rlp"
lru "github.com/hashicorp/golang-lru"
"github.com/pkg/errors"
"github.com/harmony-one/harmony/api/service"
"github.com/harmony-one/harmony/api/service/legacysync"
legdownloader "github.com/harmony-one/harmony/api/service/legacysync/downloader"
downloader_pb "github.com/harmony-one/harmony/api/service/legacysync/downloader/proto"
"github.com/harmony-one/harmony/api/service/synchronize"
"github.com/harmony-one/harmony/core"
"github.com/harmony-one/harmony/core/types"
"github.com/harmony-one/harmony/hmy/downloader"
nodeconfig "github.com/harmony-one/harmony/internal/configs/node"
"github.com/harmony-one/harmony/internal/utils"
"github.com/harmony-one/harmony/node/worker"
"github.com/harmony-one/harmony/p2p"
"github.com/harmony-one/harmony/shard"
)
// Constants related to doing syncing.
const (
SyncFrequency = 60
)
var letterRunes = []rune("abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ")
// BeaconSyncHook is the hook function called after inserted beacon in downloader
// TODO: This is a small misc piece of consensus logic. Better put it to consensus module.
func (node *Node) BeaconSyncHook() {
if node.Consensus.IsLeader() || rand.Intn(100) == 0 {
// TODO: Instead of leader, it would better be validator do this broadcast since leader do
// not have much idle resources.
node.BroadcastCrossLink()
}
}
// GenerateRandomString generates a random string with given length
func GenerateRandomString(n int) string {
b := make([]rune, n)
for i := range b {
b[i] = letterRunes[rand.Intn(len(letterRunes))]
}
return string(b)
}
// getNeighborPeers is a helper function to return list of peers
// based on different neightbor map
func getNeighborPeers(neighbor *sync.Map) []p2p.Peer {
tmp := []p2p.Peer{}
neighbor.Range(func(k, v interface{}) bool {
p := v.(p2p.Peer)
t := p.Port
p.Port = legacysync.GetSyncingPort(t)
tmp = append(tmp, p)
return true
})
return tmp
}
6 years ago
// DoSyncWithoutConsensus gets sync-ed to blockchain without joining consensus
func (node *Node) DoSyncWithoutConsensus() {
go node.DoSyncing(node.Blockchain(), node.Worker, false) //Don't join consensus
}
// IsSameHeight tells whether node is at same bc height as a peer
func (node *Node) IsSameHeight() (uint64, bool) {
if node.stateSync == nil {
node.stateSync = node.getStateSync()
}
return node.stateSync.IsSameBlockchainHeight(node.Blockchain())
}
func (node *Node) getStateSync() *legacysync.StateSync {
return legacysync.CreateStateSync(node.SelfPeer.IP, node.SelfPeer.Port,
node.GetSyncID(), node.NodeConfig.Role() == nodeconfig.ExplorerNode)
}
// SyncingPeerProvider is an interface for getting the peers in the given shard.
type SyncingPeerProvider interface {
SyncingPeers(shardID uint32) (peers []p2p.Peer, err error)
}
// LegacySyncingPeerProvider uses neighbor lists stored in a Node to serve
// syncing peer list query.
type LegacySyncingPeerProvider struct {
node *Node
shardID func() uint32
}
// NewLegacySyncingPeerProvider creates and returns a new node-based syncing
// peer provider.
func NewLegacySyncingPeerProvider(node *Node) *LegacySyncingPeerProvider {
var shardID func() uint32
if node.shardChains != nil {
shardID = node.Blockchain().ShardID
}
return &LegacySyncingPeerProvider{node: node, shardID: shardID}
}
// SyncingPeers returns peers stored in neighbor maps in the node structure.
func (p *LegacySyncingPeerProvider) SyncingPeers(shardID uint32) (peers []p2p.Peer, err error) {
switch shardID {
case p.shardID():
peers = getNeighborPeers(&p.node.Neighbors)
case 0:
peers = getNeighborPeers(&p.node.BeaconNeighbors)
default:
return nil, errors.Errorf("unsupported shard ID %v", shardID)
}
return peers, nil
}
// DNSSyncingPeerProvider uses the given DNS zone to resolve syncing peers.
type DNSSyncingPeerProvider struct {
zone, port string
lookupHost func(name string) (addrs []string, err error)
}
// NewDNSSyncingPeerProvider returns a provider that uses given DNS name and
// port number to resolve syncing peers.
func NewDNSSyncingPeerProvider(zone, port string) *DNSSyncingPeerProvider {
return &DNSSyncingPeerProvider{
zone: zone,
port: port,
lookupHost: net.LookupHost,
}
}
// SyncingPeers resolves DNS name into peers and returns them.
func (p *DNSSyncingPeerProvider) SyncingPeers(shardID uint32) (peers []p2p.Peer, err error) {
dns := fmt.Sprintf("s%d.%s", shardID, p.zone)
addrs, err := p.lookupHost(dns)
if err != nil {
return nil, errors.Wrapf(err,
"[SYNC] cannot find peers using DNS name %#v", dns)
}
for _, addr := range addrs {
peers = append(peers, p2p.Peer{IP: addr, Port: p.port})
}
return peers, nil
}
// LocalSyncingPeerProvider uses localnet deployment convention to synthesize
// syncing peers.
type LocalSyncingPeerProvider struct {
basePort, selfPort uint16
numShards, shardSize uint32
}
// NewLocalSyncingPeerProvider returns a provider that synthesizes syncing
// peers given the network configuration
func NewLocalSyncingPeerProvider(
basePort, selfPort uint16, numShards, shardSize uint32,
) *LocalSyncingPeerProvider {
return &LocalSyncingPeerProvider{
basePort: basePort,
selfPort: selfPort,
numShards: numShards,
shardSize: shardSize,
}
}
// SyncingPeers returns local syncing peers using the sharding configuration.
func (p *LocalSyncingPeerProvider) SyncingPeers(shardID uint32) (peers []p2p.Peer, err error) {
if shardID >= p.numShards {
return nil, errors.Errorf(
"shard ID %d out of range 0..%d", shardID, p.numShards-1)
}
firstPort := uint32(p.basePort) + shardID
endPort := uint32(p.basePort) + p.numShards*p.shardSize
for port := firstPort; port < endPort; port += p.numShards {
if port == uint32(p.selfPort) {
continue // do not sync from self
}
peers = append(peers, p2p.Peer{IP: "127.0.0.1", Port: fmt.Sprint(port)})
}
return peers, nil
}
// doBeaconSyncing update received beaconchain blocks and downloads missing beacon chain blocks
func (node *Node) doBeaconSyncing() {
if node.NodeConfig.IsOffline {
return
}
if !node.NodeConfig.Downloader {
// If Downloader is not working, we need also deal with blocks from beaconBlockChannel
go func(node *Node) {
// TODO ek – infinite loop; add shutdown/cleanup logic
for beaconBlock := range node.BeaconBlockChannel {
if node.beaconSync != nil {
err := node.beaconSync.UpdateBlockAndStatus(
beaconBlock, node.Beaconchain(), node.BeaconWorker, true,
)
if err != nil {
node.beaconSync.AddLastMileBlock(beaconBlock)
} else if node.Consensus.IsLeader() || rand.Intn(100) <= 1 {
// Only leader or 2% of validators broadcast crosslink to avoid spamming p2p
node.BroadcastCrossLink()
}
}
}
}(node)
}
// TODO ek – infinite loop; add shutdown/cleanup logic
for {
if node.beaconSync == nil {
utils.Logger().Info().Msg("initializing beacon sync")
node.beaconSync = node.getStateSync()
}
if node.beaconSync.GetActivePeerNumber() == 0 {
utils.Logger().Info().Msg("no peers; bootstrapping beacon sync config")
// 0 means shardID=0 here
peers, err := node.SyncingPeerProvider.SyncingPeers(0)
if err != nil {
utils.Logger().Warn().
Err(err).
Msg("cannot retrieve beacon syncing peers")
continue
}
if err := node.beaconSync.CreateSyncConfig(peers, true); err != nil {
utils.Logger().Warn().Err(err).Msg("cannot create beacon sync config")
continue
}
}
node.beaconSync.SyncLoop(node.Beaconchain(), node.BeaconWorker, true, nil)
time.Sleep(time.Duration(SyncFrequency) * time.Second)
}
}
// DoSyncing keep the node in sync with other peers, willJoinConsensus means the node will try to join consensus after catch up
func (node *Node) DoSyncing(bc *core.BlockChain, worker *worker.Worker, willJoinConsensus bool) {
if node.NodeConfig.IsOffline {
return
}
ticker := time.NewTicker(time.Duration(SyncFrequency) * time.Second)
// TODO ek – infinite loop; add shutdown/cleanup logic
for {
select {
case <-ticker.C:
node.doSync(bc, worker, willJoinConsensus)
case <-node.Consensus.BlockNumLowChan:
node.doSync(bc, worker, willJoinConsensus)
}
}
}
// doSync keep the node in sync with other peers, willJoinConsensus means the node will try to join consensus after catch up
func (node *Node) doSync(bc *core.BlockChain, worker *worker.Worker, willJoinConsensus bool) {
if node.stateSync.GetActivePeerNumber() < legacysync.NumPeersLowBound {
shardID := bc.ShardID()
peers, err := node.SyncingPeerProvider.SyncingPeers(shardID)
if err != nil {
utils.Logger().Warn().
Err(err).
Uint32("shard_id", shardID).
Msg("cannot retrieve syncing peers")
return
}
if err := node.stateSync.CreateSyncConfig(peers, false); err != nil {
utils.Logger().Warn().
Err(err).
Interface("peers", peers).
Msg("[SYNC] create peers error")
return
}
utils.Logger().Debug().Int("len", node.stateSync.GetActivePeerNumber()).Msg("[SYNC] Get Active Peers")
}
// TODO: treat fake maximum height
if node.stateSync.IsOutOfSync(bc, true) {
node.IsInSync.UnSet()
if willJoinConsensus {
node.Consensus.BlocksNotSynchronized()
}
node.stateSync.SyncLoop(bc, worker, false, node.Consensus)
if willJoinConsensus {
node.IsInSync.Set()
node.Consensus.BlocksSynchronized()
}
}
node.IsInSync.Set()
}
// SupportGRPCSyncServer do gRPC sync server
func (node *Node) SupportGRPCSyncServer() {
node.InitSyncingServer()
node.StartSyncingServer()
}
// StartGRPCSyncClient start the legacy gRPC sync process
func (node *Node) StartGRPCSyncClient() {
if node.Blockchain().ShardID() != shard.BeaconChainShardID {
utils.Logger().Info().
Uint32("shardID", node.Blockchain().ShardID()).
Msg("SupportBeaconSyncing")
go node.doBeaconSyncing()
}
node.supportSyncing()
}
// supportSyncing keeps sleeping until it's doing consensus or it's a leader.
func (node *Node) supportSyncing() {
joinConsensus := false
// Check if the current node is explorer node.
switch node.NodeConfig.Role() {
case nodeconfig.Validator:
joinConsensus = true
}
// Send new block to unsync node if the current node is not explorer node.
// TODO: leo this pushing logic has to be removed
if joinConsensus {
go node.SendNewBlockToUnsync()
}
if node.stateSync == nil {
node.stateSync = node.getStateSync()
utils.Logger().Debug().Msg("[SYNC] initialized state sync")
}
go node.DoSyncing(node.Blockchain(), node.Worker, joinConsensus)
}
// InitSyncingServer starts downloader server.
func (node *Node) InitSyncingServer() {
if node.downloaderServer == nil {
node.downloaderServer = legdownloader.NewServer(node)
}
}
// StartSyncingServer starts syncing server.
func (node *Node) StartSyncingServer() {
utils.Logger().Info().Msg("[SYNC] support_syncing: StartSyncingServer")
if node.downloaderServer.GrpcServer == nil {
node.downloaderServer.Start(node.SelfPeer.IP, legacysync.GetSyncingPort(node.SelfPeer.Port))
}
}
// SendNewBlockToUnsync send latest verified block to unsync, registered nodes
func (node *Node) SendNewBlockToUnsync() {
for {
block := <-node.Consensus.VerifiedNewBlock
blockHash, err := rlp.EncodeToBytes(block)
if err != nil {
utils.Logger().Warn().Msg("[SYNC] unable to encode block to hashes")
continue
}
node.stateMutex.Lock()
for peerID, config := range node.peerRegistrationRecord {
elapseTime := time.Now().UnixNano() - config.timestamp
if elapseTime > broadcastTimeout {
utils.Logger().Warn().Str("peerID", peerID).Msg("[SYNC] SendNewBlockToUnsync to peer timeout")
node.peerRegistrationRecord[peerID].client.Close()
delete(node.peerRegistrationRecord, peerID)
continue
}
response, err := config.client.PushNewBlock(node.GetSyncID(), blockHash, false)
// close the connection if cannot push new block to unsync node
if err != nil {
node.peerRegistrationRecord[peerID].client.Close()
delete(node.peerRegistrationRecord, peerID)
}
if response != nil && response.Type == downloader_pb.DownloaderResponse_INSYNC {
node.peerRegistrationRecord[peerID].client.Close()
delete(node.peerRegistrationRecord, peerID)
}
}
node.stateMutex.Unlock()
}
}
// CalculateResponse implements DownloadInterface on Node object.
func (node *Node) CalculateResponse(request *downloader_pb.DownloaderRequest, incomingPeer string) (*downloader_pb.DownloaderResponse, error) {
response := &downloader_pb.DownloaderResponse{}
if node.NodeConfig.IsOffline {
return response, nil
}
switch request.Type {
case downloader_pb.DownloaderRequest_BLOCKHASH:
if request.BlockHash == nil {
return response, fmt.Errorf("[SYNC] GetBlockHashes Request BlockHash is NIL")
}
if request.Size == 0 || request.Size > legacysync.SyncLoopBatchSize {
return response, fmt.Errorf("[SYNC] GetBlockHashes Request contains invalid Size %v", request.Size)
}
size := uint64(request.Size)
var startHashHeader common.Hash
copy(startHashHeader[:], request.BlockHash[:])
startHeader := node.Blockchain().GetHeaderByHash(startHashHeader)
if startHeader == nil {
return response, fmt.Errorf("[SYNC] GetBlockHashes Request cannot find startHash %s", startHashHeader.Hex())
}
startHeight := startHeader.Number().Uint64()
endHeight := node.Blockchain().CurrentBlock().NumberU64()
if startHeight >= endHeight {
utils.Logger().
Debug().
Uint64("myHeight", endHeight).
Uint64("requestHeight", startHeight).
Str("incomingIP", request.Ip).
Str("incomingPort", request.Port).
Str("incomingPeer", incomingPeer).
Msg("[SYNC] GetBlockHashes Request: I am not higher than requested node")
return response, nil
}
for blockNum := startHeight; blockNum <= startHeight+size; blockNum++ {
header := node.Blockchain().GetHeaderByNumber(blockNum)
if header == nil {
break
}
blockHash := header.Hash()
response.Payload = append(response.Payload, blockHash[:])
}
case downloader_pb.DownloaderRequest_BLOCKHEADER:
var hash common.Hash
for _, bytes := range request.Hashes {
hash.SetBytes(bytes)
Merge master to t3 0523 (#3085) * fix signers share computation to use only non-harmony nodes (#3008) * Add LastEpochInCommittee at prestaking epoch too (#3014) * write validator stats at the end of prestaking epoch (#3021) * write validator stats at the end of prestaking epoch * also need to write block reward accumulator * [build] enable static build by default (#2885) scripts/go_executable_build.sh -s (static build, default) scripts/go_executable_build.sh -S (non-static build) Signed-off-by: Leo Chen <leo@harmony.one> * Code formatting fix based on `staticcheck` (#3023) * fix some static check issues * more fixes according to staticcheck * add nil check on getHeader * Add more log info * [rpc] Revert change to v2 GetBalance RPC (#3037) * Cherrypick crosslink related fixes from t3 to master. (#3045) * Optimize crosslink verification logic and add more cache (#3032) * make crosslink broadcast smarter and more efficient (#3036) * adjust catch up speed to be a sane number * Fix crosslink broadcast condition (#3041) * Tx pool error report fix & local build fix (#3046) * [tx_list] Add transaction after you can fetch tx cost * [tx_pool] Add error sink reports for removed tx * Correct log levels for known transactions * Remove redundant hash to `enqueue` and `add` signatures * Update tests for signature changes * [build] Fix local build for MacOs * [tx_pool] Correct typo & logging level for known tx * [config] Create Testnet config * [node.sh] update testnet network config * [config] correct testnet config * [main] Enable beacon sync for explorer nodes (#3059) * [apr] should not use snapshot from last epoch, use current epoch snapshot (#3051) * use snapshot total delegation instead of current total delegation * should not use snapshot from last epoch, use current epoch snapshot (#3049) Co-authored-by: Rongjian Lan <rongjian.lan@gmail.com> Co-authored-by: Rongjian Lan <rongjian.lan@gmail.com> * cherrypick log level change * [rpc] Add active status back to Validator information [validator] Fix typo for ValidatorRPCEnhanced [Makefile] Change make exe to build non-static by default * [network] Added caching to node server (#3048) * [network] Added caching to node server * [network] renamed two singleflight.Group * [network] added two error handling * [network] added forget logic for single flight * [network] changed single flight to lru cache solution * [network] adjusted the cache size * [network] changed cache size to 10000 * Transaction api improvements (#3057) * [node] Add option to broadcast invalid tx (on by default) * update main args to take in broadcast invalid tx * update node.sh for new broadcast invalid tx option * [rpc] Do not submit transaction if tx has been finalized * Support legacy behavior and not return errors on tx submission * [main] make broadcasting invalid tx default false * [node] Fix formatting * Add timeouts for the explorer HTTP service * [test] State trans unit test (#3043) * completing create-validator tests generalizing create validator tests, started with edit validator tests one round completed before major refactor * [test] finished coding test TestCheckDuplicateFields * [test] finished debug TestCheckDuplicateFields * [test] Added TestDescription_EnsureLength * [test] added test in validator_test.go TestUpdateDescription * [test] Added test TestComputed_String * [test] refactored TestValidatorSanityCheck and added some message in error message returned from sanity check * [test] removed test index for debugging * [test] Added test TestValidatorWrapper_SanityCheck. Added value check for TestMarshalUnmarshalValidator. Fixed rlp usage defect in UnmarshalValidator * [test] added test TestVerifyBLSKeys * [test] added test TestContainsHarmonyBLSKeys * [test] added test TestCreateValidatorFromNewMsg * [test] fixed some tests errors * [test] added test TestUpdateValidatorFromEditMsg, last editted the test file, finished testing validator_test.go * [staking] added numeric.Dec.Copy method * [staking] added CommissionRates.Copy * [staking] add nil case handle to copy method and related test case * [test] added nil case for commissionRates.Copy test * [staking] finished CreateValidator.Copy and related test case * [staking] added EditValidator.Copy method and related test case * [test] added zero test cases for Copy method * [staking] implemented Delegate.Copy and related unit test case * [staking] added Undelegate.Copy and CollectReward.Copy method. Implemented corresponding test cases * [test] added two more simple unit tests * [test] solving the merge conflict with deep copy fix * [test] added schedule for sharding * [test] refactored double_sign_test.go * [test] comment out all test cases for now. * [test] added record Copy method for testing * [test] added the first test case in TestVerify * [test] finished test TestVerify * [test] Added test TestVerify, TestApplySlashRate, TestSetDifference * [test] half through TestDelegatorSlashApply * [test] fix the problem cause by unsafe usage of big.Int * [staking] added the copy methods to validatorWrapper * [test] added test cases for copy method for staking/validatorWrapper * [test] added test case TestApply * [test] added test case TestRate * [test] fix the golint errors * [test] removed commented out unit test code * [test] remote the empty line in imports * [test] moved copy methods to stake_testing module * [test] removed usage of interface stateDB * [test] removed empty lines in imports * [test] fixed golint package name issues * [test] removed a todo when writing code * [test] moved record.Copy to staking/slash/test * [test] add some changes * [test] added prototypes in staketest module * [test] fix the golint issue in staketest * [test] make prototype more beautiful * [test] refactored test TestCheckDuplicateFields * [test] add createValidator test data * [test] added positive test case for VerifyAndCreateValidatorFromMsg * [test] added create validator test * [test] added the positive test case for TestVerifyAndEditValidatorFromMsg * [test] added the tests case TestVerifyAndEditValidatorFromMsg * [test] fix one of the error is not triggered as expected * [test] more changes in test * [test] fix the positive test data for TestVerifyAndDelegateFromMsg * [test] Fixed two comment golint errors * [test] added delegate tests * [test] added a make function. WIP * [test] added undelegate positive test * [test] added negative test cases for undelegate * [test] added positive test. not passed. Need to add assert method in test suites * [test] added equal function to check Validator Equality * [test] added equals for staketest * [test] replaced deep equal with new equal methods: * [test] removed unused codes * [test] Finishing touch * [test] fix comment golint issue * [test] removed dead code in staking_verifier_test.go Co-authored-by: Ganesha Upadhyaya <ganeshrvce@gmail.com> * changed the sync port to dns port in flag * [network] Limit client sync connection (#3071) * [network] limit client connected peers * Changed signature and added tests * Set secure permissions on generated blskey pass file in node.sh * Fix spellin gtypo in node.sh * fix latest param handling in api v1 * added comment for the isGreaterThanLatestBlockNum check * fix a nil pointer crash due to previous revert Signed-off-by: Leo Chen <leo@harmony.one> * fix nil pointer crash Signed-off-by: Leo Chen <leo@harmony.one> * fix nil pointer of block Signed-off-by: Leo Chen <leo@harmony.one> * [rpc] add viewID and epoch to RPCMarshalBlock (#3009) * [rpc] add viewID and epoch to RPCMarshalBlock * add apt-get update * Do not rollback current block on failed block sync (#3101) Co-authored-by: Ganesha Upadhyaya <ganeshrvce@gmail.com> Co-authored-by: Rongjian Lan <rongjian.lan@gmail.com> Co-authored-by: Jacky Wang <jackyw.se@gmail.com> Co-authored-by: Janet Liang <56005637+janet-harmony@users.noreply.github.com> Co-authored-by: Daniel Van Der Maden <dvandermaden0@berkeley.edu> Co-authored-by: Janet Liang <janet@harmony.one> Co-authored-by: Sebastian Johnsson <sebastian.johnsson@gmail.com> Co-authored-by: hsiung <hsiung@pinterest.com> Co-authored-by: Dennis Won <jhwon0820@gmail.com> Co-authored-by: Yishuang Chen <34232522+ivorytowerdds@users.noreply.github.com>
5 years ago
encodedBlockHeader, err := node.getEncodedBlockHeaderByHash(hash)
if err == nil {
response.Payload = append(response.Payload, encodedBlockHeader)
}
}
case downloader_pb.DownloaderRequest_BLOCK:
var hash common.Hash
for _, bytes := range request.Hashes {
hash.SetBytes(bytes)
Merge master to t3 0523 (#3085) * fix signers share computation to use only non-harmony nodes (#3008) * Add LastEpochInCommittee at prestaking epoch too (#3014) * write validator stats at the end of prestaking epoch (#3021) * write validator stats at the end of prestaking epoch * also need to write block reward accumulator * [build] enable static build by default (#2885) scripts/go_executable_build.sh -s (static build, default) scripts/go_executable_build.sh -S (non-static build) Signed-off-by: Leo Chen <leo@harmony.one> * Code formatting fix based on `staticcheck` (#3023) * fix some static check issues * more fixes according to staticcheck * add nil check on getHeader * Add more log info * [rpc] Revert change to v2 GetBalance RPC (#3037) * Cherrypick crosslink related fixes from t3 to master. (#3045) * Optimize crosslink verification logic and add more cache (#3032) * make crosslink broadcast smarter and more efficient (#3036) * adjust catch up speed to be a sane number * Fix crosslink broadcast condition (#3041) * Tx pool error report fix & local build fix (#3046) * [tx_list] Add transaction after you can fetch tx cost * [tx_pool] Add error sink reports for removed tx * Correct log levels for known transactions * Remove redundant hash to `enqueue` and `add` signatures * Update tests for signature changes * [build] Fix local build for MacOs * [tx_pool] Correct typo & logging level for known tx * [config] Create Testnet config * [node.sh] update testnet network config * [config] correct testnet config * [main] Enable beacon sync for explorer nodes (#3059) * [apr] should not use snapshot from last epoch, use current epoch snapshot (#3051) * use snapshot total delegation instead of current total delegation * should not use snapshot from last epoch, use current epoch snapshot (#3049) Co-authored-by: Rongjian Lan <rongjian.lan@gmail.com> Co-authored-by: Rongjian Lan <rongjian.lan@gmail.com> * cherrypick log level change * [rpc] Add active status back to Validator information [validator] Fix typo for ValidatorRPCEnhanced [Makefile] Change make exe to build non-static by default * [network] Added caching to node server (#3048) * [network] Added caching to node server * [network] renamed two singleflight.Group * [network] added two error handling * [network] added forget logic for single flight * [network] changed single flight to lru cache solution * [network] adjusted the cache size * [network] changed cache size to 10000 * Transaction api improvements (#3057) * [node] Add option to broadcast invalid tx (on by default) * update main args to take in broadcast invalid tx * update node.sh for new broadcast invalid tx option * [rpc] Do not submit transaction if tx has been finalized * Support legacy behavior and not return errors on tx submission * [main] make broadcasting invalid tx default false * [node] Fix formatting * Add timeouts for the explorer HTTP service * [test] State trans unit test (#3043) * completing create-validator tests generalizing create validator tests, started with edit validator tests one round completed before major refactor * [test] finished coding test TestCheckDuplicateFields * [test] finished debug TestCheckDuplicateFields * [test] Added TestDescription_EnsureLength * [test] added test in validator_test.go TestUpdateDescription * [test] Added test TestComputed_String * [test] refactored TestValidatorSanityCheck and added some message in error message returned from sanity check * [test] removed test index for debugging * [test] Added test TestValidatorWrapper_SanityCheck. Added value check for TestMarshalUnmarshalValidator. Fixed rlp usage defect in UnmarshalValidator * [test] added test TestVerifyBLSKeys * [test] added test TestContainsHarmonyBLSKeys * [test] added test TestCreateValidatorFromNewMsg * [test] fixed some tests errors * [test] added test TestUpdateValidatorFromEditMsg, last editted the test file, finished testing validator_test.go * [staking] added numeric.Dec.Copy method * [staking] added CommissionRates.Copy * [staking] add nil case handle to copy method and related test case * [test] added nil case for commissionRates.Copy test * [staking] finished CreateValidator.Copy and related test case * [staking] added EditValidator.Copy method and related test case * [test] added zero test cases for Copy method * [staking] implemented Delegate.Copy and related unit test case * [staking] added Undelegate.Copy and CollectReward.Copy method. Implemented corresponding test cases * [test] added two more simple unit tests * [test] solving the merge conflict with deep copy fix * [test] added schedule for sharding * [test] refactored double_sign_test.go * [test] comment out all test cases for now. * [test] added record Copy method for testing * [test] added the first test case in TestVerify * [test] finished test TestVerify * [test] Added test TestVerify, TestApplySlashRate, TestSetDifference * [test] half through TestDelegatorSlashApply * [test] fix the problem cause by unsafe usage of big.Int * [staking] added the copy methods to validatorWrapper * [test] added test cases for copy method for staking/validatorWrapper * [test] added test case TestApply * [test] added test case TestRate * [test] fix the golint errors * [test] removed commented out unit test code * [test] remote the empty line in imports * [test] moved copy methods to stake_testing module * [test] removed usage of interface stateDB * [test] removed empty lines in imports * [test] fixed golint package name issues * [test] removed a todo when writing code * [test] moved record.Copy to staking/slash/test * [test] add some changes * [test] added prototypes in staketest module * [test] fix the golint issue in staketest * [test] make prototype more beautiful * [test] refactored test TestCheckDuplicateFields * [test] add createValidator test data * [test] added positive test case for VerifyAndCreateValidatorFromMsg * [test] added create validator test * [test] added the positive test case for TestVerifyAndEditValidatorFromMsg * [test] added the tests case TestVerifyAndEditValidatorFromMsg * [test] fix one of the error is not triggered as expected * [test] more changes in test * [test] fix the positive test data for TestVerifyAndDelegateFromMsg * [test] Fixed two comment golint errors * [test] added delegate tests * [test] added a make function. WIP * [test] added undelegate positive test * [test] added negative test cases for undelegate * [test] added positive test. not passed. Need to add assert method in test suites * [test] added equal function to check Validator Equality * [test] added equals for staketest * [test] replaced deep equal with new equal methods: * [test] removed unused codes * [test] Finishing touch * [test] fix comment golint issue * [test] removed dead code in staking_verifier_test.go Co-authored-by: Ganesha Upadhyaya <ganeshrvce@gmail.com> * changed the sync port to dns port in flag * [network] Limit client sync connection (#3071) * [network] limit client connected peers * Changed signature and added tests * Set secure permissions on generated blskey pass file in node.sh * Fix spellin gtypo in node.sh * fix latest param handling in api v1 * added comment for the isGreaterThanLatestBlockNum check * fix a nil pointer crash due to previous revert Signed-off-by: Leo Chen <leo@harmony.one> * fix nil pointer crash Signed-off-by: Leo Chen <leo@harmony.one> * fix nil pointer of block Signed-off-by: Leo Chen <leo@harmony.one> * [rpc] add viewID and epoch to RPCMarshalBlock (#3009) * [rpc] add viewID and epoch to RPCMarshalBlock * add apt-get update * Do not rollback current block on failed block sync (#3101) Co-authored-by: Ganesha Upadhyaya <ganeshrvce@gmail.com> Co-authored-by: Rongjian Lan <rongjian.lan@gmail.com> Co-authored-by: Jacky Wang <jackyw.se@gmail.com> Co-authored-by: Janet Liang <56005637+janet-harmony@users.noreply.github.com> Co-authored-by: Daniel Van Der Maden <dvandermaden0@berkeley.edu> Co-authored-by: Janet Liang <janet@harmony.one> Co-authored-by: Sebastian Johnsson <sebastian.johnsson@gmail.com> Co-authored-by: hsiung <hsiung@pinterest.com> Co-authored-by: Dennis Won <jhwon0820@gmail.com> Co-authored-by: Yishuang Chen <34232522+ivorytowerdds@users.noreply.github.com>
5 years ago
encodedBlock, err := node.getEncodedBlockByHash(hash)
if err == nil {
response.Payload = append(response.Payload, encodedBlock)
}
}
case downloader_pb.DownloaderRequest_BLOCKWITHSIG:
var hash common.Hash
for _, bytes := range request.Hashes {
hash.SetBytes(bytes)
encoded, err := node.getEncodedBlockWithSigByHash(hash)
if err != nil {
utils.Logger().Info().Err(err).Str("hash", hash.String()).Msg("failed to get block with sig")
continue
}
response.Payload = append(response.Payload, encoded)
}
case downloader_pb.DownloaderRequest_BLOCKHEIGHT:
response.BlockHeight = node.Blockchain().CurrentBlock().NumberU64()
// this is the out of sync node acts as grpc server side
case downloader_pb.DownloaderRequest_NEWBLOCK:
if node.IsInSync.IsSet() {
response.Type = downloader_pb.DownloaderResponse_INSYNC
return response, nil
}
var blockObj types.Block
err := rlp.DecodeBytes(request.BlockHash, &blockObj)
if err != nil {
utils.Logger().Warn().Msg("[SYNC] unable to decode received new block")
return response, err
}
node.stateSync.AddNewBlock(request.PeerHash, &blockObj)
case downloader_pb.DownloaderRequest_REGISTER:
peerID := string(request.PeerHash[:])
ip := request.Ip
port := request.Port
node.stateMutex.Lock()
defer node.stateMutex.Unlock()
if _, ok := node.peerRegistrationRecord[peerID]; ok {
response.Type = downloader_pb.DownloaderResponse_FAIL
utils.Logger().Warn().
Interface("ip", ip).
Interface("port", port).
Msg("[SYNC] peerRegistration record already exists")
return response, nil
} else if len(node.peerRegistrationRecord) >= maxBroadcastNodes {
response.Type = downloader_pb.DownloaderResponse_FAIL
utils.Logger().Debug().
Str("ip", ip).
Str("port", port).
Msg("[SYNC] maximum registration limit exceeds")
return response, nil
} else {
response.Type = downloader_pb.DownloaderResponse_FAIL
syncPort := legacysync.GetSyncingPort(port)
client := legdownloader.ClientSetup(ip, syncPort)
if client == nil {
utils.Logger().Warn().
Str("ip", ip).
Str("port", port).
Msg("[SYNC] unable to setup client for peerID")
return response, nil
}
config := &syncConfig{timestamp: time.Now().UnixNano(), client: client}
node.peerRegistrationRecord[peerID] = config
utils.Logger().Debug().
Str("ip", ip).
Str("port", port).
Msg("[SYNC] register peerID success")
response.Type = downloader_pb.DownloaderResponse_SUCCESS
}
case downloader_pb.DownloaderRequest_REGISTERTIMEOUT:
if !node.IsInSync.IsSet() {
count := node.stateSync.RegisterNodeInfo()
utils.Logger().Debug().
Int("number", count).
Msg("[SYNC] extra node registered")
}
}
return response, nil
}
Merge master to t3 0523 (#3085) * fix signers share computation to use only non-harmony nodes (#3008) * Add LastEpochInCommittee at prestaking epoch too (#3014) * write validator stats at the end of prestaking epoch (#3021) * write validator stats at the end of prestaking epoch * also need to write block reward accumulator * [build] enable static build by default (#2885) scripts/go_executable_build.sh -s (static build, default) scripts/go_executable_build.sh -S (non-static build) Signed-off-by: Leo Chen <leo@harmony.one> * Code formatting fix based on `staticcheck` (#3023) * fix some static check issues * more fixes according to staticcheck * add nil check on getHeader * Add more log info * [rpc] Revert change to v2 GetBalance RPC (#3037) * Cherrypick crosslink related fixes from t3 to master. (#3045) * Optimize crosslink verification logic and add more cache (#3032) * make crosslink broadcast smarter and more efficient (#3036) * adjust catch up speed to be a sane number * Fix crosslink broadcast condition (#3041) * Tx pool error report fix & local build fix (#3046) * [tx_list] Add transaction after you can fetch tx cost * [tx_pool] Add error sink reports for removed tx * Correct log levels for known transactions * Remove redundant hash to `enqueue` and `add` signatures * Update tests for signature changes * [build] Fix local build for MacOs * [tx_pool] Correct typo & logging level for known tx * [config] Create Testnet config * [node.sh] update testnet network config * [config] correct testnet config * [main] Enable beacon sync for explorer nodes (#3059) * [apr] should not use snapshot from last epoch, use current epoch snapshot (#3051) * use snapshot total delegation instead of current total delegation * should not use snapshot from last epoch, use current epoch snapshot (#3049) Co-authored-by: Rongjian Lan <rongjian.lan@gmail.com> Co-authored-by: Rongjian Lan <rongjian.lan@gmail.com> * cherrypick log level change * [rpc] Add active status back to Validator information [validator] Fix typo for ValidatorRPCEnhanced [Makefile] Change make exe to build non-static by default * [network] Added caching to node server (#3048) * [network] Added caching to node server * [network] renamed two singleflight.Group * [network] added two error handling * [network] added forget logic for single flight * [network] changed single flight to lru cache solution * [network] adjusted the cache size * [network] changed cache size to 10000 * Transaction api improvements (#3057) * [node] Add option to broadcast invalid tx (on by default) * update main args to take in broadcast invalid tx * update node.sh for new broadcast invalid tx option * [rpc] Do not submit transaction if tx has been finalized * Support legacy behavior and not return errors on tx submission * [main] make broadcasting invalid tx default false * [node] Fix formatting * Add timeouts for the explorer HTTP service * [test] State trans unit test (#3043) * completing create-validator tests generalizing create validator tests, started with edit validator tests one round completed before major refactor * [test] finished coding test TestCheckDuplicateFields * [test] finished debug TestCheckDuplicateFields * [test] Added TestDescription_EnsureLength * [test] added test in validator_test.go TestUpdateDescription * [test] Added test TestComputed_String * [test] refactored TestValidatorSanityCheck and added some message in error message returned from sanity check * [test] removed test index for debugging * [test] Added test TestValidatorWrapper_SanityCheck. Added value check for TestMarshalUnmarshalValidator. Fixed rlp usage defect in UnmarshalValidator * [test] added test TestVerifyBLSKeys * [test] added test TestContainsHarmonyBLSKeys * [test] added test TestCreateValidatorFromNewMsg * [test] fixed some tests errors * [test] added test TestUpdateValidatorFromEditMsg, last editted the test file, finished testing validator_test.go * [staking] added numeric.Dec.Copy method * [staking] added CommissionRates.Copy * [staking] add nil case handle to copy method and related test case * [test] added nil case for commissionRates.Copy test * [staking] finished CreateValidator.Copy and related test case * [staking] added EditValidator.Copy method and related test case * [test] added zero test cases for Copy method * [staking] implemented Delegate.Copy and related unit test case * [staking] added Undelegate.Copy and CollectReward.Copy method. Implemented corresponding test cases * [test] added two more simple unit tests * [test] solving the merge conflict with deep copy fix * [test] added schedule for sharding * [test] refactored double_sign_test.go * [test] comment out all test cases for now. * [test] added record Copy method for testing * [test] added the first test case in TestVerify * [test] finished test TestVerify * [test] Added test TestVerify, TestApplySlashRate, TestSetDifference * [test] half through TestDelegatorSlashApply * [test] fix the problem cause by unsafe usage of big.Int * [staking] added the copy methods to validatorWrapper * [test] added test cases for copy method for staking/validatorWrapper * [test] added test case TestApply * [test] added test case TestRate * [test] fix the golint errors * [test] removed commented out unit test code * [test] remote the empty line in imports * [test] moved copy methods to stake_testing module * [test] removed usage of interface stateDB * [test] removed empty lines in imports * [test] fixed golint package name issues * [test] removed a todo when writing code * [test] moved record.Copy to staking/slash/test * [test] add some changes * [test] added prototypes in staketest module * [test] fix the golint issue in staketest * [test] make prototype more beautiful * [test] refactored test TestCheckDuplicateFields * [test] add createValidator test data * [test] added positive test case for VerifyAndCreateValidatorFromMsg * [test] added create validator test * [test] added the positive test case for TestVerifyAndEditValidatorFromMsg * [test] added the tests case TestVerifyAndEditValidatorFromMsg * [test] fix one of the error is not triggered as expected * [test] more changes in test * [test] fix the positive test data for TestVerifyAndDelegateFromMsg * [test] Fixed two comment golint errors * [test] added delegate tests * [test] added a make function. WIP * [test] added undelegate positive test * [test] added negative test cases for undelegate * [test] added positive test. not passed. Need to add assert method in test suites * [test] added equal function to check Validator Equality * [test] added equals for staketest * [test] replaced deep equal with new equal methods: * [test] removed unused codes * [test] Finishing touch * [test] fix comment golint issue * [test] removed dead code in staking_verifier_test.go Co-authored-by: Ganesha Upadhyaya <ganeshrvce@gmail.com> * changed the sync port to dns port in flag * [network] Limit client sync connection (#3071) * [network] limit client connected peers * Changed signature and added tests * Set secure permissions on generated blskey pass file in node.sh * Fix spellin gtypo in node.sh * fix latest param handling in api v1 * added comment for the isGreaterThanLatestBlockNum check * fix a nil pointer crash due to previous revert Signed-off-by: Leo Chen <leo@harmony.one> * fix nil pointer crash Signed-off-by: Leo Chen <leo@harmony.one> * fix nil pointer of block Signed-off-by: Leo Chen <leo@harmony.one> * [rpc] add viewID and epoch to RPCMarshalBlock (#3009) * [rpc] add viewID and epoch to RPCMarshalBlock * add apt-get update * Do not rollback current block on failed block sync (#3101) Co-authored-by: Ganesha Upadhyaya <ganeshrvce@gmail.com> Co-authored-by: Rongjian Lan <rongjian.lan@gmail.com> Co-authored-by: Jacky Wang <jackyw.se@gmail.com> Co-authored-by: Janet Liang <56005637+janet-harmony@users.noreply.github.com> Co-authored-by: Daniel Van Der Maden <dvandermaden0@berkeley.edu> Co-authored-by: Janet Liang <janet@harmony.one> Co-authored-by: Sebastian Johnsson <sebastian.johnsson@gmail.com> Co-authored-by: hsiung <hsiung@pinterest.com> Co-authored-by: Dennis Won <jhwon0820@gmail.com> Co-authored-by: Yishuang Chen <34232522+ivorytowerdds@users.noreply.github.com>
5 years ago
const (
headerCacheSize = 10000
blockCacheSize = 10000
blockWithSigCacheSize = 10000
Merge master to t3 0523 (#3085) * fix signers share computation to use only non-harmony nodes (#3008) * Add LastEpochInCommittee at prestaking epoch too (#3014) * write validator stats at the end of prestaking epoch (#3021) * write validator stats at the end of prestaking epoch * also need to write block reward accumulator * [build] enable static build by default (#2885) scripts/go_executable_build.sh -s (static build, default) scripts/go_executable_build.sh -S (non-static build) Signed-off-by: Leo Chen <leo@harmony.one> * Code formatting fix based on `staticcheck` (#3023) * fix some static check issues * more fixes according to staticcheck * add nil check on getHeader * Add more log info * [rpc] Revert change to v2 GetBalance RPC (#3037) * Cherrypick crosslink related fixes from t3 to master. (#3045) * Optimize crosslink verification logic and add more cache (#3032) * make crosslink broadcast smarter and more efficient (#3036) * adjust catch up speed to be a sane number * Fix crosslink broadcast condition (#3041) * Tx pool error report fix & local build fix (#3046) * [tx_list] Add transaction after you can fetch tx cost * [tx_pool] Add error sink reports for removed tx * Correct log levels for known transactions * Remove redundant hash to `enqueue` and `add` signatures * Update tests for signature changes * [build] Fix local build for MacOs * [tx_pool] Correct typo & logging level for known tx * [config] Create Testnet config * [node.sh] update testnet network config * [config] correct testnet config * [main] Enable beacon sync for explorer nodes (#3059) * [apr] should not use snapshot from last epoch, use current epoch snapshot (#3051) * use snapshot total delegation instead of current total delegation * should not use snapshot from last epoch, use current epoch snapshot (#3049) Co-authored-by: Rongjian Lan <rongjian.lan@gmail.com> Co-authored-by: Rongjian Lan <rongjian.lan@gmail.com> * cherrypick log level change * [rpc] Add active status back to Validator information [validator] Fix typo for ValidatorRPCEnhanced [Makefile] Change make exe to build non-static by default * [network] Added caching to node server (#3048) * [network] Added caching to node server * [network] renamed two singleflight.Group * [network] added two error handling * [network] added forget logic for single flight * [network] changed single flight to lru cache solution * [network] adjusted the cache size * [network] changed cache size to 10000 * Transaction api improvements (#3057) * [node] Add option to broadcast invalid tx (on by default) * update main args to take in broadcast invalid tx * update node.sh for new broadcast invalid tx option * [rpc] Do not submit transaction if tx has been finalized * Support legacy behavior and not return errors on tx submission * [main] make broadcasting invalid tx default false * [node] Fix formatting * Add timeouts for the explorer HTTP service * [test] State trans unit test (#3043) * completing create-validator tests generalizing create validator tests, started with edit validator tests one round completed before major refactor * [test] finished coding test TestCheckDuplicateFields * [test] finished debug TestCheckDuplicateFields * [test] Added TestDescription_EnsureLength * [test] added test in validator_test.go TestUpdateDescription * [test] Added test TestComputed_String * [test] refactored TestValidatorSanityCheck and added some message in error message returned from sanity check * [test] removed test index for debugging * [test] Added test TestValidatorWrapper_SanityCheck. Added value check for TestMarshalUnmarshalValidator. Fixed rlp usage defect in UnmarshalValidator * [test] added test TestVerifyBLSKeys * [test] added test TestContainsHarmonyBLSKeys * [test] added test TestCreateValidatorFromNewMsg * [test] fixed some tests errors * [test] added test TestUpdateValidatorFromEditMsg, last editted the test file, finished testing validator_test.go * [staking] added numeric.Dec.Copy method * [staking] added CommissionRates.Copy * [staking] add nil case handle to copy method and related test case * [test] added nil case for commissionRates.Copy test * [staking] finished CreateValidator.Copy and related test case * [staking] added EditValidator.Copy method and related test case * [test] added zero test cases for Copy method * [staking] implemented Delegate.Copy and related unit test case * [staking] added Undelegate.Copy and CollectReward.Copy method. Implemented corresponding test cases * [test] added two more simple unit tests * [test] solving the merge conflict with deep copy fix * [test] added schedule for sharding * [test] refactored double_sign_test.go * [test] comment out all test cases for now. * [test] added record Copy method for testing * [test] added the first test case in TestVerify * [test] finished test TestVerify * [test] Added test TestVerify, TestApplySlashRate, TestSetDifference * [test] half through TestDelegatorSlashApply * [test] fix the problem cause by unsafe usage of big.Int * [staking] added the copy methods to validatorWrapper * [test] added test cases for copy method for staking/validatorWrapper * [test] added test case TestApply * [test] added test case TestRate * [test] fix the golint errors * [test] removed commented out unit test code * [test] remote the empty line in imports * [test] moved copy methods to stake_testing module * [test] removed usage of interface stateDB * [test] removed empty lines in imports * [test] fixed golint package name issues * [test] removed a todo when writing code * [test] moved record.Copy to staking/slash/test * [test] add some changes * [test] added prototypes in staketest module * [test] fix the golint issue in staketest * [test] make prototype more beautiful * [test] refactored test TestCheckDuplicateFields * [test] add createValidator test data * [test] added positive test case for VerifyAndCreateValidatorFromMsg * [test] added create validator test * [test] added the positive test case for TestVerifyAndEditValidatorFromMsg * [test] added the tests case TestVerifyAndEditValidatorFromMsg * [test] fix one of the error is not triggered as expected * [test] more changes in test * [test] fix the positive test data for TestVerifyAndDelegateFromMsg * [test] Fixed two comment golint errors * [test] added delegate tests * [test] added a make function. WIP * [test] added undelegate positive test * [test] added negative test cases for undelegate * [test] added positive test. not passed. Need to add assert method in test suites * [test] added equal function to check Validator Equality * [test] added equals for staketest * [test] replaced deep equal with new equal methods: * [test] removed unused codes * [test] Finishing touch * [test] fix comment golint issue * [test] removed dead code in staking_verifier_test.go Co-authored-by: Ganesha Upadhyaya <ganeshrvce@gmail.com> * changed the sync port to dns port in flag * [network] Limit client sync connection (#3071) * [network] limit client connected peers * Changed signature and added tests * Set secure permissions on generated blskey pass file in node.sh * Fix spellin gtypo in node.sh * fix latest param handling in api v1 * added comment for the isGreaterThanLatestBlockNum check * fix a nil pointer crash due to previous revert Signed-off-by: Leo Chen <leo@harmony.one> * fix nil pointer crash Signed-off-by: Leo Chen <leo@harmony.one> * fix nil pointer of block Signed-off-by: Leo Chen <leo@harmony.one> * [rpc] add viewID and epoch to RPCMarshalBlock (#3009) * [rpc] add viewID and epoch to RPCMarshalBlock * add apt-get update * Do not rollback current block on failed block sync (#3101) Co-authored-by: Ganesha Upadhyaya <ganeshrvce@gmail.com> Co-authored-by: Rongjian Lan <rongjian.lan@gmail.com> Co-authored-by: Jacky Wang <jackyw.se@gmail.com> Co-authored-by: Janet Liang <56005637+janet-harmony@users.noreply.github.com> Co-authored-by: Daniel Van Der Maden <dvandermaden0@berkeley.edu> Co-authored-by: Janet Liang <janet@harmony.one> Co-authored-by: Sebastian Johnsson <sebastian.johnsson@gmail.com> Co-authored-by: hsiung <hsiung@pinterest.com> Co-authored-by: Dennis Won <jhwon0820@gmail.com> Co-authored-by: Yishuang Chen <34232522+ivorytowerdds@users.noreply.github.com>
5 years ago
)
var (
// Cached fields for block header and block requests
headerReqCache, _ = lru.New(headerCacheSize)
blockReqCache, _ = lru.New(blockCacheSize)
blockWithSigReqCache, _ = lru.New(blockWithSigCacheSize)
Merge master to t3 0523 (#3085) * fix signers share computation to use only non-harmony nodes (#3008) * Add LastEpochInCommittee at prestaking epoch too (#3014) * write validator stats at the end of prestaking epoch (#3021) * write validator stats at the end of prestaking epoch * also need to write block reward accumulator * [build] enable static build by default (#2885) scripts/go_executable_build.sh -s (static build, default) scripts/go_executable_build.sh -S (non-static build) Signed-off-by: Leo Chen <leo@harmony.one> * Code formatting fix based on `staticcheck` (#3023) * fix some static check issues * more fixes according to staticcheck * add nil check on getHeader * Add more log info * [rpc] Revert change to v2 GetBalance RPC (#3037) * Cherrypick crosslink related fixes from t3 to master. (#3045) * Optimize crosslink verification logic and add more cache (#3032) * make crosslink broadcast smarter and more efficient (#3036) * adjust catch up speed to be a sane number * Fix crosslink broadcast condition (#3041) * Tx pool error report fix & local build fix (#3046) * [tx_list] Add transaction after you can fetch tx cost * [tx_pool] Add error sink reports for removed tx * Correct log levels for known transactions * Remove redundant hash to `enqueue` and `add` signatures * Update tests for signature changes * [build] Fix local build for MacOs * [tx_pool] Correct typo & logging level for known tx * [config] Create Testnet config * [node.sh] update testnet network config * [config] correct testnet config * [main] Enable beacon sync for explorer nodes (#3059) * [apr] should not use snapshot from last epoch, use current epoch snapshot (#3051) * use snapshot total delegation instead of current total delegation * should not use snapshot from last epoch, use current epoch snapshot (#3049) Co-authored-by: Rongjian Lan <rongjian.lan@gmail.com> Co-authored-by: Rongjian Lan <rongjian.lan@gmail.com> * cherrypick log level change * [rpc] Add active status back to Validator information [validator] Fix typo for ValidatorRPCEnhanced [Makefile] Change make exe to build non-static by default * [network] Added caching to node server (#3048) * [network] Added caching to node server * [network] renamed two singleflight.Group * [network] added two error handling * [network] added forget logic for single flight * [network] changed single flight to lru cache solution * [network] adjusted the cache size * [network] changed cache size to 10000 * Transaction api improvements (#3057) * [node] Add option to broadcast invalid tx (on by default) * update main args to take in broadcast invalid tx * update node.sh for new broadcast invalid tx option * [rpc] Do not submit transaction if tx has been finalized * Support legacy behavior and not return errors on tx submission * [main] make broadcasting invalid tx default false * [node] Fix formatting * Add timeouts for the explorer HTTP service * [test] State trans unit test (#3043) * completing create-validator tests generalizing create validator tests, started with edit validator tests one round completed before major refactor * [test] finished coding test TestCheckDuplicateFields * [test] finished debug TestCheckDuplicateFields * [test] Added TestDescription_EnsureLength * [test] added test in validator_test.go TestUpdateDescription * [test] Added test TestComputed_String * [test] refactored TestValidatorSanityCheck and added some message in error message returned from sanity check * [test] removed test index for debugging * [test] Added test TestValidatorWrapper_SanityCheck. Added value check for TestMarshalUnmarshalValidator. Fixed rlp usage defect in UnmarshalValidator * [test] added test TestVerifyBLSKeys * [test] added test TestContainsHarmonyBLSKeys * [test] added test TestCreateValidatorFromNewMsg * [test] fixed some tests errors * [test] added test TestUpdateValidatorFromEditMsg, last editted the test file, finished testing validator_test.go * [staking] added numeric.Dec.Copy method * [staking] added CommissionRates.Copy * [staking] add nil case handle to copy method and related test case * [test] added nil case for commissionRates.Copy test * [staking] finished CreateValidator.Copy and related test case * [staking] added EditValidator.Copy method and related test case * [test] added zero test cases for Copy method * [staking] implemented Delegate.Copy and related unit test case * [staking] added Undelegate.Copy and CollectReward.Copy method. Implemented corresponding test cases * [test] added two more simple unit tests * [test] solving the merge conflict with deep copy fix * [test] added schedule for sharding * [test] refactored double_sign_test.go * [test] comment out all test cases for now. * [test] added record Copy method for testing * [test] added the first test case in TestVerify * [test] finished test TestVerify * [test] Added test TestVerify, TestApplySlashRate, TestSetDifference * [test] half through TestDelegatorSlashApply * [test] fix the problem cause by unsafe usage of big.Int * [staking] added the copy methods to validatorWrapper * [test] added test cases for copy method for staking/validatorWrapper * [test] added test case TestApply * [test] added test case TestRate * [test] fix the golint errors * [test] removed commented out unit test code * [test] remote the empty line in imports * [test] moved copy methods to stake_testing module * [test] removed usage of interface stateDB * [test] removed empty lines in imports * [test] fixed golint package name issues * [test] removed a todo when writing code * [test] moved record.Copy to staking/slash/test * [test] add some changes * [test] added prototypes in staketest module * [test] fix the golint issue in staketest * [test] make prototype more beautiful * [test] refactored test TestCheckDuplicateFields * [test] add createValidator test data * [test] added positive test case for VerifyAndCreateValidatorFromMsg * [test] added create validator test * [test] added the positive test case for TestVerifyAndEditValidatorFromMsg * [test] added the tests case TestVerifyAndEditValidatorFromMsg * [test] fix one of the error is not triggered as expected * [test] more changes in test * [test] fix the positive test data for TestVerifyAndDelegateFromMsg * [test] Fixed two comment golint errors * [test] added delegate tests * [test] added a make function. WIP * [test] added undelegate positive test * [test] added negative test cases for undelegate * [test] added positive test. not passed. Need to add assert method in test suites * [test] added equal function to check Validator Equality * [test] added equals for staketest * [test] replaced deep equal with new equal methods: * [test] removed unused codes * [test] Finishing touch * [test] fix comment golint issue * [test] removed dead code in staking_verifier_test.go Co-authored-by: Ganesha Upadhyaya <ganeshrvce@gmail.com> * changed the sync port to dns port in flag * [network] Limit client sync connection (#3071) * [network] limit client connected peers * Changed signature and added tests * Set secure permissions on generated blskey pass file in node.sh * Fix spellin gtypo in node.sh * fix latest param handling in api v1 * added comment for the isGreaterThanLatestBlockNum check * fix a nil pointer crash due to previous revert Signed-off-by: Leo Chen <leo@harmony.one> * fix nil pointer crash Signed-off-by: Leo Chen <leo@harmony.one> * fix nil pointer of block Signed-off-by: Leo Chen <leo@harmony.one> * [rpc] add viewID and epoch to RPCMarshalBlock (#3009) * [rpc] add viewID and epoch to RPCMarshalBlock * add apt-get update * Do not rollback current block on failed block sync (#3101) Co-authored-by: Ganesha Upadhyaya <ganeshrvce@gmail.com> Co-authored-by: Rongjian Lan <rongjian.lan@gmail.com> Co-authored-by: Jacky Wang <jackyw.se@gmail.com> Co-authored-by: Janet Liang <56005637+janet-harmony@users.noreply.github.com> Co-authored-by: Daniel Van Der Maden <dvandermaden0@berkeley.edu> Co-authored-by: Janet Liang <janet@harmony.one> Co-authored-by: Sebastian Johnsson <sebastian.johnsson@gmail.com> Co-authored-by: hsiung <hsiung@pinterest.com> Co-authored-by: Dennis Won <jhwon0820@gmail.com> Co-authored-by: Yishuang Chen <34232522+ivorytowerdds@users.noreply.github.com>
5 years ago
errHeaderNotExist = errors.New("header not exist")
errBlockNotExist = errors.New("block not exist")
)
func (node *Node) getEncodedBlockHeaderByHash(hash common.Hash) ([]byte, error) {
if b, ok := headerReqCache.Get(hash); ok {
return b.([]byte), nil
}
h := node.Blockchain().GetHeaderByHash(hash)
if h == nil {
return nil, errHeaderNotExist
}
b, err := rlp.EncodeToBytes(h)
if err != nil {
return nil, err
}
headerReqCache.Add(hash, b)
return b, nil
}
func (node *Node) getEncodedBlockByHash(hash common.Hash) ([]byte, error) {
if b, ok := blockReqCache.Get(hash); ok {
return b.([]byte), nil
}
blk := node.Blockchain().GetBlockByHash(hash)
if blk == nil {
return nil, errBlockNotExist
}
b, err := rlp.EncodeToBytes(blk)
if err != nil {
return nil, err
}
blockReqCache.Add(hash, b)
return b, nil
}
func (node *Node) getEncodedBlockWithSigByHash(hash common.Hash) ([]byte, error) {
if b, ok := blockWithSigReqCache.Get(hash); ok {
return b.([]byte), nil
}
blk := node.Blockchain().GetBlockByHash(hash)
if blk == nil {
return nil, errBlockNotExist
}
sab, err := node.getCommitSigAndBitmap(blk)
if err != nil {
return nil, err
}
bwh := legacysync.BlockWithSig{
Block: blk,
CommitSigAndBitmap: sab,
}
b, err := rlp.EncodeToBytes(bwh)
if err != nil {
return nil, err
}
blockWithSigReqCache.Add(hash, b)
return b, nil
}
func (node *Node) getCommitSigAndBitmap(block *types.Block) ([]byte, error) {
child := node.Blockchain().GetBlockByNumber(block.NumberU64() + 1)
if child != nil {
return node.getCommitSigFromChild(block, child)
}
return node.getCommitSigFromDB(block)
}
func (node *Node) getCommitSigFromChild(parent, child *types.Block) ([]byte, error) {
if child.ParentHash() != parent.Hash() {
return nil, fmt.Errorf("child's parent hash unexpected: %v / %v",
child.ParentHash().String(), parent.Hash().String())
}
sig := child.Header().LastCommitSignature()
bitmap := child.Header().LastCommitBitmap()
return append(sig[:], bitmap...), nil
}
func (node *Node) getCommitSigFromDB(block *types.Block) ([]byte, error) {
return node.Blockchain().ReadCommitSig(block.NumberU64())
}
// SyncStatus return the syncing status, including whether node is syncing
// and the target block number.
func (node *Node) SyncStatus(shardID uint32) (bool, uint64) {
ds := node.getDownloaders()
if ds == nil {
return false, 0
}
return ds.SyncStatus(shardID)
}
// IsOutOfSync return whether the node is out of sync of the given hsardID
func (node *Node) IsOutOfSync(shardID uint32) bool {
ds := node.getDownloaders()
if ds == nil {
return false
}
isSyncing, _ := ds.SyncStatus(shardID)
return !isSyncing
Rosetta Implementation - pt2 (Stage 3.2 of Node API Overhaul) (#3312) * [rosetta] Add server stop Signed-off-by: Daniel Van Der Maden <dvandermaden0@berkeley.edu> * [rosetta] Make network naming consistent Signed-off-by: Daniel Van Der Maden <dvandermaden0@berkeley.edu> * [rosetta] Correct common package name & add error enum Signed-off-by: Daniel Van Der Maden <dvandermaden0@berkeley.edu> * [rosetta] Remove needless forward of network info to services * Implement /network/list Signed-off-by: Daniel Van Der Maden <dvandermaden0@berkeley.edu> * [rosetta] Refactor errors & add operation statuses and types Signed-off-by: Daniel Van Der Maden <dvandermaden0@berkeley.edu> * [rosetta] Implement NetworkOptions & update NetworkAPIService * Rename *_service.go files to remove the suffix * Update StartServers to use new operation types Signed-off-by: Daniel Van Der Maden <dvandermaden0@berkeley.edu> * [rosetta] Impl NetworkStatus - Finish init impl of /network endpoint * Fix import structure for rosetta.go Signed-off-by: Daniel Van Der Maden <dvandermaden0@berkeley.edu> * [test] Make explorer run as archival for localnet Signed-off-by: Daniel Van Der Maden <dvandermaden0@berkeley.edu> * [rosetta] Add unit tests * Force errors to remain the same with unit tests * Force operations to remain the same with unit tests * Ensure network checking works for all cases with unit tests Signed-off-by: Daniel Van Der Maden <dvandermaden0@berkeley.edu> * [rosetta] Add InvalidNetworkError and correct error codes Signed-off-by: Daniel Van Der Maden <dvandermaden0@berkeley.edu> * [rosetta] Add UnmarshalFromInterface for SubNetworkMetadata Signed-off-by: Daniel Van Der Maden <dvandermaden0@berkeley.edu> * [rosetta] Add network checking Signed-off-by: Daniel Van Der Maden <dvandermaden0@berkeley.edu> * [rosetta] Nit fixes & add unit test for Peer Info * Make names consistent Signed-off-by: Daniel Van Der Maden <dvandermaden0@berkeley.edu> * [rosetta] Add BlockNotFoundError & TransactionNotFoundError Signed-off-by: Daniel Van Der Maden <dvandermaden0@berkeley.edu> * [rosetta] Implement skeleton for block transactions Signed-off-by: Daniel Van Der Maden <dvandermaden0@berkeley.edu> * [rosetta] Add ReceiptNotFoundError Signed-off-by: Daniel Van Der Maden <dvandermaden0@berkeley.edu> * [rosetta] Add receipt to formatTransaction sig for contract fails Signed-off-by: Daniel Van Der Maden <dvandermaden0@berkeley.edu> * [rosetta] Add currency, ExpendGasOperation, & ContractCreationOperation * Add Error creator Signed-off-by: Daniel Van Der Maden <dvandermaden0@berkeley.edu> * [rosetta] Init impl of plain transaction formatting Signed-off-by: Daniel Van Der Maden <dvandermaden0@berkeley.edu> * [rosetta] Update network.go for new error constructor Signed-off-by: Daniel Van Der Maden <dvandermaden0@berkeley.edu> * [rosetta] Implement stx formatter & refactor BlockTransaction * Updated todo comments & function formatting Signed-off-by: Daniel Van Der Maden <dvandermaden0@berkeley.edu> * [rosetta] Impl Block & make currency non-ptr for easy copy with custom metadata Signed-off-by: Daniel Van Der Maden <dvandermaden0@berkeley.edu> * [rosetta] Fix collect rewards amount on transaction fetch Signed-off-by: Daniel Van Der Maden <dvandermaden0@berkeley.edu> * [rosetta] Fix block look-up edge case & add recovery middleware Signed-off-by: Daniel Van Der Maden <dvandermaden0@berkeley.edu> * [rosetta] Add bocks unit tests Signed-off-by: Daniel Van Der Maden <dvandermaden0@berkeley.edu> * [rosetta] Fix checkPeerID unit test in network_test.go Signed-off-by: Daniel Van Der Maden <dvandermaden0@berkeley.edu> * [rosetta] Fix staking tx amount for tx ops & update inline docs Signed-off-by: Daniel Van Der Maden <dvandermaden0@berkeley.edu> * [rosetta] Fix lint Signed-off-by: Daniel Van Der Maden <dvandermaden0@berkeley.edu> * [rosetta] Refactor getStakingOperations Signed-off-by: Daniel Van Der Maden <dvandermaden0@berkeley.edu> * [rosetta] Fix undelegate value Signed-off-by: Daniel Van Der Maden <dvandermaden0@berkeley.edu> * [rosetta] Nit - fix formatting for network.go Signed-off-by: Daniel Van Der Maden <dvandermaden0@berkeley.edu> * [node] Move genesis allocation to core & remove unused ContractDeployerKey Signed-off-by: Daniel Van Der Maden <dvandermaden0@berkeley.edu> * [rosetta] Fix precision error & add cx receipt hash on blk fetch * Add unit tests for supporting helper functions Signed-off-by: Daniel Van Der Maden <dvandermaden0@berkeley.edu> * [rosetta] Fix fmt Signed-off-by: Daniel Van Der Maden <dvandermaden0@berkeley.edu> * [rosetta] Fix cx receipt hashes for blocks * Print stack trace on panic recovery Signed-off-by: Daniel Van Der Maden <dvandermaden0@berkeley.edu> * [node] Nit - fix comment for StopRosetta Signed-off-by: Daniel Van Der Maden <dvandermaden0@berkeley.edu> * [node] Expose GetMaxPeerHeight Signed-off-by: Daniel Van Der Maden <dvandermaden0@berkeley.edu> * [rosetta] Add SyncStatus enum Signed-off-by: Daniel Van Der Maden <dvandermaden0@berkeley.edu> * [rosetta] Nit - remove redundant 'service' name in services namespace Signed-off-by: Daniel Van Der Maden <dvandermaden0@berkeley.edu>
4 years ago
}
// SyncPeers return connected sync peers for each shard
func (node *Node) SyncPeers() map[string]int {
ds := node.getDownloaders()
if ds == nil {
return nil
}
nums := ds.NumPeers()
res := make(map[string]int)
for sid, num := range nums {
s := fmt.Sprintf("shard-%v", sid)
res[s] = num
}
return res
}
func (node *Node) getDownloaders() *downloader.Downloaders {
syncService := node.serviceManager.GetService(service.Synchronize)
if syncService == nil {
return nil
}
dsService, ok := syncService.(*synchronize.Service)
if !ok {
return nil
}
return dsService.Downloaders
}