diff --git a/cl/aggregation/pool_impl.go b/cl/aggregation/pool_impl.go index df55512b9fc..55ca8654332 100644 --- a/cl/aggregation/pool_impl.go +++ b/cl/aggregation/pool_impl.go @@ -19,12 +19,12 @@ package aggregation import ( "context" "errors" - "fmt" "sync" "time" "github.com/Giulio2002/bls" "github.com/erigontech/erigon-lib/common" + "github.com/erigontech/erigon-lib/log/v3" "github.com/erigontech/erigon/cl/clparams" "github.com/erigontech/erigon/cl/cltypes/solid" "github.com/erigontech/erigon/cl/phase1/core/state/lru" @@ -44,7 +44,7 @@ type aggregationPoolImpl struct { netConfig *clparams.NetworkConfig ethClock eth_clock.EthereumClock aggregatesLock sync.RWMutex - aggregates map[common.Hash]*solid.Attestation + aggregates map[common.Hash]*solid.Attestation // don't need this anymore after electra upgrade // aggregationInCommittee is a cache for aggregation in committee, which is used after electra upgrade aggregatesInCommittee *lru.CacheWithTTL[keyAggrInCommittee, *solid.Attestation] } @@ -78,68 +78,48 @@ func (p *aggregationPoolImpl) AddAttestation(inAtt *solid.Attestation) error { if err != nil { return err } - p.aggregatesLock.Lock() - defer p.aggregatesLock.Unlock() - att, ok := p.aggregates[hashRoot] - if !ok { - p.aggregates[hashRoot] = inAtt.Copy() - return nil - } - - if utils.IsOverlappingSSZBitlist(att.AggregationBits.Bytes(), inAtt.AggregationBits.Bytes()) { - // the on bit is already set, so ignore - return ErrIsSuperset - } - // merge signature - baseSig := att.Signature - inSig := inAtt.Signature - merged, err := blsAggregate([][]byte{baseSig[:], inSig[:]}) - if err != nil { - return err - } - if len(merged) != 96 { - return errors.New("merged signature is too long") - } - var mergedSig [96]byte - copy(mergedSig[:], merged) - - epoch := p.ethClock.GetEpochAtSlot(att.Data.Slot) + epoch := p.ethClock.GetEpochAtSlot(inAtt.Data.Slot) clversion := p.ethClock.StateVersionByEpoch(epoch) if clversion.BeforeOrEqual(clparams.DenebVersion) { - // merge aggregation bits - mergedBits, err := att.AggregationBits.Union(inAtt.AggregationBits) - if err != nil { - return err + p.aggregatesLock.Lock() + defer p.aggregatesLock.Unlock() + att, ok := p.aggregates[hashRoot] + if !ok { + p.aggregates[hashRoot] = inAtt.Copy() + return nil } - // update attestation - p.aggregates[hashRoot] = &solid.Attestation{ - AggregationBits: mergedBits, - Data: att.Data, - Signature: mergedSig, + + if utils.IsOverlappingSSZBitlist(att.AggregationBits.Bytes(), inAtt.AggregationBits.Bytes()) { + // the on bit is already set, so ignore + return ErrIsSuperset } - } else { - // Electra and after case - aggrBitSize := p.beaconConfig.MaxCommitteesPerSlot * p.beaconConfig.MaxValidatorsPerCommittee - mergedAggrBits, err := att.AggregationBits.Union(inAtt.AggregationBits) + // merge signature + baseSig := att.Signature + inSig := inAtt.Signature + merged, err := blsAggregate([][]byte{baseSig[:], inSig[:]}) if err != nil { return err } - if mergedAggrBits.Cap() != int(aggrBitSize) { - return fmt.Errorf("incorrect aggregation bits size: %d", mergedAggrBits.Cap()) + if len(merged) != 96 { + return errors.New("merged signature is too long") } - mergedCommitteeBits, err := att.CommitteeBits.Union(inAtt.CommitteeBits) + var mergedSig [96]byte + copy(mergedSig[:], merged) + + // merge aggregation bits + mergedBits, err := att.AggregationBits.Merge(inAtt.AggregationBits) if err != nil { return err } + // update attestation p.aggregates[hashRoot] = &solid.Attestation{ - AggregationBits: mergedAggrBits, - CommitteeBits: mergedCommitteeBits, + AggregationBits: mergedBits, Data: att.Data, Signature: mergedSig, } - - // aggregate by committee + } else { + // Electra and after case, aggregate by committee p.aggregateByCommittee(inAtt) } return nil @@ -166,9 +146,15 @@ func (p *aggregationPoolImpl) aggregateByCommittee(inAtt *solid.Attestation) err return nil } - // merge aggregation bits and signature - mergedAggrBits, err := att.AggregationBits.Union(inAtt.AggregationBits) + if utils.IsOverlappingSSZBitlist(att.AggregationBits.Bytes(), inAtt.AggregationBits.Bytes()) { + // the on bit is already set, so ignore + return ErrIsSuperset + } + + // It's fine to directly merge aggregation bits here, because the attestation is from the same committee + mergedAggrBits, err := att.AggregationBits.Merge(inAtt.AggregationBits) if err != nil { + log.Debug("failed to merge aggregation bits", "err", err) return err } merged, err := blsAggregate([][]byte{att.Signature[:], inAtt.Signature[:]}) diff --git a/cl/aggregation/pool_test.go b/cl/aggregation/pool_test.go index 8dd22dbbd13..6b294b8cd93 100644 --- a/cl/aggregation/pool_test.go +++ b/cl/aggregation/pool_test.go @@ -40,22 +40,22 @@ var ( }, } att1_1 = &solid.Attestation{ - AggregationBits: solid.BitlistFromBytes([]byte{0b00000001, 0, 0, 0}, 2048), + AggregationBits: solid.BitlistFromBytes([]byte{0b00000011}, 2048), Data: attData1, Signature: [96]byte{'a', 'b', 'c', 'd', 'e', 'f'}, } att1_2 = &solid.Attestation{ - AggregationBits: solid.BitlistFromBytes([]byte{0b00000001, 0, 0, 0}, 2048), + AggregationBits: solid.BitlistFromBytes([]byte{0b00000011}, 2048), Data: attData1, Signature: [96]byte{'d', 'e', 'f', 'g', 'h', 'i'}, } att1_3 = &solid.Attestation{ - AggregationBits: solid.BitlistFromBytes([]byte{0b00000100, 0, 0, 0}, 2048), + AggregationBits: solid.BitlistFromBytes([]byte{0b00001100}, 2048), Data: attData1, Signature: [96]byte{'g', 'h', 'i', 'j', 'k', 'l'}, } att1_4 = &solid.Attestation{ - AggregationBits: solid.BitlistFromBytes([]byte{0b00100000, 0, 0, 0}, 2048), + AggregationBits: solid.BitlistFromBytes([]byte{0b01100000}, 2048), Data: attData1, Signature: [96]byte{'m', 'n', 'o', 'p', 'q', 'r'}, } @@ -72,7 +72,7 @@ var ( }, } att2_1 = &solid.Attestation{ - AggregationBits: solid.BitlistFromBytes([]byte{0b00000001, 0, 0, 0}, 2048), + AggregationBits: solid.BitlistFromBytes([]byte{0b00000001}, 2048), Data: attData2, Signature: [96]byte{'t', 'e', 's', 't', 'i', 'n'}, } @@ -107,21 +107,21 @@ func (t *PoolTestSuite) TearDownTest() { func (t *PoolTestSuite) TestAddAttestationElectra() { cBits1 := solid.NewBitVector(64) - cBits1.SetBitAt(0, true) + cBits1.SetBitAt(10, true) cBits2 := solid.NewBitVector(64) cBits2.SetBitAt(10, true) expectedCommitteeBits := solid.NewBitVector(64) - expectedCommitteeBits.SetBitAt(0, true) + expectedCommitteeBits.SetBitAt(10, true) expectedCommitteeBits.SetBitAt(10, true) att1 := &solid.Attestation{ - AggregationBits: solid.BitlistFromBytes([]byte{0b00000001, 0, 0, 0}, 2048*64), + AggregationBits: solid.BitlistFromBytes([]byte{0b00000011}, 2048*64), Data: attData1, Signature: [96]byte{'a', 'b', 'c', 'd', 'e', 'f'}, CommitteeBits: cBits1, } att2 := &solid.Attestation{ - AggregationBits: solid.BitlistFromBytes([]byte{0b00000000, 0b00001000, 0, 0}, 2048*64), + AggregationBits: solid.BitlistFromBytes([]byte{0b00001100}, 2048*64), Data: attData1, Signature: [96]byte{'d', 'e', 'f', 'g', 'h', 'i'}, CommitteeBits: cBits2, @@ -141,11 +141,11 @@ func (t *PoolTestSuite) TestAddAttestationElectra() { }, hashRoot: attData1Root, mockFunc: func() { - t.mockEthClock.EXPECT().GetEpochAtSlot(gomock.Any()).Return(uint64(1)).Times(1) - t.mockEthClock.EXPECT().StateVersionByEpoch(gomock.Any()).Return(clparams.ElectraVersion).Times(1) + t.mockEthClock.EXPECT().GetEpochAtSlot(gomock.Any()).Return(uint64(1)).Times(2) + t.mockEthClock.EXPECT().StateVersionByEpoch(gomock.Any()).Return(clparams.ElectraVersion).Times(2) }, expect: &solid.Attestation{ - AggregationBits: solid.BitlistFromBytes([]byte{0b0000001, 0b00001000, 0, 0}, 2048*64), + AggregationBits: solid.BitlistFromBytes([]byte{0b00001101}, 2048*64), Data: attData1, Signature: mockAggrResult, CommitteeBits: expectedCommitteeBits, @@ -162,9 +162,7 @@ func (t *PoolTestSuite) TestAddAttestationElectra() { for i := range tc.atts { pool.AddAttestation(tc.atts[i]) } - att := pool.GetAggregatationByRoot(tc.hashRoot) - //h1, _ := tc.expect.HashSSZ() - //h2, _ := att.HashSSZ() + att := pool.GetAggregatationByRootAndCommittee(tc.hashRoot, 10) t.Equal(tc.expect, att, tc.name) } } @@ -184,7 +182,11 @@ func (t *PoolTestSuite) TestAddAttestation() { att2_1, }, hashRoot: attData1Root, - expect: att1_1, + mockFunc: func() { + t.mockEthClock.EXPECT().GetEpochAtSlot(gomock.Any()).Return(uint64(1)).AnyTimes() + t.mockEthClock.EXPECT().StateVersionByEpoch(gomock.Any()).Return(clparams.DenebVersion).AnyTimes() + }, + expect: att1_1, }, { name: "att1_2 is a super set of att1_1. skip att1_1", @@ -194,7 +196,11 @@ func (t *PoolTestSuite) TestAddAttestation() { att2_1, // none of its business }, hashRoot: attData1Root, - expect: att1_2, + mockFunc: func() { + t.mockEthClock.EXPECT().GetEpochAtSlot(gomock.Any()).Return(uint64(1)).AnyTimes() + t.mockEthClock.EXPECT().StateVersionByEpoch(gomock.Any()).Return(clparams.DenebVersion).AnyTimes() + }, + expect: att1_2, }, { name: "merge att1_2, att1_3, att1_4", @@ -209,7 +215,7 @@ func (t *PoolTestSuite) TestAddAttestation() { t.mockEthClock.EXPECT().StateVersionByEpoch(gomock.Any()).Return(clparams.DenebVersion).AnyTimes() }, expect: &solid.Attestation{ - AggregationBits: solid.BitlistFromBytes([]byte{0b00100101, 0, 0, 0}, 2048), + AggregationBits: solid.BitlistFromBytes([]byte{0b01100101}, 2048), Data: attData1, Signature: mockAggrResult, }, @@ -226,8 +232,6 @@ func (t *PoolTestSuite) TestAddAttestation() { pool.AddAttestation(tc.atts[i]) } att := pool.GetAggregatationByRoot(tc.hashRoot) - //h1, _ := tc.expect.HashSSZ() - //h2, _ := att.HashSSZ() t.Equal(tc.expect, att, tc.name) } } diff --git a/cl/beacon/handler/block_production.go b/cl/beacon/handler/block_production.go index a1a43c00bd8..fbf778b0e0f 100644 --- a/cl/beacon/handler/block_production.go +++ b/cl/beacon/handler/block_production.go @@ -352,6 +352,7 @@ func (a *ApiHandler) GetEthV3ValidatorBlock( "proposerIndex", block.ProposerIndex, "slot", targetSlot, "state_root", block.StateRoot, + "attestations", block.BeaconBody.Attestations.Len(), "execution_value", block.GetExecutionValue().Uint64(), "version", block.Version(), "blinded", block.IsBlinded(), @@ -624,11 +625,11 @@ func (a *ApiHandler) produceBeaconBody( retryTime := 10 * time.Millisecond secsDiff := (targetSlot - baseBlock.Slot) * a.beaconChainCfg.SecondsPerSlot feeRecipient, _ := a.validatorParams.GetFeeRecipient(proposerIndex) - var withdrawals []*types.Withdrawal clWithdrawals, _ := state.ExpectedWithdrawals( baseState, targetSlot/a.beaconChainCfg.SlotsPerEpoch, ) + withdrawals := []*types.Withdrawal{} for _, w := range clWithdrawals { withdrawals = append(withdrawals, &types.Withdrawal{ Index: w.Index, @@ -1067,7 +1068,6 @@ func (a *ApiHandler) parseRequestBeaconBlock( if err := json.NewDecoder(r.Body).Decode(block); err != nil { return nil, err } - block.SignedBlock.Block.SetVersion(version) return block, nil case "application/octet-stream": octect, err := io.ReadAll(r.Body) @@ -1077,7 +1077,6 @@ func (a *ApiHandler) parseRequestBeaconBlock( if err := block.DecodeSSZ(octect, int(version)); err != nil { return nil, err } - block.SignedBlock.Block.SetVersion(version) return block, nil } return nil, errors.New("invalid content type") @@ -1216,12 +1215,7 @@ type attestationCandidate struct { func (a *ApiHandler) findBestAttestationsForBlockProduction( s abstract.BeaconState, ) *solid.ListSSZ[*solid.Attestation] { - currentVersion := s.Version() - aggBitsSize := int(a.beaconChainCfg.MaxValidatorsPerCommittee) - if currentVersion.AfterOrEqual(clparams.ElectraVersion) { - aggBitsSize = int(a.beaconChainCfg.MaxValidatorsPerCommittee * - a.beaconChainCfg.MaxCommitteesPerSlot) - } + stateVersion := s.Version() // Group attestations by their data root hashToAtts := make(map[libcommon.Hash][]*solid.Attestation) for _, candidate := range a.operationsPool.AttestationsPool.Raw() { @@ -1230,7 +1224,7 @@ func (a *ApiHandler) findBestAttestationsForBlockProduction( } attVersion := a.beaconChainCfg.GetCurrentStateVersion(candidate.Data.Slot / a.beaconChainCfg.SlotsPerEpoch) - if currentVersion.AfterOrEqual(clparams.ElectraVersion) && + if stateVersion.AfterOrEqual(clparams.ElectraVersion) && attVersion.Before(clparams.ElectraVersion) { // Because the on chain Attestation container changes, attestations from the prior fork can’t be included // into post-electra blocks. Therefore the first block after the fork may have zero attestations. @@ -1252,7 +1246,8 @@ func (a *ApiHandler) findBestAttestationsForBlockProduction( candidateAggregationBits := candidate.AggregationBits.Bytes() for _, curAtt := range hashToAtts[dataRoot] { currAggregationBitsBytes := curAtt.AggregationBits.Bytes() - if !utils.IsOverlappingSSZBitlist(currAggregationBitsBytes, candidateAggregationBits) { + if stateVersion <= clparams.DenebVersion && + !utils.IsOverlappingSSZBitlist(currAggregationBitsBytes, candidateAggregationBits) { // merge signatures candidateSig := candidate.Signature curSig := curAtt.Signature @@ -1262,24 +1257,38 @@ func (a *ApiHandler) findBestAttestationsForBlockProduction( continue } // merge aggregation bits - mergedAggBits := solid.NewBitList(0, aggBitsSize) - for i := 0; i < len(currAggregationBitsBytes); i++ { - mergedAggBits.Append(currAggregationBitsBytes[i] | candidateAggregationBits[i]) + mergedAggBits, err := curAtt.AggregationBits.Merge(candidate.AggregationBits) + if err != nil { + log.Warn("[Block Production] Cannot merge aggregation bits", "err", err) + continue } var buf [96]byte copy(buf[:], mergeSig) curAtt.Signature = buf curAtt.AggregationBits = mergedAggBits - if attVersion.AfterOrEqual(clparams.ElectraVersion) { - // merge committee_bits for electra - mergedCommitteeBits, err := curAtt.CommitteeBits.Union(candidate.CommitteeBits) - if err != nil { - log.Warn("[Block Production] Cannot merge committee bits", "err", err) - continue - } - curAtt.CommitteeBits = mergedCommitteeBits + mergeAny = true + } + if stateVersion >= clparams.ElectraVersion { + // merge in electra way + mergedAggrBits, ok := a.tryMergeAggregationBits(s, curAtt, candidate) + if !ok { + continue } - + mergedCommitteeBits, err := curAtt.CommitteeBits.Union(candidate.CommitteeBits) + if err != nil { + continue + } + // merge signatures + candidateSig := candidate.Signature + curSig := curAtt.Signature + mergeSig, err := bls.AggregateSignatures([][]byte{candidateSig[:], curSig[:]}) + if err != nil { + log.Warn("[Block Production] Cannot merge signatures", "err", err) + continue + } + curAtt.AggregationBits = mergedAggrBits + curAtt.CommitteeBits = mergedCommitteeBits + copy(curAtt.Signature[:], mergeSig) mergeAny = true } } @@ -1327,6 +1336,74 @@ func (a *ApiHandler) findBestAttestationsForBlockProduction( return ret } +func (a *ApiHandler) tryMergeAggregationBits(state abstract.BeaconState, att1, att2 *solid.Attestation) (*solid.BitList, bool) { + // after electra fork, aggregation_bits contains only the attester bit map of those committee appearing in committee_bits + // ref: https://github.com/ethereum/consensus-specs/blob/dev/specs/electra/validator.md#attestations + slot := att1.Data.Slot + committees1 := att1.CommitteeBits.GetOnIndices() + committees2 := att2.CommitteeBits.GetOnIndices() + bitSlice := solid.NewBitSlice() + index1, index2 := 0, 0 + committeeOffset1, committeeOffset2 := 0, 0 + + // appendBits is a helper func to append the aggregation bits of the committee to the bitSlice + appendBits := func(bitSlice *solid.BitSlice, committeeIndex int, att *solid.Attestation, offset int) (*solid.BitSlice, int) { + members, err := state.GetBeaconCommitee(slot, uint64(committeeIndex)) + if err != nil { + log.Warn("[Block Production] Cannot get committee members", "err", err) + return nil, 0 + } + for i := range members { + bitSlice.AppendBit(att.AggregationBits.GetBitAt(offset + i)) + } + return bitSlice, offset + len(members) + } + + // similar to merge sort + for index1 < len(committees1) || index2 < len(committees2) { + if index1 < len(committees1) && index2 < len(committees2) { + if committees1[index1] < committees2[index2] { + bitSlice, committeeOffset1 = appendBits(bitSlice, committees1[index1], att1, committeeOffset1) + index1++ + } else if committees1[index1] > committees2[index2] { + bitSlice, committeeOffset2 = appendBits(bitSlice, committees2[index2], att2, committeeOffset2) + index2++ + } else { + // check overlapping when the committee is the same + members, err := state.GetBeaconCommitee(slot, uint64(committees1[index1])) + if err != nil { + log.Warn("[Block Production] Cannot get committee members", "err", err) + return nil, false + } + bits1 := att1.AggregationBits + bits2 := att2.AggregationBits + for i := range members { + if bits1.GetBitAt(committeeOffset1+i) && bits2.GetBitAt(committeeOffset2+i) { + // overlapping + return nil, false + } else { + bitSlice.AppendBit(bits1.GetBitAt(committeeOffset1+i) || bits2.GetBitAt(committeeOffset2+i)) + } + } + committeeOffset1 += len(members) + committeeOffset2 += len(members) + index1++ + index2++ + } + } else if index1 < len(committees1) { + bitSlice, committeeOffset1 = appendBits(bitSlice, committees1[index1], att1, committeeOffset1) + index1++ + } else { + bitSlice, committeeOffset2 = appendBits(bitSlice, committees2[index2], att2, committeeOffset2) + index2++ + } + } + + bitSlice.AppendBit(true) // mark the end of the bitlist + mergedAggregationBits := solid.BitlistFromBytes(bitSlice.Bytes(), int(a.beaconChainCfg.MaxCommitteesPerSlot)*int(a.beaconChainCfg.MaxValidatorsPerCommittee)) + return mergedAggregationBits, true +} + // computeAttestationReward computes the reward for a specific attestation. func computeAttestationReward( s abstract.BeaconState, diff --git a/cl/clparams/config.go b/cl/clparams/config.go index 610922497a7..63ab3479b36 100644 --- a/cl/clparams/config.go +++ b/cl/clparams/config.go @@ -533,6 +533,8 @@ type BeaconChainConfig struct { DenebForkEpoch uint64 `yaml:"DENEB_FORK_EPOCH" spec:"true" json:"DENEB_FORK_EPOCH,string"` // DenebForkEpoch is used to represent the assigned fork epoch for Deneb. ElectraForkVersion ConfigForkVersion `yaml:"ELECTRA_FORK_VERSION" spec:"true" json:"ELECTRA_FORK_VERSION"` // ElectraForkVersion is used to represent the fork version for Electra. ElectraForkEpoch uint64 `yaml:"ELECTRA_FORK_EPOCH" spec:"true" json:"ELECTRA_FORK_EPOCH,string"` // ElectraForkEpoch is used to represent the assigned fork epoch for Electra. + FuluForkVersion ConfigForkVersion `yaml:"FULU_FORK_VERSION" spec:"true" json:"FULU_FORK_VERSION"` // FuluForkVersion is used to represent the fork version for Fulu. + FuluForkEpoch uint64 `yaml:"FULU_FORK_EPOCH" spec:"true" json:"FULU_FORK_EPOCH,string"` // FuluForkEpoch is used to represent the assigned fork epoch for Fulu. ForkVersionSchedule map[libcommon.Bytes4]VersionScheduleEntry `json:"-"` // Schedule of fork epochs by version. @@ -823,6 +825,8 @@ var MainnetBeaconConfig BeaconChainConfig = BeaconChainConfig{ DenebForkEpoch: 269568, ElectraForkVersion: 0x05000000, ElectraForkEpoch: math.MaxUint64, + FuluForkVersion: 0x06000000, + FuluForkEpoch: math.MaxUint64, // New values introduced in Altair hard fork 1. // Participation flag indices. diff --git a/cl/cltypes/beacon_block.go b/cl/cltypes/beacon_block.go index 5d4e930ef79..86ed448fb5d 100644 --- a/cl/cltypes/beacon_block.go +++ b/cl/cltypes/beacon_block.go @@ -162,10 +162,6 @@ func (b *BeaconBlock) Version() clparams.StateVersion { return b.Body.Version } -func (b *BeaconBlock) SetVersion(version clparams.StateVersion) { - b.Body.SetVersion(version) -} - func (b *BeaconBlock) EncodeSSZ(buf []byte) (dst []byte, err error) { return ssz2.MarshalSSZ(buf, b.Slot, b.ProposerIndex, b.ParentRoot[:], b.StateRoot[:], b.Body) } @@ -247,8 +243,8 @@ func NewBeaconBody(beaconCfg *clparams.BeaconChainConfig, version clparams.State ) if version.AfterOrEqual(clparams.ElectraVersion) { // upgrade to electra - maxAttSlashing = MaxAttesterSlashingsElectra - maxAttestation = MaxAttestationsElectra + maxAttSlashing = int(beaconCfg.MaxAttesterSlashingsElectra) + maxAttestation = int(beaconCfg.MaxAttestationsElectra) executionRequests = NewExecutionRequests(beaconCfg) } @@ -267,15 +263,6 @@ func NewBeaconBody(beaconCfg *clparams.BeaconChainConfig, version clparams.State Version: version, } } -func (b *BeaconBody) SetVersion(version clparams.StateVersion) { - b.Version = version - b.ExecutionPayload.SetVersion(version) - if version.AfterOrEqual(clparams.ElectraVersion) { - b.AttesterSlashings = solid.NewDynamicListSSZ[*AttesterSlashing](MaxAttesterSlashingsElectra) - b.Attestations = solid.NewDynamicListSSZ[*solid.Attestation](MaxAttestationsElectra) - b.ExecutionRequests = NewExecutionRequests(b.beaconCfg) - } -} func (b *BeaconBody) EncodeSSZ(dst []byte) ([]byte, error) { return ssz2.MarshalSSZ(dst, b.getSchema(false)...) @@ -616,7 +603,7 @@ func (b *DenebBeaconBlock) GetParentRoot() libcommon.Hash { } func (b *DenebBeaconBlock) GetBody() GenericBeaconBody { - return b.Block.GetBody() + return b.Block.Body } type DenebSignedBeaconBlock struct { diff --git a/cl/cltypes/block_production.go b/cl/cltypes/block_production.go index 197b5ef9150..d611164be21 100644 --- a/cl/cltypes/block_production.go +++ b/cl/cltypes/block_production.go @@ -69,7 +69,6 @@ func (b *BlindOrExecutionBeaconBlock) ToExecution() *DenebBeaconBlock { } DenebBeaconBlock := NewDenebBeaconBlock(b.Cfg, b.Version()) DenebBeaconBlock.Block = beaconBlock - DenebBeaconBlock.Block.SetVersion(b.Version()) for _, kzgProof := range b.KzgProofs { proof := KZGProof{} copy(proof[:], kzgProof[:]) diff --git a/cl/cltypes/eth1_block.go b/cl/cltypes/eth1_block.go index 5b4997684c1..111405cf4fd 100644 --- a/cl/cltypes/eth1_block.go +++ b/cl/cltypes/eth1_block.go @@ -110,10 +110,6 @@ func NewEth1BlockFromHeaderAndBody(header *types.Header, body *types.RawBody, be return block } -func (b *Eth1Block) SetVersion(version clparams.StateVersion) { - b.version = version -} - func (*Eth1Block) Static() bool { return false } diff --git a/cl/cltypes/solid/attestation.go b/cl/cltypes/solid/attestation.go index 2205143b91a..aa366182f20 100644 --- a/cl/cltypes/solid/attestation.go +++ b/cl/cltypes/solid/attestation.go @@ -173,8 +173,8 @@ func (a *Attestation) UnmarshalJSON(data []byte) error { // data: AttestationData // signature: BLSSignature type SingleAttestation struct { - CommitteeIndex uint64 `json:"committee_index"` - AttesterIndex uint64 `json:"attester_index"` + CommitteeIndex uint64 `json:"committee_index,string"` + AttesterIndex uint64 `json:"attester_index,string"` Data *AttestationData `json:"data"` Signature libcommon.Bytes96 `json:"signature"` } diff --git a/cl/cltypes/solid/bitlist.go b/cl/cltypes/solid/bitlist.go index 33d10ee4f44..2bf1392f8ff 100644 --- a/cl/cltypes/solid/bitlist.go +++ b/cl/cltypes/solid/bitlist.go @@ -35,8 +35,6 @@ type BitList struct { c int // current length of the bitlist l int - - hashBuf } // NewBitList creates a brand new BitList, just like when Zordon created the Power Rangers! @@ -128,6 +126,7 @@ func (u *BitList) Set(index int, v byte) { u.u[index] = v } +// removeMsb removes the most significant bit from the list, but doesn't change the length l. func (u *BitList) removeMsb() { for i := len(u.u) - 1; i >= 0; i-- { if u.u[i] != 0 { @@ -138,21 +137,26 @@ func (u *BitList) removeMsb() { } } -func (u *BitList) addMsb() { +// addMsb adds a most significant bit to the list, but doesn't change the length l. +func (u *BitList) addMsb() int { + byteLen := len(u.u) for i := len(u.u) - 1; i >= 0; i-- { if u.u[i] != 0 { msb := bits.Len8(u.u[i]) - if msb == 7 { + if msb == 8 { if i == len(u.u)-1 { u.u = append(u.u, 0) } + byteLen++ u.u[i+1] |= 1 } else { - u.u[i] |= 1 << uint(msb+1) + u.u[i] |= 1 << uint(msb) } break } + byteLen-- } + return byteLen } func (u *BitList) SetOnBit(bitIndex int) { @@ -168,8 +172,8 @@ func (u *BitList) SetOnBit(bitIndex int) { // set the bit u.u[bitIndex/8] |= 1 << uint(bitIndex%8) // set last bit - u.addMsb() - u.l = len(u.u) + byteLen := u.addMsb() + u.l = byteLen } // Length gives us the length of the bitlist, just like a roll call tells us how many Rangers there are. @@ -219,7 +223,15 @@ func (u *BitList) Bits() int { return 0 } // The most significant bit is present in the last byte in the array. - last := u.u[u.l-1] + var last byte + var byteLen int + for i := len(u.u) - 1; i >= 0; i-- { + if u.u[i] != 0 { + last = u.u[i] + byteLen = i + 1 + break + } + } // Determine the position of the most significant bit. msb := bits.Len8(last) @@ -230,7 +242,7 @@ func (u *BitList) Bits() int { // The absolute position of the most significant bit will be the number of // bits in the preceding bytes plus the position of the most significant // bit. Subtract this value by 1 to determine the length of the bitlist. - return 8*(u.l-1) + msb - 1 + return 8*(byteLen-1) + msb - 1 } func (u *BitList) MarshalJSON() ([]byte, error) { @@ -249,7 +261,7 @@ func (u *BitList) UnmarshalJSON(input []byte) error { return u.DecodeSSZ(hex, 0) } -func (u *BitList) Union(other *BitList) (*BitList, error) { +func (u *BitList) Merge(other *BitList) (*BitList, error) { if u.c != other.c { return nil, errors.New("bitlist union: different capacity") } @@ -263,8 +275,48 @@ func (u *BitList) Union(other *BitList) (*BitList, error) { unionFrom = other } // union + unionFrom.removeMsb() + ret.removeMsb() for i := 0; i < unionFrom.l; i++ { ret.u[i] |= unionFrom.u[i] } + unionFrom.addMsb() + byteLen := ret.addMsb() + ret.l = byteLen return ret, nil } + +// BitSlice maintains a slice of bits with underlying byte slice. +// This is just a auxiliary struct for merging BitList. +type BitSlice struct { + container []byte + length int +} + +func NewBitSlice() *BitSlice { + return &BitSlice{ + container: make([]byte, 0), + length: 0, + } +} + +// AppendBit appends one bit to the BitSlice. +func (b *BitSlice) AppendBit(bit bool) { + if b.length%8 == 0 { + b.container = append(b.container, 0) + } + if bit { + b.container[b.length/8] |= 1 << uint(b.length%8) + } + b.length++ +} + +// Bytes returns the underlying byte slice of the BitSlice. +func (b *BitSlice) Bytes() []byte { + return b.container +} + +// Length returns the length of the BitSlice. +func (b *BitSlice) Length() int { + return b.length +} diff --git a/cl/cltypes/solid/bitlist_test.go b/cl/cltypes/solid/bitlist_test.go index 82344b320f0..24f438dea01 100644 --- a/cl/cltypes/solid/bitlist_test.go +++ b/cl/cltypes/solid/bitlist_test.go @@ -125,19 +125,32 @@ func TestBitListCap(t *testing.T) { // Add more tests as needed for other functions in the BitList struct. -func TestBitlistUnion(t *testing.T) { +func TestBitlistMerge(t *testing.T) { require := require.New(t) - b1 := solid.NewBitList(5, 10) - b2 := solid.NewBitList(5, 10) + b1 := solid.BitlistFromBytes([]byte{0b11010000}, 10) + b2 := solid.BitlistFromBytes([]byte{0b00001101}, 10) - b1.Set(0, byte(0b11010000)) - b2.Set(0, byte(0b00001101)) - - merged, err := b1.Union(b2) + merged, err := b1.Merge(b2) require.NoError(err) - require.Equal(5, merged.Length(), "BitList Union did not return the expected length") - require.Equal(byte(0b11011101), merged.Get(0), "BitList Union did not return the expected value") - require.Equal(byte(0b00000000), merged.Get(1), "BitList Union did not return the expected value") + require.Equal(7, merged.Bits(), "BitList Merge did not return the expected number of bits") + require.Equal(1, merged.Length(), "BitList Union did not return the expected length") + require.Equal(byte(0b11010101), merged.Get(0), "BitList Union did not return the expected value") +} + +func TestBitSlice(t *testing.T) { + require := require.New(t) + + bs := solid.NewBitSlice() + + bs.AppendBit(true) + bs.AppendBit(false) + bs.AppendBit(true) + bs.AppendBit(false) + + bytes := bs.Bytes() + + require.Equal([]byte{0b00000101}, bytes, "BitSlice AppendBit did not append the bits correctly") + require.Equal(4, bs.Length(), "BitSlice AppendBit did not increment the length correctly") } diff --git a/cl/cltypes/solid/bitvector.go b/cl/cltypes/solid/bitvector.go index d30e180d456..7ea93e71b16 100644 --- a/cl/cltypes/solid/bitvector.go +++ b/cl/cltypes/solid/bitvector.go @@ -168,3 +168,13 @@ func (b *BitVector) Union(other *BitVector) (*BitVector, error) { } return new, nil } + +func (b *BitVector) IsOverlap(other *BitVector) bool { + // check by bytes + for i := 0; i < len(b.container) && i < len(other.container); i++ { + if b.container[i]&other.container[i] != 0 { + return true + } + } + return false +}