From 27769119add725a16d7805eef8f7f6f136cfc5bc Mon Sep 17 00:00:00 2001 From: Sambhav Jain <136801346+DarkLord017@users.noreply.github.com> Date: Fri, 20 Sep 2024 06:38:25 +0530 Subject: [PATCH 01/11] Created utils.go --- consensus/utils.go | 249 ++++++++++++++++++++++++++++++++++++++++++--- 1 file changed, 236 insertions(+), 13 deletions(-) diff --git a/consensus/utils.go b/consensus/utils.go index 4e25578..651d051 100644 --- a/consensus/utils.go +++ b/consensus/utils.go @@ -1,15 +1,238 @@ package consensus -// uses types package - -func calculate_sync_period() {} -func is_aggregate_valid() {} -func is_proof_valid() {} -func compute_signing_root() {} -func compute_domian() {} -func compute_fork_data_root() {} -func branch_to_nodes() {} -func bytes32_to_nodes() {} - -type SigningData struct{} -type ForkData struct{} +import ( + "bytes" + "crypto/sha256" + "errors" + "fmt" + "math" + + "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/ethereum/go-ethereum/crypto" +) + +type Forks struct { + Genesis ForkVersion + Altair ForkVersion + Bellatrix ForkVersion + Capella ForkVersion + Deneb ForkVersion +} + +type ForkVersion struct { + Epoch uint64 + ForkVersion [4]byte +} + +func TreeHashRoot(leaves [][]byte) ([]byte, error) { + if len(leaves) == 0 { + return nil, fmt.Errorf("no leaves provided") + } + + nodes := leaves + + for len(nodes) > 1 { + var newLevel [][]byte + + // Pair nodes and hash them + for i := 0; i < len(nodes); i += 2 { + if i+1 < len(nodes) { + // Hash pair of nodes + nodeHash := crypto.Keccak256(append(nodes[i], nodes[i+1]...)) + newLevel = append(newLevel, nodeHash) + } else { + // Handle odd number of nodes (carry last node up) + newLevel = append(newLevel, nodes[i]) + } + } + + nodes = newLevel + } + + return nodes[0], 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, // 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 + } + + // Compute the root hash of the leaf object + derivedRoot, err := TreeHashRoot(leafObject) + if err != nil { + return false + } + + // Iterate through the branch and compute the Merkle root + for i, node := range branch { + hasher := sha256.New() + + // Check if index / 2^i is odd or even + if (index/int(math.Pow(2, float64(i))))%2 != 0 { + // If odd, hash(node || derived_root) + hasher.Write(node) + hasher.Write(derivedRoot[:]) + } else { + // If even, hash(derived_root || node) + hasher.Write(derivedRoot[:]) + hasher.Write(node) + } + + // Update the derived root + derivedRootNew := sha256.Sum256(hasher.Sum(nil)) + derivedRoot = derivedRootNew[:] + } + + // Compare the final derived root with the attested header's state root + return bytes.Equal(derivedRoot[:], attestedHeader.State_root[:]) +} + +func CalculateForkVersion(forks *Forks, slot uint64) [4]byte { + epoch := slot / 32 + + switch { + case epoch >= forks.Deneb.Epoch: + return forks.Deneb.ForkVersion + case epoch >= forks.Capella.Epoch: + return forks.Capella.ForkVersion + case epoch >= forks.Bellatrix.Epoch: + return forks.Bellatrix.ForkVersion + case epoch >= forks.Altair.Epoch: + return forks.Altair.ForkVersion + default: + return 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.SyncComittee, 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, errors.New("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 := make([][]byte, 2) + data[0] = fd.CurrentVersion[:] + data[1] = fd.GenesisValidatorRoot[:] + return data +} + +func (sd *SigningData) ToBytes() [][]byte { + data := make([][]byte, 2) + data[0] = sd.ObjectRoot[:] + data[1] = sd.Domain[:] + 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 9c22eb58eaeddc62ca60a42b50cd77aa186a7172 Mon Sep 17 00:00:00 2001 From: Sambhav Jain <136801346+DarkLord017@users.noreply.github.com> Date: Fri, 20 Sep 2024 07:02:38 +0530 Subject: [PATCH 02/11] Corrected consensus_core.go to make the required functions public --- consensus/consensus_core/consensus_core.go | 92 +++++++++++----------- 1 file changed, 47 insertions(+), 45 deletions(-) diff --git a/consensus/consensus_core/consensus_core.go b/consensus/consensus_core/consensus_core.go index 398d67d..9904e03 100644 --- a/consensus/consensus_core/consensus_core.go +++ b/consensus/consensus_core/consensus_core.go @@ -3,12 +3,14 @@ // LightClientStore,genericUpdate package consensus_core +import "github.com/BlocSoc-iitr/selene/consensus/types" + type BeaconBlock struct { slot uint64 proposer_index uint64 parent_root [32]byte state_root [32]byte - body BeaconBlockBody + Body BeaconBlockBody } type Bytes32 [32]byte type BLSPubKey [48]byte @@ -31,7 +33,7 @@ type SignedBeaconBlockHeader struct { type BeaconBlockHeader struct { slot uint64 proposer_index uint64 - parent_root Bytes32 + Parent_root Bytes32 state_root Bytes32 body_root Bytes32 } @@ -80,8 +82,8 @@ type VoluntaryExit struct { validator_index uint64 } type SyncAggregate struct { - sync_committee_bits [64]byte - sync_committee_signature SignatureBytes + Sync_committee_bits [64]byte + Sync_committee_signature SignatureBytes } type Withdrawal struct { index uint64 @@ -90,21 +92,21 @@ type Withdrawal struct { amount uint64 } type ExecutionPayload struct { //not implemented - parent_hash Bytes32 - fee_recipient Address - state_root Bytes32 - receipts_root Bytes32 - logs_bloom LogsBloom - prev_randao Bytes32 - block_number uint64 - gas_limit uint64 - gas_used uint64 - timestamp uint64 - extra_data [32]byte - base_fee_per_gas uint64 - block_hash Bytes32 - transactions []byte //max length 1073741824 to be implemented - withdrawals []Withdrawal //max length 16 to be implemented + Parent_hash Bytes32 + Fee_recipient Address + State_root Bytes32 + Receipts_root Bytes32 + Logs_bloom LogsBloom + Prev_randao Bytes32 + Block_number uint64 + Gas_limit uint64 + Gas_used uint64 + Timestamp uint64 + Extra_data [32]byte + Base_fee_per_gas uint64 + Block_hash Bytes32 + Transactions types.Transaction + withdrawals types.Withdrawal blob_gas_used uint64 excess_blob_gas uint64 } @@ -126,44 +128,44 @@ type BeaconBlockBody struct { //not implemented deposits []Deposit //max length 16 to be ensured voluntary_exits SignedVoluntaryExit sync_aggregate SyncAggregate - execution_payload ExecutionPayload + Execution_payload ExecutionPayload bls_to_execution_changes []SignedBlsToExecutionChange //max length 16 to be ensured blob_kzg_commitments [][48]byte //max length 4096 to be ensured } type Header struct { - slot uint64 - proposer_index uint64 - parent_root Bytes32 - state_root Bytes32 - body_root Bytes32 + Slot uint64 + Proposer_index uint64 + Parent_root Bytes32 + State_root Bytes32 + Body_root Bytes32 } type SyncComittee struct { - pubkeys [512]BLSPubKey - aggregate_pubkey BLSPubKey + Pubkeys [512]BLSPubKey + Aggregate_pubkey BLSPubKey } type Update struct { - attested_header Header - next_sync_committee SyncComittee - next_sync_committee_branch []Bytes32 - finalized_header Header - finality_branch []Bytes32 - sync_aggregate SyncAggregate - signature_slot uint64 + AttestedHeader Header + NextSyncCommitee SyncComittee + NextSyncCommiteeBranch []Bytes32 + FinalizedHeader Header + FinalityBranch []Bytes32 + SyncAggregate SyncAggregate + SignatureSlot uint64 } type FinalityUpdate struct { - attested_header Header - finalized_header Header - finality_branch []Bytes32 - sync_aggregate SyncAggregate - signature_slot uint64 + AttestedHeader Header + FinalizedHeader Header + FinalityBranch []Bytes32 + SyncAggregate SyncAggregate + SignatureSlot uint64 } type OptimisticUpdate struct { - attested_header Header - sync_aggregate SyncAggregate - signature_slot uint64 + AttestedHeader Header + SyncAggregate SyncAggregate + SignatureSlot uint64 } type Bootstrap struct { - header Header - current_sync_committee SyncComittee - current_sync_committee_branch []Bytes32 + Header Header + CurrentSyncCommittee SyncComittee + CurrentSyncCommitteeBranch []Bytes32 } From 9e38ef161dd2694065efaff23fd580a6029e6755 Mon Sep 17 00:00:00 2001 From: Sambhav Jain <136801346+DarkLord017@users.noreply.github.com> Date: Fri, 20 Sep 2024 07:44:02 +0530 Subject: [PATCH 03/11] Update consensus_rpc.go to remove errors --- consensus/rpc/consensus_rpc.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/consensus/rpc/consensus_rpc.go b/consensus/rpc/consensus_rpc.go index 8395c79..b9a7bae 100644 --- a/consensus/rpc/consensus_rpc.go +++ b/consensus/rpc/consensus_rpc.go @@ -6,11 +6,11 @@ import ( // return types not mention and oarameters as well type ConsensusRpc interface { - GetBootstrap(block_root []byte) (consensus_core.Bootstrap, error) + GetBootstrap(block_root [32]byte) (consensus_core.Bootstrap, error) GetUpdates(period uint64, count uint8) ([]consensus_core.Update, error) GetFinalityUpdate() (consensus_core.FinalityUpdate, error) GetOptimisticUpdate() (consensus_core.OptimisticUpdate, error) - GetBlock(slot uint64) (consensus_core.BeaconBlock, error) + GetBlock(slot uint64) (*consensus_core.BeaconBlock, error) ChainId() (uint64, error) } From 431df6b6e561d98cec07dd18596f8ff4abca69f0 Mon Sep 17 00:00:00 2001 From: Sambhav Jain <136801346+DarkLord017@users.noreply.github.com> Date: Fri, 20 Sep 2024 07:45:19 +0530 Subject: [PATCH 04/11] Update nimbus_rpc.go to remove linter errors and correct input parameters --- consensus/rpc/nimbus_rpc.go | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/consensus/rpc/nimbus_rpc.go b/consensus/rpc/nimbus_rpc.go index 1f97521..971b2d0 100644 --- a/consensus/rpc/nimbus_rpc.go +++ b/consensus/rpc/nimbus_rpc.go @@ -3,11 +3,12 @@ package rpc import ( "encoding/json" "fmt" - "github.com/BlocSoc-iitr/selene/consensus/consensus_core" "io" "net/http" "strconv" "time" + + "github.com/BlocSoc-iitr/selene/consensus/consensus_core" ) // uses types package @@ -43,15 +44,17 @@ func min(a uint8, b uint8) uint8 { } return b } + type NimbusRpc struct { //ConsensusRpc rpc string } + func NewNimbusRpc(rpc string) *NimbusRpc { return &NimbusRpc{ rpc: rpc} } -func (n *NimbusRpc) GetBootstrap(block_root []byte) (consensus_core.Bootstrap, error) { +func (n *NimbusRpc) GetBootstrap(block_root [32]byte) (consensus_core.Bootstrap, error) { root_hex := fmt.Sprintf("%x", block_root) req := fmt.Sprintf("%s/eth/v1/beacon/light_client/bootstrap/0x%s", n.rpc, root_hex) var res BootstrapResponse @@ -93,14 +96,15 @@ func (n *NimbusRpc) GetOptimisticUpdate() (consensus_core.OptimisticUpdate, erro } return res.data, nil } -func (n *NimbusRpc) GetBlock(slot uint64) (consensus_core.BeaconBlock, error) { +func (n *NimbusRpc) GetBlock(slot uint64) (*consensus_core.BeaconBlock, error) { req := fmt.Sprintf("%s/eth/v2/beacon/blocks/%s", n.rpc, strconv.FormatUint(slot, 10)) var res BeaconBlockResponse err := get(req, &res) if err != nil { - return consensus_core.BeaconBlock{}, fmt.Errorf("block error: %w", err) + return nil, fmt.Errorf("block error: %w", err) } - return res.data.message, nil + + return &res.data.message, nil } func (n *NimbusRpc) ChainId() (uint64, error) { req := fmt.Sprintf("%s/eth/v1/config/spec", n.rpc) @@ -111,6 +115,7 @@ func (n *NimbusRpc) ChainId() (uint64, error) { } return res.data.chain_id, nil } + // BeaconBlock, Update,FinalityUpdate ,OptimisticUpdate,Bootstrap yet to be defined in consensus-core/src/types/mod.go // For now defined in consensus/consensus_core.go type BeaconBlockResponse struct { @@ -138,4 +143,3 @@ type Spec struct { type BootstrapResponse struct { data consensus_core.Bootstrap } - From e0d594348f88b3fdd233d59740e96799c1b12f09 Mon Sep 17 00:00:00 2001 From: Sambhav Jain <136801346+DarkLord017@users.noreply.github.com> Date: Fri, 20 Sep 2024 07:51:49 +0530 Subject: [PATCH 05/11] Update consensus_core.go --- consensus/consensus_core/consensus_core.go | 224 ++++++++++++--------- 1 file changed, 124 insertions(+), 100 deletions(-) diff --git a/consensus/consensus_core/consensus_core.go b/consensus/consensus_core/consensus_core.go index 9904e03..44185d3 100644 --- a/consensus/consensus_core/consensus_core.go +++ b/consensus/consensus_core/consensus_core.go @@ -1,157 +1,179 @@ -// go implementation of consensus_core/src/types/mod.rs -// structs not defined yet: -// LightClientStore,genericUpdate package consensus_core import "github.com/BlocSoc-iitr/selene/consensus/types" type BeaconBlock struct { - slot uint64 - proposer_index uint64 - parent_root [32]byte - state_root [32]byte - Body BeaconBlockBody + Slot uint64 + ProposerIndex uint64 + ParentRoot [32]byte + StateRoot [32]byte + Body BeaconBlockBody } + type Bytes32 [32]byte type BLSPubKey [48]byte type Address [20]byte type LogsBloom = [256]byte type SignatureBytes [96]byte + type Eth1Data struct { - deposit_root Bytes32 - deposit_count uint64 - block_hash Bytes32 + DepositRoot Bytes32 + DepositCount uint64 + BlockHash Bytes32 } + type ProposerSlashing struct { - signed_header_1 SignedBeaconBlockHeader - signed_header_2 SignedBeaconBlockHeader + SignedHeader1 SignedBeaconBlockHeader + SignedHeader2 SignedBeaconBlockHeader } + type SignedBeaconBlockHeader struct { - message BeaconBlockHeader - signature SignatureBytes + Message BeaconBlockHeader + Signature SignatureBytes } + type BeaconBlockHeader struct { - slot uint64 - proposer_index uint64 - Parent_root Bytes32 - state_root Bytes32 - body_root Bytes32 + Slot uint64 + ProposerIndex uint64 + ParentRoot Bytes32 + StateRoot Bytes32 + BodyRoot Bytes32 } + type AttesterSlashing struct { - attestation_1 IndexedAttestation - attestation_2 IndexedAttestation + Attestation1 IndexedAttestation + Attestation2 IndexedAttestation } + type IndexedAttestation struct { - attesting_indices []uint64 //max length 2048 to be ensured - data AttestationData - signature SignatureBytes + AttestingIndices []uint64 //max length 2048 to be ensured + Data AttestationData + Signature SignatureBytes } + type AttestationData struct { - slot uint64 - index uint64 - beacon_block_root Bytes32 - source Checkpoint - target Checkpoint + Slot uint64 + Index uint64 + BeaconBlockRoot Bytes32 + Source Checkpoint + Target Checkpoint } + type Checkpoint struct { - epoch uint64 - root Bytes32 + Epoch uint64 + Root Bytes32 } + type Bitlist []bool + type Attestation struct { - aggregation_bits Bitlist `ssz-max:"2048"` - data AttestationData - signature SignatureBytes + AggregationBits Bitlist `ssz-max:"2048"` + Data AttestationData + Signature SignatureBytes } + type Deposit struct { - proof [33]Bytes32 //fixed size array - data DepositData + Proof [33]Bytes32 //fixed size array + Data DepositData } + type DepositData struct { - pubkey [48]byte - withdrawal_credentials Bytes32 - amount uint64 - signature SignatureBytes + Pubkey [48]byte + WithdrawalCredentials Bytes32 + Amount uint64 + Signature SignatureBytes } + type SignedVoluntaryExit struct { - message VoluntaryExit - signature SignatureBytes + Message VoluntaryExit + Signature SignatureBytes } + type VoluntaryExit struct { - epoch uint64 - validator_index uint64 + Epoch uint64 + ValidatorIndex uint64 } + type SyncAggregate struct { - Sync_committee_bits [64]byte - Sync_committee_signature SignatureBytes + SyncCommitteeBits [64]byte + SyncCommitteeSignature SignatureBytes } + type Withdrawal struct { - index uint64 - validator_index uint64 - address Address - amount uint64 + Index uint64 + ValidatorIndex uint64 + Address Address + Amount uint64 } + type ExecutionPayload struct { //not implemented - Parent_hash Bytes32 - Fee_recipient Address - State_root Bytes32 - Receipts_root Bytes32 - Logs_bloom LogsBloom - Prev_randao Bytes32 - Block_number uint64 - Gas_limit uint64 - Gas_used uint64 - Timestamp uint64 - Extra_data [32]byte - Base_fee_per_gas uint64 - Block_hash Bytes32 - Transactions types.Transaction - withdrawals types.Withdrawal - blob_gas_used uint64 - excess_blob_gas uint64 + ParentHash Bytes32 + FeeRecipient Address + StateRoot Bytes32 + ReceiptsRoot Bytes32 + LogsBloom LogsBloom + PrevRandao Bytes32 + BlockNumber uint64 + GasLimit uint64 + GasUsed uint64 + Timestamp uint64 + ExtraData [32]byte + BaseFeePerGas uint64 + BlockHash Bytes32 + Transactions types.Transaction + Withdrawals types.Withdrawal + BlobGasUsed uint64 + ExcessBlobGas uint64 } + type SignedBlsToExecutionChange struct { - message BlsToExecutionChange - signature SignatureBytes + Message BlsToExecutionChange + Signature SignatureBytes } + type BlsToExecutionChange struct { - validator_index uint64 - from_bls_pubkey [48]byte + ValidatorIndex uint64 + FromBlsPubkey [48]byte } + type BeaconBlockBody struct { //not implemented - randao_reveal SignatureBytes - eth1_data Eth1Data - graffiti Bytes32 - proposer_slashings []ProposerSlashing //max length 16 to be insured how? - attester_slashings []AttesterSlashing //max length 2 to be ensured - attestations []Attestation //max length 128 to be ensured - deposits []Deposit //max length 16 to be ensured - voluntary_exits SignedVoluntaryExit - sync_aggregate SyncAggregate - Execution_payload ExecutionPayload - bls_to_execution_changes []SignedBlsToExecutionChange //max length 16 to be ensured - blob_kzg_commitments [][48]byte //max length 4096 to be ensured + RandaoReveal SignatureBytes + Eth1Data Eth1Data + Graffiti Bytes32 + ProposerSlashings []ProposerSlashing //max length 16 to be insured how? + AttesterSlashings []AttesterSlashing //max length 2 to be ensured + Attestations []Attestation //max length 128 to be ensured + Deposits []Deposit //max length 16 to be ensured + VoluntaryExits SignedVoluntaryExit + SyncAggregate SyncAggregate + ExecutionPayload ExecutionPayload + BlsToExecutionChanges []SignedBlsToExecutionChange //max length 16 to be ensured + BlobKzgCommitments [][48]byte //max length 4096 to be ensured } + type Header struct { - Slot uint64 - Proposer_index uint64 - Parent_root Bytes32 - State_root Bytes32 - Body_root Bytes32 + Slot uint64 + ProposerIndex uint64 + ParentRoot Bytes32 + StateRoot Bytes32 + BodyRoot Bytes32 } -type SyncComittee struct { - Pubkeys [512]BLSPubKey - Aggregate_pubkey BLSPubKey + +type SyncCommittee struct { + Pubkeys [512]BLSPubKey + AggregatePubkey BLSPubKey } + type Update struct { - AttestedHeader Header - NextSyncCommitee SyncComittee - NextSyncCommiteeBranch []Bytes32 - FinalizedHeader Header - FinalityBranch []Bytes32 - SyncAggregate SyncAggregate - SignatureSlot uint64 + AttestedHeader Header + NextSyncCommittee SyncCommittee + NextSyncCommitteeBranch []Bytes32 + FinalizedHeader Header + FinalityBranch []Bytes32 + SyncAggregate SyncAggregate + SignatureSlot uint64 } + type FinalityUpdate struct { AttestedHeader Header FinalizedHeader Header @@ -159,13 +181,15 @@ type FinalityUpdate struct { SyncAggregate SyncAggregate SignatureSlot uint64 } + type OptimisticUpdate struct { AttestedHeader Header SyncAggregate SyncAggregate SignatureSlot uint64 } + type Bootstrap struct { Header Header - CurrentSyncCommittee SyncComittee + CurrentSyncCommittee SyncCommittee CurrentSyncCommitteeBranch []Bytes32 } From 5fa516718c167b6425ab92d101e519f013c9882c Mon Sep 17 00:00:00 2001 From: Sambhav Jain <136801346+DarkLord017@users.noreply.github.com> Date: Fri, 20 Sep 2024 07:54:52 +0530 Subject: [PATCH 06/11] Update utils.go according to new consensus_core.go --- consensus/utils.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/consensus/utils.go b/consensus/utils.go index 651d051..bdcb014 100644 --- a/consensus/utils.go +++ b/consensus/utils.go @@ -125,7 +125,7 @@ func isProofValid( } // Compare the final derived root with the attested header's state root - return bytes.Equal(derivedRoot[:], attestedHeader.State_root[:]) + return bytes.Equal(derivedRoot[:], attestedHeader.StateRoot[:]) } func CalculateForkVersion(forks *Forks, slot uint64) [4]byte { @@ -159,7 +159,7 @@ func ComputeForkDataRoot(currentVersion [4]byte, genesisValidatorRoot consensus_ } // GetParticipatingKeys retrieves the participating public keys from the committee based on the bitfield represented as a byte array. -func GetParticipatingKeys(committee *consensus_core.SyncComittee, bitfield [64]byte) ([]consensus_core.BLSPubKey, error) { +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 From 9985680238e047bf99e929035649958657883388 Mon Sep 17 00:00:00 2001 From: Sambhav Jain <136801346+DarkLord017@users.noreply.github.com> Date: Fri, 20 Sep 2024 13:40:25 +0530 Subject: [PATCH 07/11] Update consensus_core.go implemented unimplemented changes --- consensus/consensus_core/consensus_core.go | 30 +++++++++++----------- 1 file changed, 15 insertions(+), 15 deletions(-) diff --git a/consensus/consensus_core/consensus_core.go b/consensus/consensus_core/consensus_core.go index 44185d3..295e912 100644 --- a/consensus/consensus_core/consensus_core.go +++ b/consensus/consensus_core/consensus_core.go @@ -46,7 +46,7 @@ type AttesterSlashing struct { } type IndexedAttestation struct { - AttestingIndices []uint64 //max length 2048 to be ensured + AttestingIndices [2048]uint64 Data AttestationData Signature SignatureBytes } @@ -106,7 +106,7 @@ type Withdrawal struct { Amount uint64 } -type ExecutionPayload struct { //not implemented +type ExecutionPayload struct { ParentHash Bytes32 FeeRecipient Address StateRoot Bytes32 @@ -120,10 +120,10 @@ type ExecutionPayload struct { //not implemented ExtraData [32]byte BaseFeePerGas uint64 BlockHash Bytes32 - Transactions types.Transaction - Withdrawals types.Withdrawal - BlobGasUsed uint64 - ExcessBlobGas uint64 + Transactions []types.Transaction `ssz-max:"1048576"` + Withdrawals []types.Withdrawal `ssz-max:"16"` + BlobGasUsed *uint64 // Deneb-specific field, use pointer for optionality + ExcessBlobGas *uint64 // Deneb-specific field, use pointer for optionality } type SignedBlsToExecutionChange struct { @@ -136,19 +136,19 @@ type BlsToExecutionChange struct { FromBlsPubkey [48]byte } -type BeaconBlockBody struct { //not implemented +// BeaconBlockBody represents the body of a beacon block. +type BeaconBlockBody struct { RandaoReveal SignatureBytes Eth1Data Eth1Data Graffiti Bytes32 - ProposerSlashings []ProposerSlashing //max length 16 to be insured how? - AttesterSlashings []AttesterSlashing //max length 2 to be ensured - Attestations []Attestation //max length 128 to be ensured - Deposits []Deposit //max length 16 to be ensured - VoluntaryExits SignedVoluntaryExit + ProposerSlashings []ProposerSlashing `ssz-max:"16"` + AttesterSlashings []AttesterSlashing `ssz-max:"2"` + Attestations []Attestation `ssz-max:"128"` + Deposits []Deposit `ssz-max:"16"` + VoluntaryExits []SignedVoluntaryExit `ssz-max:"16"` SyncAggregate SyncAggregate - ExecutionPayload ExecutionPayload - BlsToExecutionChanges []SignedBlsToExecutionChange //max length 16 to be ensured - BlobKzgCommitments [][48]byte //max length 4096 to be ensured + BlsToExecutionChanges []SignedBlsToExecutionChange `ssz-max:"16"` + BlobKzgCommitments [][48]byte `ssz-max:"4096"` } type Header struct { From b925ec3ca486402025bb1c775b386a6eaec2556b Mon Sep 17 00:00:00 2001 From: Sambhav Jain <136801346+DarkLord017@users.noreply.github.com> Date: Sat, 21 Sep 2024 08:36:08 +0530 Subject: [PATCH 08/11] Update utils.go removed unnnecesaaray structs --- consensus/utils.go | 26 +++++++------------------- 1 file changed, 7 insertions(+), 19 deletions(-) diff --git a/consensus/utils.go b/consensus/utils.go index bdcb014..dd641f3 100644 --- a/consensus/utils.go +++ b/consensus/utils.go @@ -7,25 +7,13 @@ import ( "fmt" "math" + "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/ethereum/go-ethereum/crypto" ) -type Forks struct { - Genesis ForkVersion - Altair ForkVersion - Bellatrix ForkVersion - Capella ForkVersion - Deneb ForkVersion -} - -type ForkVersion struct { - Epoch uint64 - ForkVersion [4]byte -} - func TreeHashRoot(leaves [][]byte) ([]byte, error) { if len(leaves) == 0 { return nil, fmt.Errorf("no leaves provided") @@ -128,20 +116,20 @@ func isProofValid( return bytes.Equal(derivedRoot[:], attestedHeader.StateRoot[:]) } -func CalculateForkVersion(forks *Forks, slot uint64) [4]byte { +func CalculateForkVersion(forks *config.Forks, slot uint64) [4]byte { epoch := slot / 32 switch { case epoch >= forks.Deneb.Epoch: - return forks.Deneb.ForkVersion + return [4]byte(forks.Deneb.ForkVersion) case epoch >= forks.Capella.Epoch: - return forks.Capella.ForkVersion + return [4]byte(forks.Capella.ForkVersion) case epoch >= forks.Bellatrix.Epoch: - return forks.Bellatrix.ForkVersion + return [4]byte(forks.Bellatrix.ForkVersion) case epoch >= forks.Altair.Epoch: - return forks.Altair.ForkVersion + return [4]byte(forks.Altair.ForkVersion) default: - return forks.Genesis.ForkVersion + return [4]byte(forks.Genesis.ForkVersion) } } From 5df3785031324e140836774dcd1954a7f00423e7 Mon Sep 17 00:00:00 2001 From: Sambhav Jain <136801346+DarkLord017@users.noreply.github.com> Date: Sat, 21 Sep 2024 08:55:57 +0530 Subject: [PATCH 09/11] Added toBytes to easily get TreeHashRoot --- consensus/consensus_core/consensus_core.go | 214 ++++++++++++++++++++- 1 file changed, 213 insertions(+), 1 deletion(-) diff --git a/consensus/consensus_core/consensus_core.go b/consensus/consensus_core/consensus_core.go index 295e912..15e3d5f 100644 --- a/consensus/consensus_core/consensus_core.go +++ b/consensus/consensus_core/consensus_core.go @@ -1,6 +1,10 @@ package consensus_core -import "github.com/BlocSoc-iitr/selene/consensus/types" +import ( + "encoding/binary" + + "github.com/BlocSoc-iitr/selene/consensus/types" +) type BeaconBlock struct { Slot uint64 @@ -147,6 +151,7 @@ type BeaconBlockBody struct { Deposits []Deposit `ssz-max:"16"` VoluntaryExits []SignedVoluntaryExit `ssz-max:"16"` SyncAggregate SyncAggregate + ExecutionPayload ExecutionPayload BlsToExecutionChanges []SignedBlsToExecutionChange `ssz-max:"16"` BlobKzgCommitments [][48]byte `ssz-max:"4096"` } @@ -193,3 +198,210 @@ type Bootstrap struct { CurrentSyncCommittee SyncCommittee CurrentSyncCommitteeBranch []Bytes32 } + +func (h *Header) ToBytes() [][]byte { + data := make([][]byte, 5) + + slotBytes := make([]byte, 8) + proposerIndexBytes := make([]byte, 8) + + binary.LittleEndian.PutUint64(slotBytes, h.Slot) + binary.LittleEndian.PutUint64(proposerIndexBytes, h.ProposerIndex) + + data[0] = slotBytes + data[1] = proposerIndexBytes + data[2] = h.ParentRoot[:] + data[3] = h.StateRoot[:] + data[4] = h.BodyRoot[:] + + return data +} + +func (sc *SyncCommittee) ToBytes() [][]byte { + data := make([][]byte, 513) // 512 Pubkeys + 1 AggregatePubkey + + // Convert each BLS pubkey in Pubkeys array to []byte + for i, pubkey := range sc.Pubkeys { + data[i] = pubkey[:] // Convert BLSPubKey (assumed to be a fixed size array) to []byte + } + + // Convert the AggregatePubkey to []byte + data[512] = sc.AggregatePubkey[:] + + return data +} + +func (body *BeaconBlockBody) ToBytes() [][]byte { + var data [][]byte + + // RandaoReveal + data = append(data, body.RandaoReveal[:]) + + // Eth1Data + data = append(data, body.Eth1Data.DepositRoot[:]) + depositCountBytes := make([]byte, 8) + binary.LittleEndian.PutUint64(depositCountBytes, body.Eth1Data.DepositCount) + data = append(data, depositCountBytes) + data = append(data, body.Eth1Data.BlockHash[:]) + + // Graffiti + data = append(data, body.Graffiti[:]) + + // ProposerSlashings + for _, proposerSlashing := range body.ProposerSlashings { + // SignedHeader1 + slotBytes := make([]byte, 8) + proposerIndexBytes := make([]byte, 8) + binary.LittleEndian.PutUint64(slotBytes, proposerSlashing.SignedHeader1.Message.Slot) + binary.LittleEndian.PutUint64(proposerIndexBytes, proposerSlashing.SignedHeader1.Message.ProposerIndex) + data = append(data, slotBytes) + data = append(data, proposerIndexBytes) + data = append(data, proposerSlashing.SignedHeader1.Message.ParentRoot[:]) + data = append(data, proposerSlashing.SignedHeader1.Message.StateRoot[:]) + data = append(data, proposerSlashing.SignedHeader1.Message.BodyRoot[:]) + data = append(data, proposerSlashing.SignedHeader1.Signature[:]) + + // SignedHeader2 + slotBytes = make([]byte, 8) + proposerIndexBytes = make([]byte, 8) + binary.LittleEndian.PutUint64(slotBytes, proposerSlashing.SignedHeader2.Message.Slot) + binary.LittleEndian.PutUint64(proposerIndexBytes, proposerSlashing.SignedHeader2.Message.ProposerIndex) + data = append(data, slotBytes) + data = append(data, proposerIndexBytes) + data = append(data, proposerSlashing.SignedHeader2.Message.ParentRoot[:]) + data = append(data, proposerSlashing.SignedHeader2.Message.StateRoot[:]) + data = append(data, proposerSlashing.SignedHeader2.Message.BodyRoot[:]) + data = append(data, proposerSlashing.SignedHeader2.Signature[:]) + } + + // AttesterSlashings + for _, attesterSlashing := range body.AttesterSlashings { + // Attestation1 + for _, index := range attesterSlashing.Attestation1.AttestingIndices { + indexBytes := make([]byte, 8) + binary.LittleEndian.PutUint64(indexBytes, index) + data = append(data, indexBytes) + } + data = append(data, attesterSlashing.Attestation1.Data.BeaconBlockRoot[:]) + data = append(data, attesterSlashing.Attestation1.Signature[:]) + + // Attestation2 + for _, index := range attesterSlashing.Attestation2.AttestingIndices { + indexBytes := make([]byte, 8) + binary.LittleEndian.PutUint64(indexBytes, index) + data = append(data, indexBytes) + } + data = append(data, attesterSlashing.Attestation2.Data.BeaconBlockRoot[:]) + data = append(data, attesterSlashing.Attestation2.Signature[:]) + } + + // Attestations + for _, attestation := range body.Attestations { + // AggregationBits (Bitlist) + byteLen := (len(attestation.AggregationBits) + 7) / 8 + aggregationBitsBytes := make([]byte, byteLen) + for i, bit := range attestation.AggregationBits { + if bit { + aggregationBitsBytes[i/8] |= 1 << (i % 8) + } + } + data = append(data, aggregationBitsBytes) + + // AttestationData + slotBytes := make([]byte, 8) + indexBytes := make([]byte, 8) + binary.LittleEndian.PutUint64(slotBytes, attestation.Data.Slot) + binary.LittleEndian.PutUint64(indexBytes, attestation.Data.Index) + data = append(data, slotBytes) + data = append(data, indexBytes) + data = append(data, attestation.Data.BeaconBlockRoot[:]) + data = append(data, attestation.Data.Source.Root[:]) + data = append(data, attestation.Data.Target.Root[:]) + + // Signature + data = append(data, attestation.Signature[:]) + } + + // Deposits + for _, deposit := range body.Deposits { + // Proof + for _, proof := range deposit.Proof { + data = append(data, proof[:]) + } + + // DepositData + data = append(data, deposit.Data.Pubkey[:]) + data = append(data, deposit.Data.WithdrawalCredentials[:]) + amountBytes := make([]byte, 8) + binary.LittleEndian.PutUint64(amountBytes, deposit.Data.Amount) + data = append(data, amountBytes) + data = append(data, deposit.Data.Signature[:]) + } + + // VoluntaryExits + for _, exit := range body.VoluntaryExits { + epochBytes := make([]byte, 8) + validatorIndexBytes := make([]byte, 8) + binary.LittleEndian.PutUint64(epochBytes, exit.Message.Epoch) + binary.LittleEndian.PutUint64(validatorIndexBytes, exit.Message.ValidatorIndex) + data = append(data, epochBytes) + data = append(data, validatorIndexBytes) + data = append(data, exit.Signature[:]) + } + + // SyncAggregate + data = append(data, body.SyncAggregate.SyncCommitteeBits[:]) + data = append(data, body.SyncAggregate.SyncCommitteeSignature[:]) + + // ExecutionPayload + data = append(data, body.ExecutionPayload.ParentHash[:]) + data = append(data, body.ExecutionPayload.FeeRecipient[:]) + data = append(data, body.ExecutionPayload.StateRoot[:]) + data = append(data, body.ExecutionPayload.ReceiptsRoot[:]) + data = append(data, body.ExecutionPayload.LogsBloom[:]) + data = append(data, body.ExecutionPayload.PrevRandao[:]) + blockNumberBytes := make([]byte, 8) + binary.LittleEndian.PutUint64(blockNumberBytes, body.ExecutionPayload.BlockNumber) + data = append(data, blockNumberBytes) + gasLimitBytes := make([]byte, 8) + binary.LittleEndian.PutUint64(gasLimitBytes, body.ExecutionPayload.GasLimit) + data = append(data, gasLimitBytes) + gasUsedBytes := make([]byte, 8) + binary.LittleEndian.PutUint64(gasUsedBytes, body.ExecutionPayload.GasUsed) + data = append(data, gasUsedBytes) + timestampBytes := make([]byte, 8) + binary.LittleEndian.PutUint64(timestampBytes, body.ExecutionPayload.Timestamp) + data = append(data, timestampBytes) + data = append(data, body.ExecutionPayload.ExtraData[:]) + baseFeePerGasBytes := make([]byte, 8) + binary.LittleEndian.PutUint64(baseFeePerGasBytes, body.ExecutionPayload.BaseFeePerGas) + data = append(data, baseFeePerGasBytes) + data = append(data, body.ExecutionPayload.BlockHash[:]) + + // Transactions (list of transactions) + for _, tx := range body.ExecutionPayload.Transactions { + data = append(data, tx[:]) // Assuming tx.Data() returns the transaction bytes + } + + // Withdrawals + for _, withdrawal := range body.ExecutionPayload.Withdrawals { + indexBytes := make([]byte, 8) + validatorIndexBytes := make([]byte, 8) + binary.LittleEndian.PutUint64(indexBytes, withdrawal.Index) + binary.LittleEndian.PutUint64(validatorIndexBytes, withdrawal.ValidatorIndex) + data = append(data, indexBytes) + data = append(data, validatorIndexBytes) + data = append(data, withdrawal.Address[:]) + amountBytes := make([]byte, 8) + binary.LittleEndian.PutUint64(amountBytes, withdrawal.Amount) + data = append(data, amountBytes) + } + + // BlobKzgCommitments + for _, commitment := range body.BlobKzgCommitments { + data = append(data, commitment[:]) + } + + return data +} + From d36725f9804abc72eecc95dfc51357a76f7c6ece Mon Sep 17 00:00:00 2001 From: Sambhav Jain <136801346+DarkLord017@users.noreply.github.com> Date: Sat, 21 Sep 2024 13:23:06 +0530 Subject: [PATCH 10/11] Update utils.go for efficient byte handling --- consensus/utils.go | 47 +++++++++++++++++++++++++++++++--------------- 1 file changed, 32 insertions(+), 15 deletions(-) diff --git a/consensus/utils.go b/consensus/utils.go index dd641f3..4b006c5 100644 --- a/consensus/utils.go +++ b/consensus/utils.go @@ -3,8 +3,9 @@ package consensus import ( "bytes" "crypto/sha256" - "errors" + "encoding/json" "fmt" + "log" "math" "github.com/BlocSoc-iitr/selene/config" @@ -14,12 +15,15 @@ import ( "github.com/ethereum/go-ethereum/crypto" ) -func TreeHashRoot(leaves [][]byte) ([]byte, error) { - if len(leaves) == 0 { - return nil, fmt.Errorf("no leaves provided") +// TreeHashRoot computes the Merkle root from the provided leaves in a flat []byte slice. +func TreeHashRoot(data []byte) ([]byte, error) { + // Convert the input data into a slice of leaves + leaves, err := bytesToLeaves(data) + if err != nil { + return nil, err } - nodes := leaves + nodes := leaves // Start with the leaf nodes for len(nodes) > 1 { var newLevel [][]byte @@ -39,9 +43,18 @@ func TreeHashRoot(leaves [][]byte) ([]byte, error) { nodes = newLevel } + // Return the root hash return nodes[0], 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 @@ -77,7 +90,7 @@ func isAggregateValid(sigBytes consensus_core.SignatureBytes, msg [32]byte, pks func isProofValid( attestedHeader *consensus_core.Header, - leafObject [][]byte, // Byte slice of the leaf object + 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 { @@ -152,7 +165,7 @@ func GetParticipatingKeys(committee *consensus_core.SyncCommittee, bitfield [64] numBits := len(bitfield) * 8 // Total number of bits if len(committee.Pubkeys) > numBits { - return nil, errors.New("bitfield is too short for the number of public keys") + return nil, fmt.Errorf("bitfield is too short for the number of public keys") } for i := 0; i < len(bitfield); i++ { @@ -198,17 +211,21 @@ type ForkData struct { GenesisValidatorRoot consensus_core.Bytes32 } -func (fd *ForkData) ToBytes() [][]byte { - data := make([][]byte, 2) - data[0] = fd.CurrentVersion[:] - data[1] = fd.GenesisValidatorRoot[:] +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 := make([][]byte, 2) - data[0] = sd.ObjectRoot[:] - data[1] = sd.Domain[:] +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 } From d388b612697a3dbd698016286f9cf6b897d3a767 Mon Sep 17 00:00:00 2001 From: Sambhav Jain <136801346+DarkLord017@users.noreply.github.com> Date: Sat, 21 Sep 2024 13:23:57 +0530 Subject: [PATCH 11/11] Update consensus_core.go for efficient byte handling --- consensus/consensus_core/consensus_core.go | 239 +++------------------ 1 file changed, 28 insertions(+), 211 deletions(-) diff --git a/consensus/consensus_core/consensus_core.go b/consensus/consensus_core/consensus_core.go index 15e3d5f..f0428c4 100644 --- a/consensus/consensus_core/consensus_core.go +++ b/consensus/consensus_core/consensus_core.go @@ -1,25 +1,26 @@ package consensus_core import ( - "encoding/binary" + "bytes" "github.com/BlocSoc-iitr/selene/consensus/types" + "github.com/ugorji/go/codec" ) -type BeaconBlock struct { - Slot uint64 - ProposerIndex uint64 - ParentRoot [32]byte - StateRoot [32]byte - Body BeaconBlockBody -} - type Bytes32 [32]byte type BLSPubKey [48]byte type Address [20]byte type LogsBloom = [256]byte type SignatureBytes [96]byte +type BeaconBlock struct { + Slot uint64 + ProposerIndex uint64 + ParentRoot Bytes32 + StateRoot Bytes32 + Body BeaconBlockBody +} + type Eth1Data struct { DepositRoot Bytes32 DepositCount uint64 @@ -77,7 +78,7 @@ type Attestation struct { } type Deposit struct { - Proof [33]Bytes32 //fixed size array + Proof [33]Bytes32 // fixed size array Data DepositData } @@ -199,209 +200,25 @@ type Bootstrap struct { CurrentSyncCommitteeBranch []Bytes32 } -func (h *Header) ToBytes() [][]byte { - data := make([][]byte, 5) - - slotBytes := make([]byte, 8) - proposerIndexBytes := make([]byte, 8) - - binary.LittleEndian.PutUint64(slotBytes, h.Slot) - binary.LittleEndian.PutUint64(proposerIndexBytes, h.ProposerIndex) - - data[0] = slotBytes - data[1] = proposerIndexBytes - data[2] = h.ParentRoot[:] - data[3] = h.StateRoot[:] - data[4] = h.BodyRoot[:] - - return data +// ToBytes serializes the Header struct to a byte slice. +func (h *Header) ToBytes() []byte { + var buf bytes.Buffer + enc := codec.NewEncoder(&buf, new(codec.JsonHandle)) + _ = enc.Encode(h) // Ignore error + return buf.Bytes() } -func (sc *SyncCommittee) ToBytes() [][]byte { - data := make([][]byte, 513) // 512 Pubkeys + 1 AggregatePubkey - - // Convert each BLS pubkey in Pubkeys array to []byte - for i, pubkey := range sc.Pubkeys { - data[i] = pubkey[:] // Convert BLSPubKey (assumed to be a fixed size array) to []byte - } - - // Convert the AggregatePubkey to []byte - data[512] = sc.AggregatePubkey[:] - - return data +func (b *BeaconBlockBody) ToBytes() []byte { + var buf bytes.Buffer + enc := codec.NewEncoder(&buf, new(codec.JsonHandle)) + _ = enc.Encode(b) // Ignore error + return buf.Bytes() } -func (body *BeaconBlockBody) ToBytes() [][]byte { - var data [][]byte - - // RandaoReveal - data = append(data, body.RandaoReveal[:]) - - // Eth1Data - data = append(data, body.Eth1Data.DepositRoot[:]) - depositCountBytes := make([]byte, 8) - binary.LittleEndian.PutUint64(depositCountBytes, body.Eth1Data.DepositCount) - data = append(data, depositCountBytes) - data = append(data, body.Eth1Data.BlockHash[:]) - - // Graffiti - data = append(data, body.Graffiti[:]) - - // ProposerSlashings - for _, proposerSlashing := range body.ProposerSlashings { - // SignedHeader1 - slotBytes := make([]byte, 8) - proposerIndexBytes := make([]byte, 8) - binary.LittleEndian.PutUint64(slotBytes, proposerSlashing.SignedHeader1.Message.Slot) - binary.LittleEndian.PutUint64(proposerIndexBytes, proposerSlashing.SignedHeader1.Message.ProposerIndex) - data = append(data, slotBytes) - data = append(data, proposerIndexBytes) - data = append(data, proposerSlashing.SignedHeader1.Message.ParentRoot[:]) - data = append(data, proposerSlashing.SignedHeader1.Message.StateRoot[:]) - data = append(data, proposerSlashing.SignedHeader1.Message.BodyRoot[:]) - data = append(data, proposerSlashing.SignedHeader1.Signature[:]) - - // SignedHeader2 - slotBytes = make([]byte, 8) - proposerIndexBytes = make([]byte, 8) - binary.LittleEndian.PutUint64(slotBytes, proposerSlashing.SignedHeader2.Message.Slot) - binary.LittleEndian.PutUint64(proposerIndexBytes, proposerSlashing.SignedHeader2.Message.ProposerIndex) - data = append(data, slotBytes) - data = append(data, proposerIndexBytes) - data = append(data, proposerSlashing.SignedHeader2.Message.ParentRoot[:]) - data = append(data, proposerSlashing.SignedHeader2.Message.StateRoot[:]) - data = append(data, proposerSlashing.SignedHeader2.Message.BodyRoot[:]) - data = append(data, proposerSlashing.SignedHeader2.Signature[:]) - } - - // AttesterSlashings - for _, attesterSlashing := range body.AttesterSlashings { - // Attestation1 - for _, index := range attesterSlashing.Attestation1.AttestingIndices { - indexBytes := make([]byte, 8) - binary.LittleEndian.PutUint64(indexBytes, index) - data = append(data, indexBytes) - } - data = append(data, attesterSlashing.Attestation1.Data.BeaconBlockRoot[:]) - data = append(data, attesterSlashing.Attestation1.Signature[:]) - - // Attestation2 - for _, index := range attesterSlashing.Attestation2.AttestingIndices { - indexBytes := make([]byte, 8) - binary.LittleEndian.PutUint64(indexBytes, index) - data = append(data, indexBytes) - } - data = append(data, attesterSlashing.Attestation2.Data.BeaconBlockRoot[:]) - data = append(data, attesterSlashing.Attestation2.Signature[:]) - } - - // Attestations - for _, attestation := range body.Attestations { - // AggregationBits (Bitlist) - byteLen := (len(attestation.AggregationBits) + 7) / 8 - aggregationBitsBytes := make([]byte, byteLen) - for i, bit := range attestation.AggregationBits { - if bit { - aggregationBitsBytes[i/8] |= 1 << (i % 8) - } - } - data = append(data, aggregationBitsBytes) - - // AttestationData - slotBytes := make([]byte, 8) - indexBytes := make([]byte, 8) - binary.LittleEndian.PutUint64(slotBytes, attestation.Data.Slot) - binary.LittleEndian.PutUint64(indexBytes, attestation.Data.Index) - data = append(data, slotBytes) - data = append(data, indexBytes) - data = append(data, attestation.Data.BeaconBlockRoot[:]) - data = append(data, attestation.Data.Source.Root[:]) - data = append(data, attestation.Data.Target.Root[:]) - - // Signature - data = append(data, attestation.Signature[:]) - } - - // Deposits - for _, deposit := range body.Deposits { - // Proof - for _, proof := range deposit.Proof { - data = append(data, proof[:]) - } - - // DepositData - data = append(data, deposit.Data.Pubkey[:]) - data = append(data, deposit.Data.WithdrawalCredentials[:]) - amountBytes := make([]byte, 8) - binary.LittleEndian.PutUint64(amountBytes, deposit.Data.Amount) - data = append(data, amountBytes) - data = append(data, deposit.Data.Signature[:]) - } - - // VoluntaryExits - for _, exit := range body.VoluntaryExits { - epochBytes := make([]byte, 8) - validatorIndexBytes := make([]byte, 8) - binary.LittleEndian.PutUint64(epochBytes, exit.Message.Epoch) - binary.LittleEndian.PutUint64(validatorIndexBytes, exit.Message.ValidatorIndex) - data = append(data, epochBytes) - data = append(data, validatorIndexBytes) - data = append(data, exit.Signature[:]) - } - - // SyncAggregate - data = append(data, body.SyncAggregate.SyncCommitteeBits[:]) - data = append(data, body.SyncAggregate.SyncCommitteeSignature[:]) - - // ExecutionPayload - data = append(data, body.ExecutionPayload.ParentHash[:]) - data = append(data, body.ExecutionPayload.FeeRecipient[:]) - data = append(data, body.ExecutionPayload.StateRoot[:]) - data = append(data, body.ExecutionPayload.ReceiptsRoot[:]) - data = append(data, body.ExecutionPayload.LogsBloom[:]) - data = append(data, body.ExecutionPayload.PrevRandao[:]) - blockNumberBytes := make([]byte, 8) - binary.LittleEndian.PutUint64(blockNumberBytes, body.ExecutionPayload.BlockNumber) - data = append(data, blockNumberBytes) - gasLimitBytes := make([]byte, 8) - binary.LittleEndian.PutUint64(gasLimitBytes, body.ExecutionPayload.GasLimit) - data = append(data, gasLimitBytes) - gasUsedBytes := make([]byte, 8) - binary.LittleEndian.PutUint64(gasUsedBytes, body.ExecutionPayload.GasUsed) - data = append(data, gasUsedBytes) - timestampBytes := make([]byte, 8) - binary.LittleEndian.PutUint64(timestampBytes, body.ExecutionPayload.Timestamp) - data = append(data, timestampBytes) - data = append(data, body.ExecutionPayload.ExtraData[:]) - baseFeePerGasBytes := make([]byte, 8) - binary.LittleEndian.PutUint64(baseFeePerGasBytes, body.ExecutionPayload.BaseFeePerGas) - data = append(data, baseFeePerGasBytes) - data = append(data, body.ExecutionPayload.BlockHash[:]) - - // Transactions (list of transactions) - for _, tx := range body.ExecutionPayload.Transactions { - data = append(data, tx[:]) // Assuming tx.Data() returns the transaction bytes - } - - // Withdrawals - for _, withdrawal := range body.ExecutionPayload.Withdrawals { - indexBytes := make([]byte, 8) - validatorIndexBytes := make([]byte, 8) - binary.LittleEndian.PutUint64(indexBytes, withdrawal.Index) - binary.LittleEndian.PutUint64(validatorIndexBytes, withdrawal.ValidatorIndex) - data = append(data, indexBytes) - data = append(data, validatorIndexBytes) - data = append(data, withdrawal.Address[:]) - amountBytes := make([]byte, 8) - binary.LittleEndian.PutUint64(amountBytes, withdrawal.Amount) - data = append(data, amountBytes) - } - - // BlobKzgCommitments - for _, commitment := range body.BlobKzgCommitments { - data = append(data, commitment[:]) - } - - return data +// ToBytes serializes the SyncCommittee struct to a byte slice. +func (sc *SyncCommittee) ToBytes() []byte { + var buf bytes.Buffer + enc := codec.NewEncoder(&buf, new(codec.JsonHandle)) + _ = enc.Encode(sc) // Ignore error + return buf.Bytes() } -