[common][vpj] Tag EOP with per-partition record counts for batch push verification#2663
[common][vpj] Tag EOP with per-partition record counts for batch push verification#2663sushantmane wants to merge 4 commits intolinkedin:mainfrom
Conversation
verification Track per-partition record counts during batch push and embed them as PubSub message headers on the End-of-Push control message. This enables server-side verification (in a follow-up PR) without requiring changes to the EOP Avro schema. Changes: - PubSubMessageHeaders: new "prc" header constant - VeniceWriter: new broadcastEndOfPush/sendControlMessage overloads with headers - DataWriterTaskTracker: per-partition tracking interface - MR path: counter group in MRJobCounterHelper + tracker implementations - Spark path: new MapLongAccumulator + tracker implementations - VenicePushJob: collect per-partition counts, pass to broadcastEndOfPush
There was a problem hiding this comment.
Pull request overview
Adds per-partition record-count tracking in VPJ and propagates those counts on the End-of-Push (EOP) control message via a new PubSub header (prc), enabling downstream batch-push verification without changing the EOP Avro schema.
Changes:
- Introduces
VENICE_PARTITION_RECORD_COUNT_HEADER(prc) and wiresVeniceWriter.broadcastEndOfPush(...)to optionally attach per-partition counts as PubSub headers. - Implements per-partition record counting in both MR (counters) and Spark (custom
MapLongAccumulator) viaDataWriterTaskTrackeradditions. - Updates
VenicePushJobto collect per-partition counts from the task tracker and pass them tobroadcastEndOfPush, with unit tests for writer, MR counters, and Spark accumulator.
Reviewed changes
Copilot reviewed 14 out of 14 changed files in this pull request and generated 3 comments.
Show a summary per file
| File | Description |
|---|---|
| internal/venice-common/src/main/java/com/linkedin/venice/pubsub/api/PubSubMessageHeaders.java | Adds the prc header constant for per-partition record counts. |
| internal/venice-common/src/main/java/com/linkedin/venice/writer/VeniceWriter.java | Adds EOP/control-message overloads to emit per-partition record counts in PubSub headers. |
| internal/venice-common/src/test/java/com/linkedin/venice/writer/VeniceWriterUnitTest.java | Tests broadcastEndOfPush with/without per-partition record-count headers. |
| clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/task/datawriter/DataWriterTaskTracker.java | Extends the tracker interface with per-partition tracking + retrieval methods. |
| clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/task/datawriter/AbstractPartitionWriter.java | Calls the new per-partition tracking hook when sending a record. |
| clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/mapreduce/counter/MRJobCounterHelper.java | Adds MR counter group + helpers for per-partition record counts. |
| clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/mapreduce/datawriter/task/ReporterBackedMapReduceDataWriterTaskTracker.java | Tracks per-partition counts via Reporter-backed MR counters. |
| clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/mapreduce/datawriter/task/CounterBackedMapReduceDataWriterTaskTracker.java | Exposes per-partition counts from MR Counters after job completion. |
| clients/venice-push-job/src/main/java/com/linkedin/venice/spark/datawriter/task/MapLongAccumulator.java | Adds a Spark accumulator for per-key (partition) long aggregation. |
| clients/venice-push-job/src/main/java/com/linkedin/venice/spark/datawriter/task/DataWriterAccumulators.java | Registers the new per-partition accumulator with Spark. |
| clients/venice-push-job/src/main/java/com/linkedin/venice/spark/datawriter/task/SparkDataWriterTaskTracker.java | Tracks and returns per-partition record counts via the Spark accumulator. |
| clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/VenicePushJob.java | Collects per-partition record counts from the tracker and passes them into EOP broadcast. |
| clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/mapreduce/counter/MRJobCounterHelperTest.java | Adds tests for per-partition counter parsing (and related behaviors). |
| clients/venice-push-job/src/test/java/com/linkedin/venice/spark/datawriter/task/MapLongAccumulatorTest.java | Adds unit tests for accumulator add/merge/reset/copy/immutability. |
💡 Add Copilot custom instructions for smarter, more guided reviews. Learn how to get started.
internal/venice-common/src/main/java/com/linkedin/venice/writer/VeniceWriter.java
Show resolved
Hide resolved
...-push-job/src/main/java/com/linkedin/venice/hadoop/mapreduce/counter/MRJobCounterHelper.java
Show resolved
Hide resolved
...h-job/src/test/java/com/linkedin/venice/hadoop/mapreduce/counter/MRJobCounterHelperTest.java
Show resolved
Hide resolved
There was a problem hiding this comment.
Pull request overview
Copilot reviewed 14 out of 14 changed files in this pull request and generated 3 comments.
💡 Add Copilot custom instructions for smarter, more guided reviews. Learn how to get started.
| public static Map<Integer, Long> getPerPartitionRecordCounts(Counters counters) { | ||
| Map<Integer, Long> result = new HashMap<>(); | ||
| for (Counters.Group group: counters) { | ||
| if (group.getName().equals(PER_PARTITION_RECORD_COUNT_GROUP)) { | ||
| for (Counters.Counter counter: group) { | ||
| result.put(Integer.parseInt(counter.getName()), counter.getValue()); | ||
| } |
There was a problem hiding this comment.
getPerPartitionRecordCounts will throw a NullPointerException when counters is null (enhanced for-loop over a null reference). Other getters in this class explicitly return 0 on null counters; this method should mirror that by returning an empty map when counters is null.
| messageSent++; | ||
| telemetry(); | ||
| dataWriterTaskTracker.trackRecordSentToPubSub(); | ||
| dataWriterTaskTracker.trackRecordSentToPubSubForPartition(getTaskId()); |
There was a problem hiding this comment.
trackRecordSentToPubSubForPartition(getTaskId()) is invoked for every produced record. On Spark this allocates a new scala.Tuple2 per record and performs a ConcurrentHashMap.merge, and on MR it updates a per-partition counter per record—both can add noticeable overhead at high throughput. Consider batching per-partition counts locally in the partition writer and reporting them periodically (or at close) instead of per message.
| dataWriterTaskTracker.trackRecordSentToPubSubForPartition(getTaskId()); | |
| // Only track partition-level PubSub records once per partition writer to avoid per-record overhead. | |
| if (messageSent == 1) { | |
| dataWriterTaskTracker.trackRecordSentToPubSubForPartition(getTaskId()); | |
| } |
| @Test | ||
| public void testIncrPartitionRecordCountWithMockReporter() { | ||
| // Use a Counters-based approach to verify the increment path indirectly: | ||
| // Increment via Reporter, then read back via Counters | ||
| Counters counters = new Counters(); | ||
| Counters.Counter counter = counters.findCounter("Per Partition Record Count", "3"); | ||
| counter.increment(10); | ||
| counter.increment(5); | ||
|
|
||
| Map<Integer, Long> result = MRJobCounterHelper.getPerPartitionRecordCounts(counters); | ||
| Assert.assertEquals(result.size(), 1); | ||
| Assert.assertEquals((long) result.get(3), 15L); | ||
| } |
There was a problem hiding this comment.
testIncrPartitionRecordCountWithMockReporter doesn’t call MRJobCounterHelper.incrPartitionRecordCount(...) or use a Reporter; it increments Counters directly. Either rename the test to reflect that it validates getPerPartitionRecordCounts parsing, or add a dedicated test that exercises the Reporter-based increment method.
…Map, rename test Code review findings: - VeniceWriter: make 5-param sendControlMessage delegate to 6-param overload, eliminating duplicated getDebugInfo/isEndOfSegment/synchronized logic - MapLongAccumulator: ConcurrentHashMap → HashMap — Spark tasks are single-threaded (each gets a fresh copy()), no concurrent access occurs - MRJobCounterHelperTest: rename misleading test to reflect what it actually tests (counter retrieval round-trip, not Reporter-based increment)
Summary
Track per-partition record counts during batch push and embed them as PubSub message headers on the End-of-Push control message. This enables server-side verification (in a follow-up PR) without requiring changes to the EOP Avro schema.
Part 1 of 4 in the batch push record count verification series.
Changes
PubSubMessageHeaders: newprcheader constantVeniceWriter: newbroadcastEndOfPush/sendControlMessageoverloads with per-partition headers (8-byte long per partition)DataWriterTaskTracker: per-partition tracking interface (trackRecordSentToPubSubForPartition,getPerPartitionRecordCounts)MRJobCounterHelper+ tracker implementationsMapLongAccumulator+ tracker implementationsVenicePushJob: collect per-partition counts, pass tobroadcastEndOfPushBackward compatibility
Old servers ignore unknown PubSub headers — no impact.
Test plan
VeniceWriterUnitTest— broadcastEndOfPush with/without countsMRJobCounterHelperTest— per-partition counter round-tripMapLongAccumulatorTest— add, merge, reset, copy