Skip to content

[WIP][server][protocol] Add accurate unique key count tracking for store versions#2676

Open
m-nagarajan wants to merge 6 commits intolinkedin:mainfrom
m-nagarajan:mnagaraj/addKeyCountMetric
Open

[WIP][server][protocol] Add accurate unique key count tracking for store versions#2676
m-nagarajan wants to merge 6 commits intolinkedin:mainfrom
m-nagarajan:mnagaraj/addKeyCountMetric

Conversation

@m-nagarajan
Copy link
Copy Markdown
Contributor

@m-nagarajan m-nagarajan commented Mar 31, 2026

Problem Statement

Venice currently lacks an accurate unique key count for store versions. The orthogonal PR (PR #2671) provides only approximate counts of all key activities, but not the actual key count present in the store.

Solution

Adds an exact unique key count that is maintained per-partition in PartitionConsumptionState via an AtomicLong field, persisted atomically with consumption offsets through OffsetRecord / PartitionState (schema v21), and exposed as an OTel ASYNC_GAUGE metric with VERSION_ROLE and REPLICA_TYPE dimensions.

End-to-End Flow

Phase 1: Batch Push (SOP → EOP) — all replicas

Fallback for PR #2642 (VPJ per-partition record count via EOP headers). Once #2642 ships, the batch baseline will come from the VPJ-computed count and this server-side counting becomes unnecessary.

VT record (PUT, non-chunk)
  → processKafkaDataMessage()
    → trackUniqueKeyCount()
      → if batch counting enabled && pre-EOP && not chunk fragment:
          PCS.incrementUniqueKeyCountForBatchRecord(keyBytes)
            - compares key with lastBatchKeyForDedup (unsigned byte order)
            - if key <= lastKey: skip (speculative execution duplicate)
            - else: count it, update lastKey
            - first call: CAS(-1 → 0) + incrementAndGet → 1
  → syncOffset() periodically:
      OffsetRecord.setUniqueKeyCount(PCS.getUniqueKeyCount())
      → serialized to PartitionState.avsc v21 (field: uniqueKeyCount, default -1)
  → processEndOfPush():
      PCS.finalizeUniqueKeyCountForBatchPush()
        if still -1 (empty partition): set to 0
        clears lastBatchKeyForDedup

Phase 2a: Default RMD for Hybrid Batch — leader only (A/A hybrid stores)

VT record (PUT, VALUE case, no existing RMD)
  → putInStorageEngine()
    → if addRmdToBatchPushForHybridStores && !DaVinci && !chunkFragment && pre-EOP:
        storageEngine.putWithReplicationMetadata(key, value, defaultTs0Rmd)
        (manifest uses pre-computed RMD; non-chunked prepends per-record schemaId)

VT record (DELETE, VALUE case)
  → removeFromStorageEngine()
    → if addRmdToBatchPushForHybridStores && !DaVinci && pre-EOP:
        storageEngine.deleteWithReplicationMetadata(key, defaultTs0Rmd)

Phase 2b: RT Signal Computation — leader only (A/A hybrid stores, post-EOP)

RT record arrives
  → processActiveActiveMessage()
    → DCR conflict resolution produces mergeConflictResult
    → determine wasAlive BEFORE setTransientRecord overwrites cache:
        - rmd==null + addRmdToBatchPush=true → false (truly new key)
        - rmd==null + addRmdToBatchPush=false → resolve old value Lazy
        - rmd with ts=BATCH_RMD_SENTINEL_TIMESTAMP(0) → true (batch key, never RT-written)
        - rmd with ts>0 → resolve old value Lazy (transient cache or RocksDB)
    → store wasAlive in MergeConflictResultWrapper.oldValueAlive
  → processMessageAndMaybeProduceToKafka()
    → if uniqueKeyCountForHybridStoreEnabled && PCS.uniqueKeyCount >= 0:
        wasAlive = wrapper.wasOldValueAlive()
        isAlive  = (mergeResult.newValue != null)
        
        !wasAlive && isAlive  → PCS.incrementUniqueKeyCount(), vtHeaders = "kcs": +1
        wasAlive && !isAlive  → PCS.decrementUniqueKeyCount(), vtHeaders = "kcs": -1
        otherwise             → no signal (update to existing key)
    → producePutOrDeleteToKafka(..., vtHeaders)
        VeniceWriter attaches "kcs" header to VT record (manifest only for chunked)

Phase 2c: Follower Signal Application (post-EOP)

VT record with "kcs" header arrives on follower
  → processKafkaDataMessage()
    → trackUniqueKeyCount()
      → if hybridEnabled && A/A && post-EOP && PCS.uniqueKeyCount >= 0
          && not leader-produced && (PUT non-chunk or DELETE):
            read "kcs" header value[0]
            +1 → PCS.incrementUniqueKeyCount()
            -1 → PCS.decrementUniqueKeyCount()
            other → rate-limited warning log

Persistence & Crash Recovery

syncOffset() (periodic + EOP):
  PCS.uniqueKeyCount → OffsetRecord → PartitionState avro → storageMetadataService

Crash + restart:
  PartitionState deserialized → OffsetRecord.getUniqueKeyCount() → PCS constructor
  → resumes from checkpointed count (batch replays post-checkpoint records only)
  → lastBatchKeyForDedup resets to null (dedup partially degrades for speculative records after checkpoint)

OTel Metric Emission

ASYNC_GAUGE: ingestion.unique_key_count
  dimensions: {store_name, cluster_name, version_role, replica_type}
  callback per (VersionRole, ReplicaType):
    → find version for role (current/future/backup)
    → iterate PCS for that version's StoreIngestionTask
    → filter by leader/follower state
    → sum uniqueKeyCount values where >= 0
    → return sum, or -1 if no partition is actively tracked
  always registered; emits -1 when feature is off

Three independent config flags

  1. Batch key counting (server.unique.key.count.for.all.batch.push.enabled, default false): Fallback for PR [common][vpj][da-vinci] Add record count verification for batch push and HLL-based repush validation #2642 (VPJ per-partition record count via EOP headers). All replicas count logical PUTs during batch push (SOP→EOP) with key-order dedup for speculative execution. Chunk fragments filtered. Mid-batch checkpoints capture partial counts. Becomes unnecessary once [common][vpj][da-vinci] Add record count verification for batch push and HLL-based repush validation #2642 ships.

  2. Default ts=0 RMD for hybrid batch (server.add.rmd.to.batch.push.for.hybrid.stores, default false): During batch push for hybrid A/A stores, writes a default timestamp-zero RMD alongside every batch PUT/DELETE. Ensures every batch key has RMD so DCR avoids the putWithoutRmd() fallback — enabling zero-additional-I/O for field-level/UPDATE stores. TTL filter treats ts=0 as sentinel (BATCH_RMD_SENTINEL_TIMESTAMP).

  3. Hybrid RT signal (server.unique.key.count.for.hybrid.store.enabled, default false): Leader computes +1/-1 signal from before/after key existence during DCR. wasAlive determined from RMD state (ts=0 → batch key alive, rmd null + 2a → new key, ts>0 → resolve old value). Signal propagated to followers via "kcs" VT header.

Key design decisions

  • AtomicLong with -1 sentinel: -1 = not tracked, >= 0 = active count. The >= 0 guard is the single source of truth for "is the count usable?"
  • wasOldValueAlive captured eagerly in MergeConflictResultWrapper BEFORE setTransientRecord() overwrites the transient cache — avoids stale Lazy resolution
  • RMD-based wasAlive optimization: ts=0 → alive (batch key, no value read), rmd null + 2a → dead (no read), ts>0 → resolve old value (transient cache hit or RocksDB)
  • Speculative execution dedup via ArrayUtils.compareUnsigned key-order comparison matching VPJ/RocksDB sort order
  • BATCH_RMD_SENTINEL_TIMESTAMP = 0L constant in RmdConstants used in producer (AASTI), consumer (TTL filter), and signal computation
  • Pre-computed PubSubMessageHeader instances (KEY_CREATED_SIGNAL / KEY_DELETED_SIGNAL) avoid per-record byte array allocation
  • VeniceWriter.delete() and putLargeValue() gain header-passing overloads; manifest-only headers for chunked PUTs
  • OTel metric uses EnumMap<ReplicaType, AsyncMetricEntityStateOneEnum<VersionRole>> workaround (pending AsyncMetricEntityStateTwoEnums in PR [da-vinci][server] Add OTel metrics to AggVersionedStorageEngineStats #2673)

Code changes

  • Added new code behind a config:
    • server.add.rmd.to.batch.push.for.hybrid.stores (default false)
    • server.unique.key.count.for.all.batch.push.enabled (default false)
    • server.unique.key.count.for.hybrid.store.enabled (default false)
  • Introduced new log lines.
    • Confirmed if logs need to be rate limited to avoid excessive logging. (Rate-limited via RedundantExceptionFilter for unexpected signal values)

Concurrency-Specific Checks

Both reviewer and PR author to verify

  • Code has no race conditions or thread safety issues. (AtomicLong for unique key count; single-threaded drainer for batch counting; key-level lock serializes same-key DCR; wasOldValueAlive captured before cache mutation)
  • Proper synchronization mechanisms (e.g., synchronized, RWLock) are used where needed.
  • No blocking calls inside critical sections that could lead to deadlocks or performance degradation.
  • Verified thread-safe collections are used (e.g., ConcurrentHashMap, CopyOnWriteArrayList). (VeniceConcurrentHashMap for OTel stats maps)
  • Validated proper exception handling in multi-threaded code to avoid silent thread termination.

How was this PR tested?

  • New unit tests added. (UniqueKeyCountTest — PCS operations, batch dedup, production code paths via doCallRealMethod/reflection for putInStorageEngine, removeFromStorageEngine, trackUniqueKeyCount, processMessage, processEndOfPush; IngestionOtelStatsTest — ASYNC_GAUGE callback with leader/follower filtering)
  • New integration tests added. (UniqueKeyCountScenarioTest — config-driven lifecycle with OTel gauge validation; TestUniqueKeyCount — E2E with real Venice cluster: batch push, hybrid RT, incremental push, chunked store, version swap)
  • Modified or extended existing tests. (ActiveActiveStoreIngestionTaskTest, MergeConflictResultWrapperTest, VeniceWriterUnitTest, TestVeniceKafkaInputTTLFilter, IngestionOtelMetricEntityTest, ServerMetricEntityTest)
  • Verified backward compatibility (if applicable). (Schema v21 adds uniqueKeyCount with default -1; v20 data deserializes correctly; TTL filter ts=0 sentinel is backward-compatible)

161 unit tests + 6 E2E integration tests, all passing.

Does this PR introduce any user-facing or breaking changes?

  • No. You can skip the rest of this section.

All three features are behind disabled-by-default config flags. No behavior changes until explicitly enabled.

Copilot AI review requested due to automatic review settings March 31, 2026 01:20
# Conflicts:
#	clients/da-vinci-client/src/main/java/com/linkedin/davinci/config/VeniceServerConfig.java
Copy link
Copy Markdown

Copilot AI left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Pull request overview

Adds exact per-partition unique key count tracking for store versions (persisted with offsets and exported via OTel), plus optional batch RMD sentinel behavior for hybrid A/A batch pushes.

Changes:

  • Introduces uniqueKeyCount persistence via PartitionState schema v21 + OffsetRecord, and in-memory tracking via PartitionConsumptionState.
  • Propagates key-count deltas from leaders to followers via a VT "kcs" header, and adds VeniceWriter overloads to pass custom PubSub headers.
  • Exposes the count as an OTel ASYNC_GAUGE metric dimensioned by VERSION_ROLE and REPLICA_TYPE, and updates TTL filtering to treat ts=0 RMD as a non-filterable sentinel.

Reviewed changes

Copilot reviewed 21 out of 21 changed files in this pull request and generated 5 comments.

Show a summary per file
File Description
internal/venice-common/src/test/java/com/linkedin/venice/writer/VeniceWriterUnitTest.java Adds unit tests asserting custom "kcs" headers are emitted on put/delete records.
internal/venice-common/src/main/resources/avro/PartitionState/v21/PartitionState.avsc Adds schema v21 including persisted uniqueKeyCount field (default -1).
internal/venice-common/src/main/java/com/linkedin/venice/writer/VeniceWriter.java Adds header-passing overloads for delete/put-large manifest emission to propagate "kcs" signals.
internal/venice-common/src/main/java/com/linkedin/venice/serialization/avro/AvroProtocolDefinition.java Bumps PartitionState protocol version to 21.
internal/venice-common/src/main/java/com/linkedin/venice/offsets/OffsetRecord.java Initializes and exposes persisted uniqueKeyCount through OffsetRecord.
internal/venice-common/src/main/java/com/linkedin/venice/ConfigKeys.java Adds config keys for batch RMD sentinel + batch counting + hybrid RT key count tracking.
clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/input/kafka/ttl/TestVeniceKafkaInputTTLFilter.java Adds tests ensuring ts=0 sentinel RMD is not TTL-filtered.
clients/venice-push-job/src/main/java/com/linkedin/venice/common/VeniceRmdTTLFilter.java Implements ts=0 sentinel exemption for value-level RMD timestamp TTL filtering.
clients/da-vinci-client/src/test/java/com/linkedin/davinci/stats/ServerMetricEntityTest.java Updates expected server metric entity count for the new OTel gauge.
clients/da-vinci-client/src/test/java/com/linkedin/davinci/stats/ingestion/IngestionOtelStatsTest.java Adds tests validating unique key count gauge callback behavior and emission.
clients/da-vinci-client/src/test/java/com/linkedin/davinci/stats/ingestion/IngestionOtelMetricEntityTest.java Adds expectation for the new ingestion.unique_key_count metric entity.
clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/UniqueKeyCountTest.java Adds PCS/OffsetRecord/unit-level tests for unique key count semantics and header encoding.
clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/UniqueKeyCountMockTest.java Adds mock-level tests covering schema/message-type filtering and hybrid/batch guards.
clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/UniqueKeyCountIntegrationTest.java Adds lifecycle/persistence + OTel emission “integration-style” tests for unique key count.
clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTaskTest.java Updates method signatures for producing to VT to include optional headers.
clients/da-vinci-client/src/main/java/com/linkedin/davinci/stats/ingestion/IngestionOtelStats.java Registers new async gauge metric with VERSION_ROLE + REPLICA_TYPE dimensions.
clients/da-vinci-client/src/main/java/com/linkedin/davinci/stats/ingestion/IngestionOtelMetricEntity.java Defines the new UNIQUE_KEY_COUNT OTel metric entity.
clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java Persists uniqueKeyCount on syncOffset; adds batch counting + follower "kcs" application logic.
clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/PartitionConsumptionState.java Adds AtomicLong uniqueKeyCount with batch finalize + increment/decrement helpers and restore from checkpoint.
clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java Adds batch RMD sentinel writes for hybrid stores and leader-side "kcs" signal computation + header propagation.
clients/da-vinci-client/src/main/java/com/linkedin/davinci/config/VeniceServerConfig.java Wires new feature flags into server config.

💡 Add Copilot custom instructions for smarter, more guided reviews. Learn how to get started.

Comment on lines 112 to 113
emptyPartitionState.upstreamVersionTopicPubSubPosition = PubSubSymbolicPosition.EARLIEST.toWireFormatBuffer();
emptyPartitionState.lastConsumedVersionTopicPubSubPosition = PubSubSymbolicPosition.EARLIEST.toWireFormatBuffer();
Copy link

Copilot AI Mar 31, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

getEmptyPartitionState() sets lastConsumedVersionTopicPubSubPosition twice (lines 111 and 113) and also assigns upstreamVersionTopicPubSubPosition twice (lines 105 and 112). This duplication is redundant and makes it harder to reason about the intended defaults; remove the extra assignments so each field is initialized once.

Suggested change
emptyPartitionState.upstreamVersionTopicPubSubPosition = PubSubSymbolicPosition.EARLIEST.toWireFormatBuffer();
emptyPartitionState.lastConsumedVersionTopicPubSubPosition = PubSubSymbolicPosition.EARLIEST.toWireFormatBuffer();

Copilot uses AI. Check for mistakes.
Comment on lines +896 to +900
PubSubMessageHeaders headers =
(pubSubMessageHeaders != null) ? pubSubMessageHeaders : EmptyPubSubMessageHeaders.SINGLETON;
CompletableFuture<PubSubProduceResult> produceResultFuture;
synchronized (this.partitionLocks[partition]) {
produceResultFuture = sendMessage(
Copy link

Copilot AI Mar 31, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This new overload forwards the caller-provided PubSubMessageHeaders instance into sendMessage(), which can append additional headers (e.g., view partition header) by mutating the same object. If callers reuse a headers instance across records, this can leak headers between messages. Consider making a defensive copy of the provided headers (or explicitly documenting that the writer may mutate the provided instance).

Copilot uses AI. Check for mistakes.
Comment on lines +123 to +127
@Test
public void testChunkFragmentBatchPutReturnsValueType() {
// Chunk fragment PUT → also returns VALUE, but our code filters by schemaId != CHUNK
doReturn(false).when(ingestionTask).isDaVinciClient();
doReturn(false).when(pcs).isEndOfPushReceived();
Copy link

Copilot AI Mar 31, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

testChunkFragmentBatchPutReturnsValueType doesn't execute any production code path or contain assertions, so it will always pass and doesn't validate the intended behavior. Add an assertion (e.g., call getStorageOperationTypeForPut(...) and verify the returned type / expected exception) or remove the test.

Copilot uses AI. Check for mistakes.
Comment on lines +497 to +498
assert !key.contains("_") : "Config key should use dots, not underscores: " + key;
assert key.startsWith("server.") : "Config key should start with 'server.': " + key;
Copy link

Copilot AI Mar 31, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This test uses Java assert statements for validation. Since JVM assertions are typically disabled in test runs unless -ea is set, these checks may be skipped silently. Use TestNG assertions (Assert.assertTrue/False) so the test always enforces the expectation.

Suggested change
assert !key.contains("_") : "Config key should use dots, not underscores: " + key;
assert key.startsWith("server.") : "Config key should start with 'server.': " + key;
Assert.assertFalse(key.contains("_"), "Config key should use dots, not underscores: " + key);
Assert.assertTrue(key.startsWith("server."), "Config key should start with 'server.': " + key);

Copilot uses AI. Check for mistakes.
Comment on lines +899 to +911
synchronized (this.partitionLocks[partition]) {
produceResultFuture = sendMessage(
producerMetadata -> kafkaKey,
MessageType.DELETE,
delete,
false,
partition,
callback,
true,
leaderMetadataWrapper,
logicalTs,
headers);
}
Copy link

Copilot AI Mar 31, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

sendMessage(...) already synchronizes on partitionLocks[partition], so synchronizing again around the call here is redundant (the lock is re-entrant but the extra block adds noise and can mislead readers into thinking additional ordering is required). Consider removing the outer synchronized (this.partitionLocks[partition]) block and rely on sendMessage's internal per-partition locking.

Suggested change
synchronized (this.partitionLocks[partition]) {
produceResultFuture = sendMessage(
producerMetadata -> kafkaKey,
MessageType.DELETE,
delete,
false,
partition,
callback,
true,
leaderMetadataWrapper,
logicalTs,
headers);
}
produceResultFuture = sendMessage(
producerMetadata -> kafkaKey,
MessageType.DELETE,
delete,
false,
partition,
callback,
true,
leaderMetadataWrapper,
logicalTs,
headers);

Copilot uses AI. Check for mistakes.
@m-nagarajan m-nagarajan added the work-in-progress Author is still working on this PR label Mar 31, 2026
@m-nagarajan m-nagarajan changed the title [da-vinci][protocol] Add accurate unique key count tracking for store versions [WIP][da-vinci][protocol] Add accurate unique key count tracking for store versions Mar 31, 2026
@m-nagarajan m-nagarajan changed the title [WIP][da-vinci][protocol] Add accurate unique key count tracking for store versions [WIP][server][protocol] Add accurate unique key count tracking for store versions Mar 31, 2026
Copilot AI review requested due to automatic review settings March 31, 2026 06:42
Copy link
Copy Markdown

Copilot AI left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Pull request overview

Copilot reviewed 23 out of 23 changed files in this pull request and generated 4 comments.


💡 Add Copilot custom instructions for smarter, more guided reviews. Learn how to get started.

Comment on lines +748 to +753
// wasAlive: did a live value exist before this write? (old value != null)
// isAlive: does a live value exist after resolution? (merge result != null)
PubSubMessageHeaders vtHeaders = EmptyPubSubMessageHeaders.SINGLETON;
if (uniqueKeyCountForHybridStoreEnabled && partitionConsumptionState.getUniqueKeyCount() >= 0) {
boolean wasAlive = (mergeConflictResultWrapper.getOldValueByteBufferProvider().get() != null);
boolean isAlive = (mergeConflictResult.getNewValue() != null);
Copy link

Copilot AI Mar 31, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Unique key count signal computation forces evaluation of mergeConflictResultWrapper.getOldValueByteBufferProvider().get() to derive wasAlive. Since this provider is backed by a lazy value lookup, this can introduce an extra storage read on RT writes where conflict resolution could have been decided from RMD alone. Consider deriving wasAlive from already-available metadata (e.g., tombstone state in RMD) or carrying an explicit “old value existed” flag from the conflict resolution path, so the signal computation doesn’t defeat the laziness and add per-record I/O.

Copilot uses AI. Check for mistakes.
Comment on lines +2032 to +2044
synchronized (this.partitionLocks[partition]) {
return sendMessage(
manifestKeyProvider,
MessageType.PUT,
manifestPayload,
false,
partition,
callback,
true,
leaderMetadataWrapper,
logicalTs,
manifestHeaders);
}
Copy link

Copilot AI Mar 31, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Redundant synchronization: this method synchronizes on partitionLocks[partition] and then calls the private sendMessage(...) overload which synchronizes on partitionLocks[partition] again. Since the inner sendMessage already provides the ordering/DIV atomicity guarantees, the outer lock can be removed to reduce overhead and contention.

Suggested change
synchronized (this.partitionLocks[partition]) {
return sendMessage(
manifestKeyProvider,
MessageType.PUT,
manifestPayload,
false,
partition,
callback,
true,
leaderMetadataWrapper,
logicalTs,
manifestHeaders);
}
return sendMessage(
manifestKeyProvider,
MessageType.PUT,
manifestPayload,
false,
partition,
callback,
true,
leaderMetadataWrapper,
logicalTs,
manifestHeaders);

Copilot uses AI. Check for mistakes.
Comment on lines +222 to +229
Assert.assertTrue(
offsetRecord.getUniqueKeyCount() >= 0,
"Expected unique key count to be non-negative after batch push, but got: "
+ offsetRecord.getUniqueKeyCount());
Assert.assertTrue(
offsetRecord.getUniqueKeyCount() > 0,
"Expected unique key count to be positive after 100-record batch push, but got: "
+ offsetRecord.getUniqueKeyCount());
Copy link

Copilot AI Mar 31, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This test’s Javadoc says the unique key count should equal 100 for a single-partition push, but the assertions only check >= 0 / > 0. Since this feature is intended to be accurate, asserting the exact expected count (100) would catch drift/regressions (e.g., chunk-fragment miscounting or duplicate counting).

Suggested change
Assert.assertTrue(
offsetRecord.getUniqueKeyCount() >= 0,
"Expected unique key count to be non-negative after batch push, but got: "
+ offsetRecord.getUniqueKeyCount());
Assert.assertTrue(
offsetRecord.getUniqueKeyCount() > 0,
"Expected unique key count to be positive after 100-record batch push, but got: "
+ offsetRecord.getUniqueKeyCount());
long uniqueKeyCount = offsetRecord.getUniqueKeyCount();
Assert.assertEquals(
uniqueKeyCount,
100L,
"Expected unique key count to be exactly 100 after 100-record batch push, but got: " + uniqueKeyCount);

Copilot uses AI. Check for mistakes.
Comment on lines +287 to +299
// Validate the unique key count increased
TestUtils.waitForNonDeterministicAssertion(60, TimeUnit.SECONDS, true, () -> {
OffsetRecord offsetRecord = getOffsetRecord(topicName, 0);
Assert.assertTrue(
offsetRecord.getUniqueKeyCount() > batchCount[0],
"Expected unique key count (" + offsetRecord.getUniqueKeyCount() + ") to be greater than batch count ("
+ batchCount[0] + ") after inserting 10 new keys");
});

long[] countAfterInserts = new long[1];
TestUtils.waitForNonDeterministicAssertion(30, TimeUnit.SECONDS, true, () -> {
countAfterInserts[0] = getOffsetRecord(topicName, 0).getUniqueKeyCount();
Assert.assertTrue(countAfterInserts[0] > batchCount[0]);
Copy link

Copilot AI Mar 31, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

After inserting 10 new RT keys, the test only asserts the count is greater than the batch baseline. To validate the correctness of the +1/-1 signal pipeline (and not just “some change happened”), this should assert the exact expected delta (baseline + 10) once the RT keys are confirmed readable.

Suggested change
// Validate the unique key count increased
TestUtils.waitForNonDeterministicAssertion(60, TimeUnit.SECONDS, true, () -> {
OffsetRecord offsetRecord = getOffsetRecord(topicName, 0);
Assert.assertTrue(
offsetRecord.getUniqueKeyCount() > batchCount[0],
"Expected unique key count (" + offsetRecord.getUniqueKeyCount() + ") to be greater than batch count ("
+ batchCount[0] + ") after inserting 10 new keys");
});
long[] countAfterInserts = new long[1];
TestUtils.waitForNonDeterministicAssertion(30, TimeUnit.SECONDS, true, () -> {
countAfterInserts[0] = getOffsetRecord(topicName, 0).getUniqueKeyCount();
Assert.assertTrue(countAfterInserts[0] > batchCount[0]);
// Validate the unique key count increased by exactly 10 after inserting 10 new unique keys
TestUtils.waitForNonDeterministicAssertion(60, TimeUnit.SECONDS, true, () -> {
OffsetRecord offsetRecord = getOffsetRecord(topicName, 0);
long actualCount = offsetRecord.getUniqueKeyCount();
long expectedCount = batchCount[0] + 10;
Assert.assertEquals(
actualCount,
expectedCount,
"Expected unique key count to be batch count + 10 after inserting 10 new keys, but got: " + actualCount
+ " (batch count: " + batchCount[0] + ")");
});
long[] countAfterInserts = new long[1];
TestUtils.waitForNonDeterministicAssertion(30, TimeUnit.SECONDS, true, () -> {
countAfterInserts[0] = getOffsetRecord(topicName, 0).getUniqueKeyCount();
long expectedCount = batchCount[0] + 10;
Assert.assertEquals(
countAfterInserts[0],
expectedCount,
"Expected post-insert unique key count snapshot to equal batch count + 10 before deletes");

Copilot uses AI. Check for mistakes.
Copilot AI review requested due to automatic review settings March 31, 2026 08:12
Copy link
Copy Markdown

Copilot AI left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Pull request overview

Copilot reviewed 22 out of 22 changed files in this pull request and generated no new comments.


💡 Add Copilot custom instructions for smarter, more guided reviews. Learn how to get started.

Copy link
Copy Markdown

Copilot AI left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Pull request overview

Copilot reviewed 25 out of 25 changed files in this pull request and generated 1 comment.


💡 Add Copilot custom instructions for smarter, more guided reviews. Learn how to get started.

Comment on lines +110 to +111
// Chunk fragments: schemaId == CHUNK -> filtered out by processKafkaDataMessage, NOT counted
// Manifest: schemaId == CHUNKED_VALUE_MANIFEST -> passes filter, counted
Copy link

Copilot AI Apr 3, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

doBatchChunked(..., int chunksPerKey) never uses chunksPerKey, and the implementation doesn't actually model chunk fragments vs manifest (it just increments once per logical key). This is misleading given the method name/Javadoc. Either remove the unused parameter and simplify the Javadoc, or loop over simulated chunk fragments (without calling the increment) to make the intent explicit.

Suggested change
// Chunk fragments: schemaId == CHUNK -> filtered out by processKafkaDataMessage, NOT counted
// Manifest: schemaId == CHUNKED_VALUE_MANIFEST -> passes filter, counted
for (int chunk = 0; chunk < chunksPerKey; chunk++) {
// Chunk fragments: schemaId == CHUNK -> filtered out by processKafkaDataMessage, NOT counted.
// Intentionally no incrementUniqueKeyCountForBatchRecord call here.
}
// Manifest: schemaId == CHUNKED_VALUE_MANIFEST -> passes filter, counted once per logical key.

Copilot uses AI. Check for mistakes.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

work-in-progress Author is still working on this PR

Projects

None yet

Development

Successfully merging this pull request may close these issues.

2 participants