Compare commits

...

3 Commits

Author SHA1 Message Date
satushh
9692993b6a remove redundant justified checkpoint update in pullTips 2025-12-19 23:07:40 +05:30
Manu NALEPA
2ac30f5ce6 Pending aggregates: When multiple aggregated attestations only differing by the aggregator index are in the pending queue, only process one of them. (#16153)
**What type of PR is this?**
Other

**What does this PR do? Why is it needed?**
When an (potentially aggregated) attestation is received **before** the
block being voted for, Prysm queues this attestation, then processes the
queue when the block has been received.

This behavior is consistent with the [Phase0 specification
](https://github.com/ethereum/consensus-specs/blob/master/specs/phase0/p2p-interface.md#beacon_attestation_subnet_id).

> [IGNORE] The block being voted for
(attestation.data.beacon_block_root) has been seen (via gossip or
non-gossip sources) (a client MAY queue attestations for processing once
block is retrieved).

Once the block being voted for is processed, previously queued
(potentially aggregated) attestations are then processed, and
broadcasted.

Processing (potentially aggregated) attestations takes some non
negligible time. For this reason, (potentially aggregated) attestations
are deduplicated before being introduced into the pending queue, to
avoid eventually processing duplicates.

Before this PR, two aggregated attestations were considered duplicated
if all of the following conditions were gathered:
1. Attestations have the same version, 
2. **Attestations have the same aggregator index (aka., the same
validator aggregated them)**,
3. Attestations have the same slot, 
4. Attestations have the same committee index, and
5. Attestations have the same aggregation bits

Aggregated attestations are then broadcasted.
The final purpose of aggregated attestations is to be packed into the
next block by the next proposer.
When packing attestations, the aggregator index is not used any more.

This pull request modifies the deduplication function used in the
pending aggregated attestations queue by considering that multiple
aggregated attestations only differing by the aggregator index are
equivalent (removing `2.` of the previous list.)

As a consequence, the count of aggregated attestations to be introduced
in the pending queue is reduced from 1 aggregated attestation by
aggregator to, in the best case,
[MAX_COMMITTEE_PER_SLOT=64](https://github.com/ethereum/consensus-specs/blob/master/specs/phase0/beacon-chain.md#misc-1).

Also, only a single aggregated attestation for a given version, slot,
committee index and aggregation bits will be re-broadcasted. This is a
correct behavior, since no data to be included in a block will be lost.
(We can even say that this will reduce by a bit the total networking
volume.)

**How to test**:
1. Start a beacon node (preferably, on a slow computer) from a
checkpoint.
2. Filter logs containing `Synced new block` and `Verified and saved
pending attestations to pool`. (You can pipe logs into `grep -E "Synced
new block|Verified and saved pending attestations to pool"`.

- In `Synced new block` logs, monitor the `sinceSlotStartTime` value.
This should monotonically decrease.
- In `Verified and saved pending attestations to pool`, monitor the
`pendingAggregateAttAndProofCount` value. It should be a "honest" value.
"honest" is not really quantifiable here, since it depends on the
aggregators. But it's likely to be less than
`5*MAX_COMMITTEE_PER_SLOT=320`.

**Which issues(s) does this PR fix?**

Partially fixes:
- https://github.com/OffchainLabs/prysm/issues/16160

**Other notes for review**
Please read commit by commit, with commit messages.
The important commit is b748c04a67.

**Acknowledgements**
- [x] I have read
[CONTRIBUTING.md](https://github.com/prysmaticlabs/prysm/blob/develop/CONTRIBUTING.md).
- [x] I have included a uniquely named [changelog fragment
file](https://github.com/prysmaticlabs/prysm/blob/develop/CONTRIBUTING.md#maintaining-changelogmd).
- [x] I have added a description with sufficient context for reviewers
to understand this PR.
- [x] I have tested that my changes work as expected and I added a
testing plan to the PR description (if applicable).
2025-12-19 14:05:50 +00:00
Manu NALEPA
7418c00ad6 validateDataColumn: Remove error logs. (#16157)
**What type of PR is this?**
Other

**What does this PR do? Why is it needed?**
When we receive data column sidecars via gossip, if the sidecar does not
respect the validation rules, a scary ERROR log is displayed. We can't
to anything about it, since the error comes from an invalid incoming
sidecar, so there is no need to print an ERROR message.

Node: As all REJECTED gossip message, a DEBUG log is also always
displayed.

Example of ERROR log:
```
[2025-12-18 15:38:26.46] ERROR sync: Failed to decode message error=invalid ssz encoding. first variable element offset indexes into fixed value data
[2025-12-18 15:38:26.46] DEBUG sync: Gossip message was rejected agent=erigon/caplin error=invalid ssz encoding. first variable element offset indexes into fixed value data gossipScore=0 multiaddress=/ip4/141.147.32.105/tcp/9000 peerID=16Uiu2HAmHu88k97iBist1vJg7cPNuTjJFRARKvDF7yaH3Pv3Vmso topic=/eth2/c6ecb76c/data_column_sidecar_30/ssz_snappy
```

(After this PR, the DEBUG one will still be printed.)

**Acknowledgements**

- [x] I have read
[CONTRIBUTING.md](https://github.com/prysmaticlabs/prysm/blob/develop/CONTRIBUTING.md).
- [x] I have included a uniquely named [changelog fragment
file](https://github.com/prysmaticlabs/prysm/blob/develop/CONTRIBUTING.md#maintaining-changelogmd).
- [x] I have added a description with sufficient context for reviewers
to understand this PR.
- [x] I have tested that my changes work as expected and I added a
testing plan to the PR description (if applicable).
2025-12-18 16:18:02 +00:00
10 changed files with 175 additions and 52 deletions

View File

@@ -88,9 +88,6 @@ func (s *Store) pullTips(state state.BeaconState, node *Node, jc, fc *ethpb.Chec
}
}
if uf.Epoch > s.unrealizedFinalizedCheckpoint.Epoch {
s.unrealizedJustifiedCheckpoint = &forkchoicetypes.Checkpoint{
Epoch: uj.Epoch, Root: bytesutil.ToBytes32(uj.Root),
}
s.unrealizedFinalizedCheckpoint = &forkchoicetypes.Checkpoint{
Epoch: uf.Epoch, Root: bytesutil.ToBytes32(uf.Root),
}

View File

@@ -3,6 +3,7 @@ package sync
import (
"bytes"
"context"
"fmt"
"maps"
"slices"
"sync"
@@ -243,8 +244,10 @@ func requestDirectSidecarsFromPeers(
}
// Compute missing indices by root, excluding those already in storage.
var lastRoot [fieldparams.RootLength]byte
missingIndicesByRoot := make(map[[fieldparams.RootLength]byte]map[uint64]bool, len(incompleteRoots))
for root := range incompleteRoots {
lastRoot = root
storedIndices := storedIndicesByRoot[root]
missingIndices := make(map[uint64]bool, len(requestedIndices))
@@ -259,6 +262,7 @@ func requestDirectSidecarsFromPeers(
}
}
initialMissingRootCount := len(missingIndicesByRoot)
initialMissingCount := computeTotalCount(missingIndicesByRoot)
indicesByRootByPeer, err := computeIndicesByRootByPeer(params.P2P, slotByRoot, missingIndicesByRoot, connectedPeers)
@@ -301,11 +305,19 @@ func requestDirectSidecarsFromPeers(
}
}
log.WithFields(logrus.Fields{
"duration": time.Since(start),
"initialMissingCount": initialMissingCount,
"finalMissingCount": computeTotalCount(missingIndicesByRoot),
}).Debug("Requested direct data column sidecars from peers")
log := log.WithFields(logrus.Fields{
"duration": time.Since(start),
"initialMissingRootCount": initialMissingRootCount,
"initialMissingCount": initialMissingCount,
"finalMissingRootCount": len(missingIndicesByRoot),
"finalMissingCount": computeTotalCount(missingIndicesByRoot),
})
if initialMissingRootCount == 1 {
log = log.WithField("root", fmt.Sprintf("%#x", lastRoot))
}
log.Debug("Requested direct data column sidecars from peers")
return verifiedColumnsByRoot, nil
}

View File

@@ -3,9 +3,9 @@ package sync
import (
"bytes"
"context"
"encoding/hex"
"fmt"
"slices"
"time"
"github.com/OffchainLabs/prysm/v7/beacon-chain/blockchain"
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/blocks"
@@ -21,13 +21,23 @@ import (
"github.com/OffchainLabs/prysm/v7/monitoring/tracing/trace"
ethpb "github.com/OffchainLabs/prysm/v7/proto/prysm/v1alpha1"
"github.com/OffchainLabs/prysm/v7/runtime/version"
"github.com/OffchainLabs/prysm/v7/time"
"github.com/OffchainLabs/prysm/v7/time/slots"
pubsub "github.com/libp2p/go-libp2p-pubsub"
"github.com/pkg/errors"
"github.com/sirupsen/logrus"
)
var pendingAttsLimit = 32768
const pendingAttsLimit = 32768
// aggregatorIndexFilter defines how aggregator index should be handled in equality checks.
type aggregatorIndexFilter int
const (
// ignoreAggregatorIndex means aggregates differing only by aggregator index are considered equal.
ignoreAggregatorIndex aggregatorIndexFilter = iota
// includeAggregatorIndex means aggregator index must also match for aggregates to be considered equal.
includeAggregatorIndex
)
// This method processes pending attestations as a "known" block as arrived. With validations,
// the valid attestations get saved into the operation mem pool, and the invalid attestations gets deleted
@@ -50,16 +60,7 @@ func (s *Service) processPendingAttsForBlock(ctx context.Context, bRoot [32]byte
attestations := s.blkRootToPendingAtts[bRoot]
s.pendingAttsLock.RUnlock()
if len(attestations) > 0 {
start := time.Now()
s.processAttestations(ctx, attestations)
duration := time.Since(start)
log.WithFields(logrus.Fields{
"blockRoot": hex.EncodeToString(bytesutil.Trunc(bRoot[:])),
"pendingAttsCount": len(attestations),
"duration": duration,
}).Debug("Verified and saved pending attestations to pool")
}
s.processAttestations(ctx, attestations)
randGen := rand.NewGenerator()
// Delete the missing block root key from pending attestation queue so a node will not request for the block again.
@@ -79,26 +80,71 @@ func (s *Service) processPendingAttsForBlock(ctx context.Context, bRoot [32]byte
return s.sendBatchRootRequest(ctx, pendingRoots, randGen)
}
// processAttestations processes a list of attestations.
// It assumes (for logging purposes only) that all attestations pertain to the same block.
func (s *Service) processAttestations(ctx context.Context, attestations []any) {
if len(attestations) == 0 {
return
}
firstAttestation := attestations[0]
var blockRoot []byte
switch v := firstAttestation.(type) {
case ethpb.Att:
blockRoot = v.GetData().BeaconBlockRoot
case ethpb.SignedAggregateAttAndProof:
blockRoot = v.AggregateAttestationAndProof().AggregateVal().GetData().BeaconBlockRoot
default:
log.Warnf("Unexpected attestation type %T, skipping processing", v)
return
}
validAggregates := make([]ethpb.SignedAggregateAttAndProof, 0, len(attestations))
startAggregate := time.Now()
atts := make([]ethpb.Att, 0, len(attestations))
aggregateAttAndProofCount := 0
for _, att := range attestations {
switch v := att.(type) {
case ethpb.Att:
atts = append(atts, v)
case ethpb.SignedAggregateAttAndProof:
s.processAggregate(ctx, v)
aggregateAttAndProofCount++
// Avoid processing multiple aggregates only differing by aggregator index.
if slices.ContainsFunc(validAggregates, func(other ethpb.SignedAggregateAttAndProof) bool {
return pendingAggregatesAreEqual(v, other, ignoreAggregatorIndex)
}) {
continue
}
if err := s.processAggregate(ctx, v); err != nil {
log.WithError(err).Debug("Pending aggregate attestation could not be processed")
continue
}
validAggregates = append(validAggregates, v)
default:
log.Warnf("Unexpected attestation type %T, skipping", v)
}
}
durationAggregateAttAndProof := time.Since(startAggregate)
startAtts := time.Now()
for _, bucket := range bucketAttestationsByData(atts) {
s.processAttestationBucket(ctx, bucket)
}
durationAtts := time.Since(startAtts)
log.WithFields(logrus.Fields{
"blockRoot": fmt.Sprintf("%#x", blockRoot),
"totalCount": len(attestations),
"aggregateAttAndProofCount": aggregateAttAndProofCount,
"uniqueAggregateAttAndProofCount": len(validAggregates),
"attCount": len(atts),
"durationTotal": durationAggregateAttAndProof + durationAtts,
"durationAggregateAttAndProof": durationAggregateAttAndProof,
"durationAtts": durationAtts,
}).Debug("Verified and saved pending attestations to pool")
}
// attestationBucket groups attestations with the same AttestationData for batch processing.
@@ -303,21 +349,20 @@ func (s *Service) processVerifiedAttestation(
})
}
func (s *Service) processAggregate(ctx context.Context, aggregate ethpb.SignedAggregateAttAndProof) {
func (s *Service) processAggregate(ctx context.Context, aggregate ethpb.SignedAggregateAttAndProof) error {
res, err := s.validateAggregatedAtt(ctx, aggregate)
if err != nil {
log.WithError(err).Debug("Pending aggregated attestation failed validation")
return
return errors.Wrap(err, "validate aggregated att")
}
if res != pubsub.ValidationAccept || !s.validateBlockInAttestation(ctx, aggregate) {
log.Debug("Pending aggregated attestation failed validation")
return
return errors.New("Pending aggregated attestation failed validation")
}
att := aggregate.AggregateAttestationAndProof().AggregateVal()
if err := s.saveAttestation(att); err != nil {
log.WithError(err).Debug("Could not save aggregated attestation")
return
return errors.Wrap(err, "save attestation")
}
_ = s.setAggregatorIndexEpochSeen(att.GetData().Target.Epoch, aggregate.AggregateAttestationAndProof().GetAggregatorIndex())
@@ -325,6 +370,8 @@ func (s *Service) processAggregate(ctx context.Context, aggregate ethpb.SignedAg
if err := s.cfg.p2p.Broadcast(ctx, aggregate); err != nil {
log.WithError(err).Debug("Could not broadcast aggregated attestation")
}
return nil
}
// This defines how pending aggregates are saved in the map. The key is the
@@ -336,7 +383,7 @@ func (s *Service) savePendingAggregate(agg ethpb.SignedAggregateAttAndProof) {
s.savePending(root, agg, func(other any) bool {
a, ok := other.(ethpb.SignedAggregateAttAndProof)
return ok && pendingAggregatesAreEqual(agg, a)
return ok && pendingAggregatesAreEqual(agg, a, includeAggregatorIndex)
})
}
@@ -391,13 +438,19 @@ func (s *Service) savePending(root [32]byte, pending any, isEqual func(other any
s.blkRootToPendingAtts[root] = append(s.blkRootToPendingAtts[root], pending)
}
func pendingAggregatesAreEqual(a, b ethpb.SignedAggregateAttAndProof) bool {
// pendingAggregatesAreEqual checks if two pending aggregate attestations are equal.
// The filter parameter controls whether aggregator index is considered in the equality check.
func pendingAggregatesAreEqual(a, b ethpb.SignedAggregateAttAndProof, filter aggregatorIndexFilter) bool {
if a.Version() != b.Version() {
return false
}
if a.AggregateAttestationAndProof().GetAggregatorIndex() != b.AggregateAttestationAndProof().GetAggregatorIndex() {
return false
if filter == includeAggregatorIndex {
if a.AggregateAttestationAndProof().GetAggregatorIndex() != b.AggregateAttestationAndProof().GetAggregatorIndex() {
return false
}
}
aAtt := a.AggregateAttestationAndProof().AggregateVal()
bAtt := b.AggregateAttestationAndProof().AggregateVal()
if aAtt.GetData().Slot != bAtt.GetData().Slot {

View File

@@ -94,7 +94,7 @@ func TestProcessPendingAtts_NoBlockRequestBlock(t *testing.T) {
// Process block A (which exists and has no pending attestations)
// This should skip processing attestations for A and request blocks B and C
require.NoError(t, r.processPendingAttsForBlock(t.Context(), rootA))
require.LogsContain(t, hook, "Requesting block by root")
require.LogsContain(t, hook, "Requesting blocks by root")
}
func TestProcessPendingAtts_HasBlockSaveUnaggregatedAtt(t *testing.T) {
@@ -911,17 +911,17 @@ func Test_pendingAggregatesAreEqual(t *testing.T) {
},
AggregationBits: bitfield.Bitlist{0b1111},
}}}
assert.Equal(t, true, pendingAggregatesAreEqual(a, b))
assert.Equal(t, true, pendingAggregatesAreEqual(a, b, includeAggregatorIndex))
})
t.Run("different version", func(t *testing.T) {
a := &ethpb.SignedAggregateAttestationAndProof{Message: &ethpb.AggregateAttestationAndProof{AggregatorIndex: 1}}
b := &ethpb.SignedAggregateAttestationAndProofElectra{Message: &ethpb.AggregateAttestationAndProofElectra{AggregatorIndex: 1}}
assert.Equal(t, false, pendingAggregatesAreEqual(a, b))
assert.Equal(t, false, pendingAggregatesAreEqual(a, b, includeAggregatorIndex))
})
t.Run("different aggregator index", func(t *testing.T) {
a := &ethpb.SignedAggregateAttestationAndProof{Message: &ethpb.AggregateAttestationAndProof{AggregatorIndex: 1}}
b := &ethpb.SignedAggregateAttestationAndProof{Message: &ethpb.AggregateAttestationAndProof{AggregatorIndex: 2}}
assert.Equal(t, false, pendingAggregatesAreEqual(a, b))
assert.Equal(t, false, pendingAggregatesAreEqual(a, b, includeAggregatorIndex))
})
t.Run("different slot", func(t *testing.T) {
a := &ethpb.SignedAggregateAttestationAndProof{
@@ -942,7 +942,7 @@ func Test_pendingAggregatesAreEqual(t *testing.T) {
},
AggregationBits: bitfield.Bitlist{0b1111},
}}}
assert.Equal(t, false, pendingAggregatesAreEqual(a, b))
assert.Equal(t, false, pendingAggregatesAreEqual(a, b, includeAggregatorIndex))
})
t.Run("different committee index", func(t *testing.T) {
a := &ethpb.SignedAggregateAttestationAndProof{
@@ -963,7 +963,7 @@ func Test_pendingAggregatesAreEqual(t *testing.T) {
},
AggregationBits: bitfield.Bitlist{0b1111},
}}}
assert.Equal(t, false, pendingAggregatesAreEqual(a, b))
assert.Equal(t, false, pendingAggregatesAreEqual(a, b, includeAggregatorIndex))
})
t.Run("different aggregation bits", func(t *testing.T) {
a := &ethpb.SignedAggregateAttestationAndProof{
@@ -984,7 +984,30 @@ func Test_pendingAggregatesAreEqual(t *testing.T) {
},
AggregationBits: bitfield.Bitlist{0b1000},
}}}
assert.Equal(t, false, pendingAggregatesAreEqual(a, b))
assert.Equal(t, false, pendingAggregatesAreEqual(a, b, includeAggregatorIndex))
})
t.Run("different aggregator index should be equal while ignoring aggregator index", func(t *testing.T) {
a := &ethpb.SignedAggregateAttestationAndProof{
Message: &ethpb.AggregateAttestationAndProof{
AggregatorIndex: 1,
Aggregate: &ethpb.Attestation{
Data: &ethpb.AttestationData{
Slot: 1,
CommitteeIndex: 1,
},
AggregationBits: bitfield.Bitlist{0b1111},
}}}
b := &ethpb.SignedAggregateAttestationAndProof{
Message: &ethpb.AggregateAttestationAndProof{
AggregatorIndex: 2,
Aggregate: &ethpb.Attestation{
Data: &ethpb.AttestationData{
Slot: 1,
CommitteeIndex: 1,
},
AggregationBits: bitfield.Bitlist{0b1111},
}}}
assert.Equal(t, true, pendingAggregatesAreEqual(a, b, ignoreAggregatorIndex))
})
}

View File

@@ -2,7 +2,6 @@ package sync
import (
"context"
"encoding/hex"
"fmt"
"slices"
"sync"
@@ -44,11 +43,13 @@ func (s *Service) processPendingBlocksQueue() {
if !s.chainIsStarted() {
return
}
locker.Lock()
defer locker.Unlock()
if err := s.processPendingBlocks(s.ctx); err != nil {
log.WithError(err).Debug("Could not process pending blocks")
}
locker.Unlock()
})
}
@@ -73,8 +74,10 @@ func (s *Service) processPendingBlocks(ctx context.Context) error {
randGen := rand.NewGenerator()
var parentRoots [][32]byte
blkRoots := make([][32]byte, 0, len(sortedSlots)*maxBlocksPerSlot)
// Iterate through sorted slots.
for _, slot := range sortedSlots {
for i, slot := range sortedSlots {
// Skip processing if slot is in the future.
if slot > s.cfg.clock.CurrentSlot() {
continue
@@ -91,6 +94,9 @@ func (s *Service) processPendingBlocks(ctx context.Context) error {
// Process each block in the queue.
for _, b := range blocksInCache {
start := time.Now()
totalDuration := time.Duration(0)
if err := blocks.BeaconBlockIsNil(b); err != nil {
continue
}
@@ -147,19 +153,34 @@ func (s *Service) processPendingBlocks(ctx context.Context) error {
}
cancelFunction()
// Process pending attestations for this block.
if err := s.processPendingAttsForBlock(ctx, blkRoot); err != nil {
log.WithError(err).Debug("Failed to process pending attestations for block")
}
blkRoots = append(blkRoots, blkRoot)
// Remove the processed block from the queue.
if err := s.removeBlockFromQueue(b, blkRoot); err != nil {
return err
}
log.WithFields(logrus.Fields{"slot": slot, "blockRoot": hex.EncodeToString(bytesutil.Trunc(blkRoot[:]))}).Debug("Processed pending block and cleared it in cache")
duration := time.Since(start)
totalDuration += duration
log.WithFields(logrus.Fields{
"slotIndex": fmt.Sprintf("%d/%d", i+1, len(sortedSlots)),
"slot": slot,
"root": fmt.Sprintf("%#x", blkRoot),
"duration": duration,
"totalDuration": totalDuration,
}).Debug("Processed pending block and cleared it in cache")
}
span.End()
}
for _, blkRoot := range blkRoots {
// Process pending attestations for this block.
if err := s.processPendingAttsForBlock(ctx, blkRoot); err != nil {
log.WithError(err).Debug("Failed to process pending attestations for block")
}
}
return s.sendBatchRootRequest(ctx, parentRoots, randGen)
}
@@ -379,6 +400,19 @@ func (s *Service) sendBatchRootRequest(ctx context.Context, roots [][32]byte, ra
req = roots[:maxReqBlock]
}
if logrus.GetLevel() >= logrus.DebugLevel {
rootsStr := make([]string, 0, len(roots))
for _, req := range roots {
rootsStr = append(rootsStr, fmt.Sprintf("%#x", req))
}
log.WithFields(logrus.Fields{
"peer": pid,
"count": len(req),
"roots": rootsStr,
}).Debug("Requesting blocks by root")
}
// Send the request to the peer.
if err := s.sendBeaconBlocksRequest(ctx, &req, pid); err != nil {
tracing.AnnotateError(span, err)
@@ -438,8 +472,6 @@ func (s *Service) filterOutPendingAndSynced(roots [][fieldparams.RootLength]byte
roots = append(roots[:i], roots[i+1:]...)
continue
}
log.WithField("blockRoot", fmt.Sprintf("%#x", r)).Debug("Requesting block by root")
}
return roots
}

View File

@@ -51,14 +51,12 @@ func (s *Service) validateDataColumn(ctx context.Context, pid peer.ID, msg *pubs
// Decode the message, reject if it fails.
m, err := s.decodePubsubMessage(msg)
if err != nil {
log.WithError(err).Error("Failed to decode message")
return pubsub.ValidationReject, err
}
// Reject messages that are not of the expected type.
dcsc, ok := m.(*eth.DataColumnSidecar)
if !ok {
log.WithField("message", m).Error("Message is not of type *eth.DataColumnSidecar")
return pubsub.ValidationReject, errWrongMessage
}

View File

@@ -361,7 +361,7 @@ func (dv *RODataColumnsVerifier) SidecarParentSeen(parentSeen func([fieldparams.
}
if !dv.fc.HasNode(parentRoot) {
return columnErrBuilder(errSidecarParentNotSeen)
return columnErrBuilder(errors.Wrapf(errSidecarParentNotSeen, "parent root: %#x", parentRoot))
}
}

3
changelog/manu-agg.md Normal file
View File

@@ -0,0 +1,3 @@
### Changed
- Pending aggregates: When multiple aggregated attestations only differing by the aggregator index are in the pending queue, only process one of them.

View File

@@ -0,0 +1,2 @@
### Changed
- `validateDataColumn`: Remove error logs.

View File

@@ -0,0 +1,3 @@
### Fixed
- Removed redundant justified checkpoint update in pullTips.