From 363da70a67912a498dcbcd6dca034491067bc86e Mon Sep 17 00:00:00 2001 From: DarkLord017 Date: Sat, 21 Sep 2024 23:11:01 +0530 Subject: [PATCH 1/4] Remove utils from consensus and move to utils --- consensus/consensus.go | 47 ++++---- consensus/utils.go | 237 ----------------------------------------- 2 files changed, 24 insertions(+), 260 deletions(-) delete mode 100644 consensus/utils.go diff --git a/consensus/consensus.go b/consensus/consensus.go index a685ffa..2dfe4f9 100644 --- a/consensus/consensus.go +++ b/consensus/consensus.go @@ -21,6 +21,7 @@ import ( "github.com/BlocSoc-iitr/selene/config/checkpoints" "github.com/BlocSoc-iitr/selene/consensus/consensus_core" "github.com/BlocSoc-iitr/selene/consensus/rpc" + "github.com/BlocSoc-iitr/selene/utils" "github.com/BlocSoc-iitr/selene/utils/bls" geth "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/core/types" @@ -254,7 +255,7 @@ func (in *Inner) get_execution_payload(ctx context.Context, slot *uint64) (*cons return nil, err } - blockHash, err := TreeHashRoot(block.Body.ToBytes()) + blockHash, err := utils.TreeHashRoot(block.Body.ToBytes()) if err != nil { return nil, err } @@ -265,12 +266,12 @@ func (in *Inner) get_execution_payload(ctx context.Context, slot *uint64) (*cons var errGettingBlockHash error if *slot == latestSlot { - verifiedBlockHash, errGettingBlockHash = TreeHashRoot(in.Store.OptimisticHeader.ToBytes()) + verifiedBlockHash, errGettingBlockHash = utils.TreeHashRoot(in.Store.OptimisticHeader.ToBytes()) if errGettingBlockHash != nil { return nil, ErrPayloadNotFound } } else if *slot == finalizedSlot { - verifiedBlockHash, errGettingBlockHash = TreeHashRoot(in.Store.FinalizedHeader.ToBytes()) + verifiedBlockHash, errGettingBlockHash = utils.TreeHashRoot(in.Store.FinalizedHeader.ToBytes()) if errGettingBlockHash != nil { return nil, ErrPayloadNotFound } @@ -368,7 +369,7 @@ func (in *Inner) advance() error { if in.Store.NextSyncCommitee == nil { log.Printf("checking for sync committee update") - currentPeriod := CalcSyncPeriod(in.Store.FinalizedHeader.Slot) + currentPeriod := utils.CalcSyncPeriod(in.Store.FinalizedHeader.Slot) updates, err := in.RPC.GetUpdates(currentPeriod, 1) if err != nil { return err @@ -394,7 +395,7 @@ func (in *Inner) sync(checkpoint [32]byte) error { in.bootstrap(checkpoint) // Calculate the current sync period - currentPeriod := CalcSyncPeriod(in.Store.FinalizedHeader.Slot) + currentPeriod := utils.CalcSyncPeriod(in.Store.FinalizedHeader.Slot) // Fetch updates updates, err := in.RPC.GetUpdates(currentPeriod, MAX_REQUEST_LIGHT_CLIENT_UPDATES) @@ -520,7 +521,7 @@ func verify_bootstrap(checkpoint [32]byte, bootstrap consensus_core.Bootstrap) { return } - headerHash, err := TreeHashRoot(bootstrap.Header.ToBytes()) + headerHash, err := utils.TreeHashRoot(bootstrap.Header.ToBytes()) if err != nil { log.Println("failed to hash header") return @@ -560,8 +561,8 @@ func (in *Inner) verify_generic_update(update *GenericUpdate, expectedCurrentSlo return ErrInvalidTimestamp } - storePeriod := CalcSyncPeriod(store.FinalizedHeader.Slot) - updateSigPeriod := CalcSyncPeriod(update.SignatureSlot) + storePeriod := utils.CalcSyncPeriod(store.FinalizedHeader.Slot) + updateSigPeriod := utils.CalcSyncPeriod(update.SignatureSlot) var validPeriod bool if store.NextSyncCommitee != nil { @@ -574,7 +575,7 @@ func (in *Inner) verify_generic_update(update *GenericUpdate, expectedCurrentSlo return ErrInvalidPeriod } - updateAttestedPeriod := CalcSyncPeriod(update.AttestedHeader.Slot) + updateAttestedPeriod := utils.CalcSyncPeriod(update.AttestedHeader.Slot) updateHasNextCommittee := store.NextSyncCommitee == nil && update.NextSyncCommittee != nil && updateAttestedPeriod == storePeriod if update.AttestedHeader.Slot <= store.FinalizedHeader.Slot && !updateHasNextCommittee { @@ -606,13 +607,13 @@ func (in *Inner) verify_generic_update(update *GenericUpdate, expectedCurrentSlo } else { syncCommittee = in.Store.NextSyncCommitee } - pks, err := GetParticipatingKeys(syncCommittee, update.SyncAggregate.SyncCommitteeBits) + pks, err := utils.GetParticipatingKeys(syncCommittee, update.SyncAggregate.SyncCommitteeBits) if err != nil { return fmt.Errorf("failed to get participating keys: %w", err) } - forkVersion := CalculateForkVersion(&forks, update.SignatureSlot) - forkDataRoot := ComputeForkDataRoot(forkVersion, consensus_core.Bytes32(in.Config.Chain.GenesisRoot)) + forkVersion := utils.CalculateForkVersion(&forks, update.SignatureSlot) + forkDataRoot := utils.ComputeForkDataRoot(forkVersion, consensus_core.Bytes32(in.Config.Chain.GenesisRoot)) if !verifySyncCommitteeSignature(pks, &update.AttestedHeader, &update.SyncAggregate.SyncCommitteeSignature, forkDataRoot) { return ErrInvalidSignature @@ -667,13 +668,13 @@ func (in *Inner) apply_generic_update(store *LightClientStore, update *GenericUp store.OptimisticHeader = update.AttestedHeader } - updateAttestedPeriod := CalcSyncPeriod(update.AttestedHeader.Slot) + updateAttestedPeriod := utils.CalcSyncPeriod(update.AttestedHeader.Slot) updateFinalizedSlot := uint64(0) if update.FinalizedHeader != (consensus_core.Header{}) { updateFinalizedSlot = update.FinalizedHeader.Slot } - updateFinalizedPeriod := CalcSyncPeriod(updateFinalizedSlot) + updateFinalizedPeriod := utils.CalcSyncPeriod(updateFinalizedSlot) updateHasFinalizedNextCommittee := in.Store.NextSyncCommitee == nil && in.has_sync_update(update) && in.has_finality_update(update) && @@ -692,7 +693,7 @@ func (in *Inner) apply_generic_update(store *LightClientStore, update *GenericUp // Apply the update if conditions are met if shouldApplyUpdate { - storePeriod := CalcSyncPeriod(store.FinalizedHeader.Slot) + storePeriod := utils.CalcSyncPeriod(store.FinalizedHeader.Slot) // Sync committee update logic if store.NextSyncCommitee == nil { @@ -714,7 +715,7 @@ func (in *Inner) apply_generic_update(store *LightClientStore, update *GenericUp } if store.FinalizedHeader.Slot%32 == 0 { - checkpoint, err := TreeHashRoot(store.FinalizedHeader.ToBytes()) + checkpoint, err := utils.TreeHashRoot(store.FinalizedHeader.ToBytes()) if err != nil { return nil } @@ -824,7 +825,7 @@ func verifySyncCommitteeSignature( } // Compute headerRoot - headerRoot, err := TreeHashRoot(attestedHeader.ToBytes()) + headerRoot, err := utils.TreeHashRoot(attestedHeader.ToBytes()) if err != nil { return false } @@ -842,7 +843,7 @@ func verifySyncCommitteeSignature( } } - return isAggregateValid(*signature, signingRoot, g2Points) + return utils.IsAggregateValid(*signature, signingRoot, g2Points) } func ComputeCommitteeSignRoot(header consensus_core.Bytes32, fork consensus_core.Bytes32) consensus_core.Bytes32 { @@ -850,10 +851,10 @@ func ComputeCommitteeSignRoot(header consensus_core.Bytes32, fork consensus_core domainType := [4]byte{7, 0, 0, 0} // Compute the domain - domain := ComputeDomain(domainType, fork) + domain := utils.ComputeDomain(domainType, fork) // Compute and return the signing root - return ComputeSigningRoot(header, domain) + return utils.ComputeSigningRoot(header, domain) } func (in *Inner) Age(slot uint64) time.Duration { expectedTime := slot*12 + in.Config.Chain.GenesisTime @@ -883,7 +884,7 @@ func isFinalityProofValid(attestedHeader *consensus_core.Header, finalizedHeader if err != nil { return false } - return isProofValid(attestedHeader, finalizedHeader.ToBytes(), finalityBranchForProof, 6, 41) + return utils.IsProofValid(attestedHeader, finalizedHeader.ToBytes(), finalityBranchForProof, 6, 41) } func isCurrentCommitteeProofValid(attestedHeader *consensus_core.Header, currentCommittee *consensus_core.SyncCommittee, currentCommitteeBranch []consensus_core.Bytes32) bool { @@ -891,7 +892,7 @@ func isCurrentCommitteeProofValid(attestedHeader *consensus_core.Header, current if err != nil { return false } - return isProofValid(attestedHeader, currentCommittee.ToBytes(), CurrentCommitteeForProof, 5, 22) + return utils.IsProofValid(attestedHeader, currentCommittee.ToBytes(), CurrentCommitteeForProof, 5, 22) } func isNextCommitteeProofValid(attestedHeader *consensus_core.Header, currentCommittee *consensus_core.SyncCommittee, currentCommitteeBranch []consensus_core.Bytes32) bool { @@ -899,7 +900,7 @@ func isNextCommitteeProofValid(attestedHeader *consensus_core.Header, currentCom if err != nil { return false } - return isProofValid(attestedHeader, currentCommittee.ToBytes(), currentCommitteeBranchForProof, 5, 23) + return utils.IsProofValid(attestedHeader, currentCommittee.ToBytes(), currentCommitteeBranchForProof, 5, 23) } func PayloadToBlock(value *consensus_core.ExecutionPayload) (*common.Block, error) { diff --git a/consensus/utils.go b/consensus/utils.go deleted file mode 100644 index 303ca7a..0000000 --- a/consensus/utils.go +++ /dev/null @@ -1,237 +0,0 @@ -package consensus - -import ( - "bytes" - "crypto/sha256" - "encoding/json" - "fmt" - "log" - - "github.com/BlocSoc-iitr/selene/config" - "github.com/BlocSoc-iitr/selene/consensus/consensus_core" - "github.com/BlocSoc-iitr/selene/utils/bls" - - bls12381 "github.com/consensys/gnark-crypto/ecc/bls12-381" - "github.com/pkg/errors" - merkletree "github.com/wealdtech/go-merkletree" -) - -// TreeHashRoot computes the Merkle root from the provided leaves in a flat []byte slice. -// TreeHashRoot calculates the root hash from the input data. -func TreeHashRoot(data []byte) ([]byte, error) { - // Convert the input data into a slice of leaves - leaves, err := bytesToLeaves(data) - if err != nil { - return nil, fmt.Errorf("error converting bytes to leaves: %w", err) - } - - // Create the Merkle tree using the leaves - tree, errorCreatingMerkleTree := merkletree.New(leaves) - if errorCreatingMerkleTree != nil { - return nil, fmt.Errorf("error creating Merkle tree: %w", err) - } - - // Fetch the root hash of the tree - root := tree.Root() - if root == nil { - return nil, errors.New("failed to calculate the Merkle root: root is nil") - } - - return root, nil -} - -func bytesToLeaves(data []byte) ([][]byte, error) { - var leaves [][]byte - if err := json.Unmarshal(data, &leaves); err != nil { - return nil, err - } - - return leaves, nil -} - -func CalcSyncPeriod(slot uint64) uint64 { - epoch := slot / 32 - return epoch / 256 -} - -// isAggregateValid checks if the provided signature is valid for the given message and public keys. -func isAggregateValid(sigBytes consensus_core.SignatureBytes, msg [32]byte, pks []*bls.G2Point) bool { - var sigInBytes [96]byte - copy(sigInBytes[:], sigBytes[:]) - // Deserialize the signature from bytes - var sig bls12381.G1Affine - if err := sig.Unmarshal(sigInBytes[:]); err != nil { - return false - } - - // Map the message to a point on the curve - msgPoint := bls.MapToCurve(msg) - - // Aggregate the public keys - aggPubKey := bls.AggregatePublicKeys(pks) - - // Prepare the pairing check inputs - P := [2]bls12381.G1Affine{*msgPoint, sig} - Q := [2]bls12381.G2Affine{*aggPubKey.G2Affine, *bls.GetG2Generator()} - - // Perform the pairing check - ok, err := bls12381.PairingCheck(P[:], Q[:]) - if err != nil { - return false - } - return ok -} - -func isProofValid( - attestedHeader *consensus_core.Header, - leafObject []byte, // Single byte slice of the leaf object - branch [][]byte, // Slice of byte slices for the branch - depth, index int, // Depth of the Merkle proof and index of the leaf -) bool { - // If the branch length is not equal to the depth, return false - if len(branch) != depth { - return false - } - - // Initialize the derived root as the leaf object's hash - derivedRoot, errFetchingTreeHashRoot := TreeHashRoot(leafObject) - - if errFetchingTreeHashRoot != nil { - fmt.Printf("Error fetching tree hash root: %v", errFetchingTreeHashRoot) - return false - } - - // Iterate over the proof's hashes - for i, hash := range branch { - hasher := sha256.New() - - // Use the index to determine how to combine the hashes - if (index>>i)&1 == 1 { - // If index is odd, hash node || derivedRoot - hasher.Write(hash) - hasher.Write(derivedRoot) - } else { - // If index is even, hash derivedRoot || node - hasher.Write(derivedRoot) - hasher.Write(hash) - } - // Update derivedRoot for the next level - derivedRoot = hasher.Sum(nil) - } - - // Compare the final derived root with the attested header's state root - return bytes.Equal(derivedRoot, attestedHeader.StateRoot[:]) -} - -func CalculateForkVersion(forks *config.Forks, slot uint64) [4]byte { - epoch := slot / 32 - - switch { - case epoch >= forks.Deneb.Epoch: - return [4]byte(forks.Deneb.ForkVersion) - case epoch >= forks.Capella.Epoch: - return [4]byte(forks.Capella.ForkVersion) - case epoch >= forks.Bellatrix.Epoch: - return [4]byte(forks.Bellatrix.ForkVersion) - case epoch >= forks.Altair.Epoch: - return [4]byte(forks.Altair.ForkVersion) - default: - return [4]byte(forks.Genesis.ForkVersion) - } -} - -func ComputeForkDataRoot(currentVersion [4]byte, genesisValidatorRoot consensus_core.Bytes32) consensus_core.Bytes32 { - forkData := ForkData{ - CurrentVersion: currentVersion, - GenesisValidatorRoot: genesisValidatorRoot, - } - - hash, err := TreeHashRoot(forkData.ToBytes()) - if err != nil { - return consensus_core.Bytes32{} - } - return consensus_core.Bytes32(hash) -} - -// GetParticipatingKeys retrieves the participating public keys from the committee based on the bitfield represented as a byte array. -func GetParticipatingKeys(committee *consensus_core.SyncCommittee, bitfield [64]byte) ([]consensus_core.BLSPubKey, error) { - var pks []consensus_core.BLSPubKey - numBits := len(bitfield) * 8 // Total number of bits - - if len(committee.Pubkeys) > numBits { - return nil, fmt.Errorf("bitfield is too short for the number of public keys") - } - - for i := 0; i < len(bitfield); i++ { - byteVal := bitfield[i] - for bit := 0; bit < 8; bit++ { - if (byteVal & (1 << bit)) != 0 { - index := i*8 + bit - if index >= len(committee.Pubkeys) { - break - } - pks = append(pks, committee.Pubkeys[index]) - } - } - } - - return pks, nil -} - -func ComputeSigningRoot(objectRoot, domain consensus_core.Bytes32) consensus_core.Bytes32 { - signingData := SigningData{ - ObjectRoot: objectRoot, - Domain: domain, - } - hash, err := TreeHashRoot(signingData.ToBytes()) - if err != nil { - return consensus_core.Bytes32{} - } - return consensus_core.Bytes32(hash) -} - -func ComputeDomain(domainType [4]byte, forkDataRoot consensus_core.Bytes32) consensus_core.Bytes32 { - data := append(domainType[:], forkDataRoot[:28]...) - return sha256.Sum256(data) -} - -type SigningData struct { - ObjectRoot consensus_core.Bytes32 - Domain consensus_core.Bytes32 -} - -type ForkData struct { - CurrentVersion [4]byte - GenesisValidatorRoot consensus_core.Bytes32 -} - -func (fd *ForkData) ToBytes() []byte { - data, err := json.Marshal(fd) - if err != nil { - log.Println("Error marshaling ForkData:", err) - return nil // Or return an empty slice, based on your preference - } - return data -} - -func (sd *SigningData) ToBytes() []byte { - data, err := json.Marshal(sd) - if err != nil { - log.Println("Error marshaling SigningData:", err) - return nil // Or return an empty slice, based on your preference - } - return data -} - -func bytes32ToNode(bytes consensus_core.Bytes32) []byte { - return []byte(bytes[:]) -} - -// branchToNodes converts a slice of Bytes32 to a slice of Node -func branchToNodes(branch []consensus_core.Bytes32) ([][]byte, error) { - nodes := make([][]byte, len(branch)) - for i, b32 := range branch { - nodes[i] = bytes32ToNode(b32) - } - return nodes, nil -} From 94eec89e56de99f66c0b97238a6a6bff85e40952 Mon Sep 17 00:00:00 2001 From: DarkLord017 Date: Sun, 22 Sep 2024 08:51:19 +0530 Subject: [PATCH 2/4] Moved utils of consensus too --- config/base.go | 26 +-- config/config.go | 29 +-- config/networks.go | 37 ++-- config/types.go | 39 +--- consensus/consensus.go | 8 +- consensus/consensus_core/consensus_core.go | 12 ++ utils/utils.go | 234 +++++++++++++++++++++ 7 files changed, 301 insertions(+), 84 deletions(-) diff --git a/config/base.go b/config/base.go index 98afea0..d3419ba 100644 --- a/config/base.go +++ b/config/base.go @@ -1,18 +1,20 @@ package config +import "github.com/BlocSoc-iitr/selene/consensus/consensus_core" + // base config for a network type BaseConfig struct { - RpcBindIp string `json:"rpc_bind_ip"` - RpcPort uint16 `json:"rpc_port"` - ConsensusRpc *string `json:"consensus_rpc"` - DefaultCheckpoint [32]byte `json:"default_checkpoint"` // In cli.go, checkpoint is currently taken as []byte{} - Chain ChainConfig `json:"chain"` // but it should be [32]byte as it is a hash - Forks Forks `json:"forks"` - MaxCheckpointAge uint64 `json:"max_checkpoint_age"` - DataDir *string `json:"data_dir"` - LoadExternalFallback bool `json:"load_external_fallback"` - StrictCheckpointAge bool `json:"strict_checkpoint_age"` + RpcBindIp string `json:"rpc_bind_ip"` + RpcPort uint16 `json:"rpc_port"` + ConsensusRpc *string `json:"consensus_rpc"` + DefaultCheckpoint [32]byte `json:"default_checkpoint"` // In cli.go, checkpoint is currently taken as []byte{} + Chain ChainConfig `json:"chain"` // but it should be [32]byte as it is a hash + Forks consensus_core.Forks `json:"forks"` + MaxCheckpointAge uint64 `json:"max_checkpoint_age"` + DataDir *string `json:"data_dir"` + LoadExternalFallback bool `json:"load_external_fallback"` + StrictCheckpointAge bool `json:"strict_checkpoint_age"` } // implement a default method for the above struct @@ -24,10 +26,10 @@ func (b BaseConfig) Default() BaseConfig { ConsensusRpc: nil, DefaultCheckpoint: [32]byte{}, Chain: ChainConfig{}, - Forks: Forks{}, + Forks: consensus_core.Forks{}, MaxCheckpointAge: 0, DataDir: nil, LoadExternalFallback: false, StrictCheckpointAge: false, } -} \ No newline at end of file +} diff --git a/config/config.go b/config/config.go index eb55a79..9cb8052 100644 --- a/config/config.go +++ b/config/config.go @@ -3,25 +3,26 @@ package config import ( "fmt" + "github.com/BlocSoc-iitr/selene/consensus/consensus_core" "github.com/BlocSoc-iitr/selene/utils" "github.com/spf13/viper" ) type Config struct { - ConsensusRpc string `json:"consensus_rpc"` - ExecutionRpc string `json:"execution_rpc"` - RpcBindIp *string `json:"rpc_bind_ip"` - RpcPort *uint16 `json:"rpc_port"` - DefaultCheckpoint [32]byte `json:"default_checkpoint"` // In cli.go, checkpoint is currently taken as []byte{} - Checkpoint *[32]byte `json:"checkpoint"` // but it should be of 32 bytes or [32]byte{} - DataDir *string `json:"data_dir"` - Chain ChainConfig `json:"chain"` - Forks Forks `json:"forks"` - MaxCheckpointAge uint64 `json:"max_checkpoint_age"` - Fallback *string `json:"fallback"` - LoadExternalFallback bool `json:"load_external_fallback"` - StrictCheckpointAge bool `json:"strict_checkpoint_age"` - DatabaseType *string `json:"database_type"` + ConsensusRpc string `json:"consensus_rpc"` + ExecutionRpc string `json:"execution_rpc"` + RpcBindIp *string `json:"rpc_bind_ip"` + RpcPort *uint16 `json:"rpc_port"` + DefaultCheckpoint [32]byte `json:"default_checkpoint"` // In cli.go, checkpoint is currently taken as []byte{} + Checkpoint *[32]byte `json:"checkpoint"` // but it should be of 32 bytes or [32]byte{} + DataDir *string `json:"data_dir"` + Chain ChainConfig `json:"chain"` + Forks consensus_core.Forks `json:"forks"` + MaxCheckpointAge uint64 `json:"max_checkpoint_age"` + Fallback *string `json:"fallback"` + LoadExternalFallback bool `json:"load_external_fallback"` + StrictCheckpointAge bool `json:"strict_checkpoint_age"` + DatabaseType *string `json:"database_type"` } // only if we are using CLI diff --git a/config/networks.go b/config/networks.go index c98d71a..0f527a3 100644 --- a/config/networks.go +++ b/config/networks.go @@ -6,6 +6,7 @@ import ( "path/filepath" "strings" + "github.com/BlocSoc-iitr/selene/consensus/consensus_core" "github.com/BlocSoc-iitr/selene/utils" "github.com/pkg/errors" ) @@ -98,20 +99,20 @@ func Mainnet() (BaseConfig, error) { GenesisTime: 1606824023, GenesisRoot: genesisRoot, }, - Forks: Forks{ - Genesis: Fork{ + Forks: consensus_core.Forks{ + Genesis: consensus_core.Fork{ Epoch: 0, ForkVersion: []byte{0x00, 0x00, 0x00, 0x00}}, - Altair: Fork{ + Altair: consensus_core.Fork{ Epoch: 74240, ForkVersion: []byte{0x01, 0x00, 0x00, 0x00}}, - Bellatrix: Fork{ + Bellatrix: consensus_core.Fork{ Epoch: 144896, ForkVersion: []byte{0x02, 0x00, 0x00, 0x00}}, - Capella: Fork{ + Capella: consensus_core.Fork{ Epoch: 194048, ForkVersion: []byte{0x03, 0x00, 0x00, 0x00}}, - Deneb: Fork{ + Deneb: consensus_core.Fork{ Epoch: 269568, ForkVersion: []byte{0x04, 0x00, 0x00, 0x00}}, }, @@ -141,20 +142,20 @@ func Goerli() (BaseConfig, error) { GenesisTime: 1616508000, GenesisRoot: genesisRoot, }, - Forks: Forks{ - Genesis: Fork{ + Forks: consensus_core.Forks{ + Genesis: consensus_core.Fork{ Epoch: 0, ForkVersion: []byte{0x00, 0x10, 0x20, 0x00}}, - Altair: Fork{ + Altair: consensus_core.Fork{ Epoch: 36660, ForkVersion: []byte{0x01, 0x10, 0x20, 0x00}}, - Bellatrix: Fork{ + Bellatrix: consensus_core.Fork{ Epoch: 112260, ForkVersion: []byte{0x02, 0x10, 0x20, 0x00}}, - Capella: Fork{ + Capella: consensus_core.Fork{ Epoch: 162304, ForkVersion: []byte{0x03, 0x10, 0x20, 0x00}}, - Deneb: Fork{ + Deneb: consensus_core.Fork{ Epoch: 231680, ForkVersion: []byte{0x04, 0x10, 0x20, 0x00}}, }, @@ -184,20 +185,20 @@ func Sepolia() (BaseConfig, error) { GenesisTime: 1655733600, GenesisRoot: genesisRoot, }, - Forks: Forks{ - Genesis: Fork{ + Forks: consensus_core.Forks{ + Genesis: consensus_core.Fork{ Epoch: 0, ForkVersion: []byte{0x90, 0x00, 0x00, 0x69}}, - Altair: Fork{ + Altair: consensus_core.Fork{ Epoch: 50, ForkVersion: []byte{0x90, 0x00, 0x00, 0x70}}, - Bellatrix: Fork{ + Bellatrix: consensus_core.Fork{ Epoch: 100, ForkVersion: []byte{0x90, 0x00, 0x00, 0x71}}, - Capella: Fork{ + Capella: consensus_core.Fork{ Epoch: 56832, ForkVersion: []byte{0x90, 0x00, 0x00, 0x72}}, - Deneb: Fork{ + Deneb: consensus_core.Fork{ Epoch: 132608, ForkVersion: []byte{0x90, 0x00, 0x00, 0x73}}, }, diff --git a/config/types.go b/config/types.go index aa8d5b7..ec76902 100644 --- a/config/types.go +++ b/config/types.go @@ -1,24 +1,16 @@ package config + import ( "encoding/hex" "encoding/json" ) + type ChainConfig struct { ChainID uint64 `json:"chain_id"` GenesisTime uint64 `json:"genesis_time"` GenesisRoot []byte `json:"genesis_root"` } -type Fork struct { - Epoch uint64 `json:"epoch"` - ForkVersion []byte `json:"fork_version"` -} -type Forks struct { - Genesis Fork `json:"genesis"` - Altair Fork `json:"altair"` - Bellatrix Fork `json:"bellatrix"` - Capella Fork `json:"capella"` - Deneb Fork `json:"deneb"` -} + func (c ChainConfig) MarshalJSON() ([]byte, error) { type Alias ChainConfig return json.Marshal(&struct { @@ -44,28 +36,3 @@ func (c *ChainConfig) UnmarshalJSON(data []byte) error { c.GenesisRoot, err = hex.DecodeString(aux.GenesisRoot) return err } -func (f Fork) MarshalJSON() ([]byte, error) { - type Alias Fork - return json.Marshal(&struct { - Alias - ForkVersion string `json:"fork_version"` - }{ - Alias: (Alias)(f), - ForkVersion: hex.EncodeToString(f.ForkVersion), - }) -} -func (f *Fork) UnmarshalJSON(data []byte) error { - type Alias Fork - aux := &struct { - *Alias - ForkVersion string `json:"fork_version"` - }{ - Alias: (*Alias)(f), - } - if err := json.Unmarshal(data, &aux); err != nil { - return err - } - var err error - f.ForkVersion, err = hex.DecodeString(aux.ForkVersion) - return err -} diff --git a/consensus/consensus.go b/consensus/consensus.go index 2dfe4f9..2a25cf3 100644 --- a/consensus/consensus.go +++ b/consensus/consensus.go @@ -545,7 +545,7 @@ func apply_bootstrap(store *LightClientStore, bootstrap consensus_core.Bootstrap } -func (in *Inner) verify_generic_update(update *GenericUpdate, expectedCurrentSlot uint64, store *LightClientStore, genesisRoots []byte, forks config.Forks) error { +func (in *Inner) verify_generic_update(update *GenericUpdate, expectedCurrentSlot uint64, store *LightClientStore, genesisRoots []byte, forks consensus_core.Forks) error { { bits := getBits(update.SyncAggregate.SyncCommitteeBits) if bits == 0 { @@ -880,7 +880,7 @@ func (in *Inner) is_valid_checkpoint(blockHashSlot uint64) bool { } func isFinalityProofValid(attestedHeader *consensus_core.Header, finalizedHeader *consensus_core.Header, finalityBranch []consensus_core.Bytes32) bool { - finalityBranchForProof, err := branchToNodes(finalityBranch) + finalityBranchForProof, err := utils.BranchToNodes(finalityBranch) if err != nil { return false } @@ -888,7 +888,7 @@ func isFinalityProofValid(attestedHeader *consensus_core.Header, finalizedHeader } func isCurrentCommitteeProofValid(attestedHeader *consensus_core.Header, currentCommittee *consensus_core.SyncCommittee, currentCommitteeBranch []consensus_core.Bytes32) bool { - CurrentCommitteeForProof, err := branchToNodes(currentCommitteeBranch) + CurrentCommitteeForProof, err := utils.BranchToNodes(currentCommitteeBranch) if err != nil { return false } @@ -896,7 +896,7 @@ func isCurrentCommitteeProofValid(attestedHeader *consensus_core.Header, current } func isNextCommitteeProofValid(attestedHeader *consensus_core.Header, currentCommittee *consensus_core.SyncCommittee, currentCommitteeBranch []consensus_core.Bytes32) bool { - currentCommitteeBranchForProof, err := branchToNodes(currentCommitteeBranch) + currentCommitteeBranchForProof, err := utils.BranchToNodes(currentCommitteeBranch) if err != nil { return false } diff --git a/consensus/consensus_core/consensus_core.go b/consensus/consensus_core/consensus_core.go index df60f60..aab009b 100644 --- a/consensus/consensus_core/consensus_core.go +++ b/consensus/consensus_core/consensus_core.go @@ -200,6 +200,18 @@ type Bootstrap struct { CurrentSyncCommitteeBranch []Bytes32 } +type Fork struct { + Epoch uint64 `json:"epoch"` + ForkVersion []byte `json:"fork_version"` +} +type Forks struct { + Genesis Fork `json:"genesis"` + Altair Fork `json:"altair"` + Bellatrix Fork `json:"bellatrix"` + Capella Fork `json:"capella"` + Deneb Fork `json:"deneb"` +} + // ToBytes serializes the Header struct to a byte slice. func (h *Header) ToBytes() []byte { var buf bytes.Buffer diff --git a/utils/utils.go b/utils/utils.go index be170c7..c44317c 100644 --- a/utils/utils.go +++ b/utils/utils.go @@ -2,12 +2,26 @@ package utils import ( "encoding/hex" + "strings" + + "bytes" + "crypto/sha256" "encoding/json" "fmt" + "log" + "github.com/ethereum/go-ethereum/common" + + "github.com/BlocSoc-iitr/selene/consensus/consensus_core" + "github.com/BlocSoc-iitr/selene/utils/bls" + + bls12381 "github.com/consensys/gnark-crypto/ecc/bls12-381" + "github.com/pkg/errors" + merkletree "github.com/wealdtech/go-merkletree" ) +// if we need to export the functions , just make their first letter capitalised func Hex_str_to_bytes(s string) ([]byte, error) { s = strings.TrimPrefix(s, "0x") @@ -45,3 +59,223 @@ func Bytes_serialize(bytes []byte) ([]byte, error) { hexString := hex.EncodeToString(bytes) return json.Marshal(hexString) } + +// TreeHashRoot computes the Merkle root from the provided leaves in a flat []byte slice. +// TreeHashRoot calculates the root hash from the input data. +func TreeHashRoot(data []byte) ([]byte, error) { + // Convert the input data into a slice of leaves + leaves, err := bytesToLeaves(data) + if err != nil { + return nil, fmt.Errorf("error converting bytes to leaves: %w", err) + } + + // Create the Merkle tree using the leaves + tree, errorCreatingMerkleTree := merkletree.New(leaves) + if errorCreatingMerkleTree != nil { + return nil, fmt.Errorf("error creating Merkle tree: %w", err) + } + + // Fetch the root hash of the tree + root := tree.Root() + if root == nil { + return nil, errors.New("failed to calculate the Merkle root: root is nil") + } + + return root, nil +} + +func bytesToLeaves(data []byte) ([][]byte, error) { + var leaves [][]byte + if err := json.Unmarshal(data, &leaves); err != nil { + return nil, err + } + + return leaves, nil +} + +func CalcSyncPeriod(slot uint64) uint64 { + epoch := slot / 32 + return epoch / 256 +} + +// isAggregateValid checks if the provided signature is valid for the given message and public keys. +func IsAggregateValid(sigBytes consensus_core.SignatureBytes, msg [32]byte, pks []*bls.G2Point) bool { + var sigInBytes [96]byte + copy(sigInBytes[:], sigBytes[:]) + // Deserialize the signature from bytes + var sig bls12381.G1Affine + if err := sig.Unmarshal(sigInBytes[:]); err != nil { + return false + } + + // Map the message to a point on the curve + msgPoint := bls.MapToCurve(msg) + + // Aggregate the public keys + aggPubKey := bls.AggregatePublicKeys(pks) + + // Prepare the pairing check inputs + P := [2]bls12381.G1Affine{*msgPoint, sig} + Q := [2]bls12381.G2Affine{*aggPubKey.G2Affine, *bls.GetG2Generator()} + + // Perform the pairing check + ok, err := bls12381.PairingCheck(P[:], Q[:]) + if err != nil { + return false + } + return ok +} + +func IsProofValid( + attestedHeader *consensus_core.Header, + leafObject []byte, // Single byte slice of the leaf object + branch [][]byte, // Slice of byte slices for the branch + depth, index int, // Depth of the Merkle proof and index of the leaf +) bool { + // If the branch length is not equal to the depth, return false + if len(branch) != depth { + return false + } + + // Initialize the derived root as the leaf object's hash + derivedRoot, errFetchingTreeHashRoot := TreeHashRoot(leafObject) + + if errFetchingTreeHashRoot != nil { + fmt.Printf("Error fetching tree hash root: %v", errFetchingTreeHashRoot) + return false + } + + // Iterate over the proof's hashes + for i, hash := range branch { + hasher := sha256.New() + + // Use the index to determine how to combine the hashes + if (index>>i)&1 == 1 { + // If index is odd, hash node || derivedRoot + hasher.Write(hash) + hasher.Write(derivedRoot) + } else { + // If index is even, hash derivedRoot || node + hasher.Write(derivedRoot) + hasher.Write(hash) + } + // Update derivedRoot for the next level + derivedRoot = hasher.Sum(nil) + } + + // Compare the final derived root with the attested header's state root + return bytes.Equal(derivedRoot, attestedHeader.StateRoot[:]) +} + +func CalculateForkVersion(forks *consensus_core.Forks, slot uint64) [4]byte { + epoch := slot / 32 + + switch { + case epoch >= forks.Deneb.Epoch: + return [4]byte(forks.Deneb.ForkVersion) + case epoch >= forks.Capella.Epoch: + return [4]byte(forks.Capella.ForkVersion) + case epoch >= forks.Bellatrix.Epoch: + return [4]byte(forks.Bellatrix.ForkVersion) + case epoch >= forks.Altair.Epoch: + return [4]byte(forks.Altair.ForkVersion) + default: + return [4]byte(forks.Genesis.ForkVersion) + } +} + +func ComputeForkDataRoot(currentVersion [4]byte, genesisValidatorRoot consensus_core.Bytes32) consensus_core.Bytes32 { + forkData := ForkData{ + CurrentVersion: currentVersion, + GenesisValidatorRoot: genesisValidatorRoot, + } + + hash, err := TreeHashRoot(forkData.ToBytes()) + if err != nil { + return consensus_core.Bytes32{} + } + return consensus_core.Bytes32(hash) +} + +// GetParticipatingKeys retrieves the participating public keys from the committee based on the bitfield represented as a byte array. +func GetParticipatingKeys(committee *consensus_core.SyncCommittee, bitfield [64]byte) ([]consensus_core.BLSPubKey, error) { + var pks []consensus_core.BLSPubKey + numBits := len(bitfield) * 8 // Total number of bits + + if len(committee.Pubkeys) > numBits { + return nil, fmt.Errorf("bitfield is too short for the number of public keys") + } + + for i := 0; i < len(bitfield); i++ { + byteVal := bitfield[i] + for bit := 0; bit < 8; bit++ { + if (byteVal & (1 << bit)) != 0 { + index := i*8 + bit + if index >= len(committee.Pubkeys) { + break + } + pks = append(pks, committee.Pubkeys[index]) + } + } + } + + return pks, nil +} + +func ComputeSigningRoot(objectRoot, domain consensus_core.Bytes32) consensus_core.Bytes32 { + signingData := SigningData{ + ObjectRoot: objectRoot, + Domain: domain, + } + hash, err := TreeHashRoot(signingData.ToBytes()) + if err != nil { + return consensus_core.Bytes32{} + } + return consensus_core.Bytes32(hash) +} + +func ComputeDomain(domainType [4]byte, forkDataRoot consensus_core.Bytes32) consensus_core.Bytes32 { + data := append(domainType[:], forkDataRoot[:28]...) + return sha256.Sum256(data) +} + +type SigningData struct { + ObjectRoot consensus_core.Bytes32 + Domain consensus_core.Bytes32 +} + +type ForkData struct { + CurrentVersion [4]byte + GenesisValidatorRoot consensus_core.Bytes32 +} + +func (fd *ForkData) ToBytes() []byte { + data, err := json.Marshal(fd) + if err != nil { + log.Println("Error marshaling ForkData:", err) + return nil // Or return an empty slice, based on your preference + } + return data +} + +func (sd *SigningData) ToBytes() []byte { + data, err := json.Marshal(sd) + if err != nil { + log.Println("Error marshaling SigningData:", err) + return nil // Or return an empty slice, based on your preference + } + return data +} + +func Bytes32ToNode(bytes consensus_core.Bytes32) []byte { + return []byte(bytes[:]) +} + +// branchToNodes converts a slice of Bytes32 to a slice of Node +func BranchToNodes(branch []consensus_core.Bytes32) ([][]byte, error) { + nodes := make([][]byte, len(branch)) + for i, b32 := range branch { + nodes[i] = Bytes32ToNode(b32) + } + return nodes, nil +} From 9b7137d0ec8efa1ba0e233f31b67ae0fd2f0041a Mon Sep 17 00:00:00 2001 From: DarkLord017 Date: Sun, 22 Sep 2024 08:56:08 +0530 Subject: [PATCH 3/4] Corrected tests to use consesnsus_Core for forks --- config/config_test.go | 26 ++++++++++++++------------ 1 file changed, 14 insertions(+), 12 deletions(-) diff --git a/config/config_test.go b/config/config_test.go index 0bc3a1d..6b31d0a 100644 --- a/config/config_test.go +++ b/config/config_test.go @@ -3,10 +3,12 @@ package config import ( "encoding/hex" "fmt" - "github.com/spf13/viper" "os" "reflect" "testing" + + "github.com/BlocSoc-iitr/selene/consensus/consensus_core" + "github.com/spf13/viper" ) var ( @@ -23,9 +25,9 @@ var ( defaultCheckpoint = [32]byte{} ) -///////////////////////////// -///// from_file() tests ///// -///////////////////////////// +// /////////////////////////// +// /// from_file() tests ///// +// /////////////////////////// func TestMainnetBaseConfig(t *testing.T) { network := "MAINNET" path := "./config.toml" @@ -44,8 +46,8 @@ func TestMainnetBaseConfig(t *testing.T) { if !reflect.DeepEqual(config.Chain, mainnetConfig.Chain) { t.Errorf("Expected Chain to be %v, but got %v", mainnetConfig.Chain, config.Chain) } - if !reflect.DeepEqual(config.Forks, mainnetConfig.Forks) { - t.Errorf("Expected Forks to be %v, but got %v", mainnetConfig.Forks, config.Forks) + if !reflect.DeepEqual(consensus_core.Forks, mainnetConfig.Forks) { + t.Errorf("Expected Forks to be %v, but got %v", mainnetConfig.Forks, consensus_core.Forks) } if *config.RpcBindIp != mainnetConfig.RpcBindIp { t.Errorf("Expected RpcBindIP to be %s, but got %s", mainnetConfig.RpcBindIp, *config.RpcBindIp) @@ -179,9 +181,9 @@ func createConfigFile(v *viper.Viper) { } } -////////////////////////////////// -///// to_base_config() tests ///// -////////////////////////////////// +// //////////////////////////////// +// /// to_base_config() tests ///// +// //////////////////////////////// func TestReturnsCorrectBaseConfig(t *testing.T) { config := Config{ ConsensusRpc: consensusRpc, @@ -201,8 +203,8 @@ func TestReturnsCorrectBaseConfig(t *testing.T) { if !reflect.DeepEqual(baseConfig.Chain, config.Chain) { t.Errorf("Expected Chain to be %v, got %v", config.Chain, baseConfig.Chain) } - if !reflect.DeepEqual(baseConfig.Forks, config.Forks) { - t.Errorf("Expected Forks to be %v, got %v", config.Forks, baseConfig.Forks) + if !reflect.DeepEqual(baseConfig.Forks, consensus_core.Forks) { + t.Errorf("Expected Forks to be %v, got %v", consensus_core.Forks, baseConfig.Forks) } if baseConfig.MaxCheckpointAge != config.MaxCheckpointAge { t.Errorf("Expected Max Checkpoint age to be %v, got %v", config.MaxCheckpointAge, baseConfig.MaxCheckpointAge) @@ -230,4 +232,4 @@ func TestReturnsCorrectDefaultValues(t *testing.T) { if baseConfig.RpcBindIp != "127.0.0.1" { t.Errorf("Expected Max Checkpoint age to be %v, got %v", "127.0.0.1", baseConfig.RpcBindIp) } -} \ No newline at end of file +} From 1ce44682575d4118bf57e50a7c363a7e8f710c27 Mon Sep 17 00:00:00 2001 From: DarkLord017 Date: Sun, 22 Sep 2024 09:01:21 +0530 Subject: [PATCH 4/4] Corrected tests to use consesnsus_Core for forks --- config/config_test.go | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/config/config_test.go b/config/config_test.go index 6b31d0a..c34095c 100644 --- a/config/config_test.go +++ b/config/config_test.go @@ -46,8 +46,8 @@ func TestMainnetBaseConfig(t *testing.T) { if !reflect.DeepEqual(config.Chain, mainnetConfig.Chain) { t.Errorf("Expected Chain to be %v, but got %v", mainnetConfig.Chain, config.Chain) } - if !reflect.DeepEqual(consensus_core.Forks, mainnetConfig.Forks) { - t.Errorf("Expected Forks to be %v, but got %v", mainnetConfig.Forks, consensus_core.Forks) + if !reflect.DeepEqual(config.Forks, mainnetConfig.Forks) { + t.Errorf("Expected Forks to be %v, but got %v", mainnetConfig.Forks, config.Forks) } if *config.RpcBindIp != mainnetConfig.RpcBindIp { t.Errorf("Expected RpcBindIP to be %s, but got %s", mainnetConfig.RpcBindIp, *config.RpcBindIp) @@ -191,7 +191,7 @@ func TestReturnsCorrectBaseConfig(t *testing.T) { RpcPort: &rpcPort, DefaultCheckpoint: defaultCheckpoint, Chain: ChainConfig{}, - Forks: Forks{}, + Forks: consensus_core.Forks{}, MaxCheckpointAge: uint64(maxCheckpointAge), DataDir: &dataDirectory, LoadExternalFallback: loadExternalFallback, @@ -203,8 +203,8 @@ func TestReturnsCorrectBaseConfig(t *testing.T) { if !reflect.DeepEqual(baseConfig.Chain, config.Chain) { t.Errorf("Expected Chain to be %v, got %v", config.Chain, baseConfig.Chain) } - if !reflect.DeepEqual(baseConfig.Forks, consensus_core.Forks) { - t.Errorf("Expected Forks to be %v, got %v", consensus_core.Forks, baseConfig.Forks) + if !reflect.DeepEqual(baseConfig.Forks, config.Forks) { + t.Errorf("Expected Forks to be %v, got %v", config.Forks, baseConfig.Forks) } if baseConfig.MaxCheckpointAge != config.MaxCheckpointAge { t.Errorf("Expected Max Checkpoint age to be %v, got %v", config.MaxCheckpointAge, baseConfig.MaxCheckpointAge) @@ -215,7 +215,7 @@ func TestReturnsCorrectDefaultValues(t *testing.T) { ConsensusRpc: consensusRpc, DefaultCheckpoint: defaultCheckpoint, Chain: ChainConfig{}, - Forks: Forks{}, + Forks: consensus_core.Forks{}, MaxCheckpointAge: uint64(maxCheckpointAge), DataDir: &dataDirectory, LoadExternalFallback: loadExternalFallback,