Skip to content

Commit cd915b9

Browse files
authored
[vpj] Replace org.apache.kafka.common.TopicPartition with PubSubTopicPartition in VPJ (#1460)
Replace the usage of Kafka TopicPartition with Venice PubSubOffset instances to enable support for non-Kafka pub-sub clients. This change is part of the broader effort to decouple Venice from Kafka-specific classes and ensure compatibility with other pub-sub systems.
1 parent 3392d62 commit cd915b9

File tree

9 files changed

+86
-63
lines changed

9 files changed

+86
-63
lines changed

clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -159,7 +159,6 @@
159159
import org.apache.avro.AvroRuntimeException;
160160
import org.apache.avro.Schema;
161161
import org.apache.helix.manager.zk.ZKHelixAdmin;
162-
import org.apache.kafka.clients.CommonClientConfigs;
163162
import org.apache.logging.log4j.LogManager;
164163
import org.apache.logging.log4j.Logger;
165164

@@ -4173,7 +4172,7 @@ public boolean isPartitionConsumingOrHasPendingIngestionAction(int userPartition
41734172
}
41744173

41754174
/**
4176-
* Override the {@link CommonClientConfigs#BOOTSTRAP_SERVERS_CONFIG} config with a remote Kafka bootstrap url.
4175+
* Override the {@link com.linkedin.venice.ConfigKeys#KAFKA_BOOTSTRAP_SERVERS} config with a remote Kafka bootstrap url.
41774176
*/
41784177
protected Properties createKafkaConsumerProperties(
41794178
Properties localConsumerProps,

clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/kafka/KafkaInputDictTrainer.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -182,7 +182,7 @@ synchronized byte[] trainDict(Optional<PubSubConsumerAdapter> reusedConsumerOpti
182182
// Get one split per partition
183183
KafkaInputSplit[] splits = (KafkaInputSplit[]) kafkaInputFormat.getSplitsByRecordsPerSplit(jobConf, Long.MAX_VALUE);
184184
// The following sort is trying to get a deterministic dict with the same input.
185-
Arrays.sort(splits, Comparator.comparingInt(o -> o.getTopicPartition().partition()));
185+
Arrays.sort(splits, Comparator.comparingInt(o -> o.getTopicPartition().getPartitionNumber()));
186186
// Try to gather some records from each partition
187187
PushJobZstdConfig zstdConfig = new PushJobZstdConfig(props, splits.length);
188188
ZstdDictTrainer trainer = trainerSupplier.orElseGet(zstdConfig::getZstdDictTrainer);

clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/kafka/KafkaInputFormat.java

Lines changed: 11 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -8,10 +8,13 @@
88
import com.linkedin.venice.hadoop.input.kafka.avro.KafkaInputMapperValue;
99
import com.linkedin.venice.hadoop.mapreduce.datawriter.task.ReporterBackedMapReduceDataWriterTaskTracker;
1010
import com.linkedin.venice.hadoop.task.datawriter.DataWriterTaskTracker;
11+
import com.linkedin.venice.pubsub.PubSubTopicPartitionImpl;
1112
import com.linkedin.venice.pubsub.PubSubTopicRepository;
1213
import com.linkedin.venice.pubsub.adapter.kafka.admin.ApacheKafkaAdminAdapterFactory;
1314
import com.linkedin.venice.pubsub.adapter.kafka.consumer.ApacheKafkaConsumerAdapterFactory;
1415
import com.linkedin.venice.pubsub.api.PubSubConsumerAdapter;
16+
import com.linkedin.venice.pubsub.api.PubSubTopic;
17+
import com.linkedin.venice.pubsub.api.PubSubTopicPartition;
1518
import com.linkedin.venice.pubsub.api.exceptions.PubSubOpTimeoutException;
1619
import com.linkedin.venice.pubsub.manager.TopicManager;
1720
import com.linkedin.venice.pubsub.manager.TopicManagerContext;
@@ -30,7 +33,6 @@
3033
import org.apache.hadoop.mapred.JobConf;
3134
import org.apache.hadoop.mapred.RecordReader;
3235
import org.apache.hadoop.mapred.Reporter;
33-
import org.apache.kafka.common.TopicPartition;
3436

3537

3638
/**
@@ -49,7 +51,7 @@ public class KafkaInputFormat implements InputFormat<KafkaInputMapperKey, KafkaI
4951
public static final long DEFAULT_KAFKA_INPUT_MAX_RECORDS_PER_MAPPER = 5000000L;
5052
private final PubSubTopicRepository pubSubTopicRepository = new PubSubTopicRepository();
5153

52-
protected Map<TopicPartition, Long> getLatestOffsets(JobConf config) {
54+
protected Map<PubSubTopicPartition, Long> getLatestOffsets(JobConf config) {
5355
VeniceProperties consumerProperties = KafkaInputUtils.getConsumerProperties(config);
5456
TopicManagerContext topicManagerContext =
5557
new TopicManagerContext.Builder().setPubSubPropertiesSupplier(k -> consumerProperties)
@@ -61,16 +63,16 @@ protected Map<TopicPartition, Long> getLatestOffsets(JobConf config) {
6163
.build();
6264
try (TopicManager topicManager =
6365
new TopicManagerRepository(topicManagerContext, config.get(KAFKA_INPUT_BROKER_URL)).getLocalTopicManager()) {
64-
String topic = config.get(KAFKA_INPUT_TOPIC);
65-
66+
PubSubTopic topic = pubSubTopicRepository.getTopic(config.get(KAFKA_INPUT_TOPIC));
6667
Map<Integer, Long> latestOffsets = RetryUtils.executeWithMaxAttempt(
67-
() -> topicManager.getTopicLatestOffsets(pubSubTopicRepository.getTopic(topic)),
68+
() -> topicManager.getTopicLatestOffsets(topic),
6869
10,
6970
Duration.ofMinutes(1),
7071
Arrays.asList(PubSubOpTimeoutException.class));
71-
Map<TopicPartition, Long> partitionOffsetMap = new HashMap<>(latestOffsets.size());
72+
Map<PubSubTopicPartition, Long> partitionOffsetMap = new HashMap<>(latestOffsets.size());
7273
latestOffsets.forEach(
73-
(partitionId, latestOffset) -> partitionOffsetMap.put(new TopicPartition(topic, partitionId), latestOffset));
74+
(partitionId, latestOffset) -> partitionOffsetMap
75+
.put(new PubSubTopicPartitionImpl(topic, partitionId), latestOffset));
7476
return partitionOffsetMap;
7577
}
7678
}
@@ -92,7 +94,7 @@ public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
9294
}
9395

9496
public InputSplit[] getSplitsByRecordsPerSplit(JobConf job, long maxRecordsPerSplit) {
95-
Map<TopicPartition, Long> latestOffsets = getLatestOffsets(job);
97+
Map<PubSubTopicPartition, Long> latestOffsets = getLatestOffsets(job);
9698
List<InputSplit> splits = new LinkedList<>();
9799
latestOffsets.forEach((topicPartition, end) -> {
98100

@@ -103,7 +105,7 @@ public InputSplit[] getSplitsByRecordsPerSplit(JobConf job, long maxRecordsPerSp
103105
long splitStart = 0;
104106
while (splitStart < end) {
105107
long splitEnd = Math.min(splitStart + maxRecordsPerSplit, end);
106-
splits.add(new KafkaInputSplit(topicPartition.topic(), topicPartition.partition(), splitStart, splitEnd));
108+
splits.add(new KafkaInputSplit(pubSubTopicRepository, topicPartition, splitStart, splitEnd));
107109
splitStart = splitEnd;
108110
}
109111
});

clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/kafka/KafkaInputRecordReader.java

Lines changed: 2 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -16,13 +16,11 @@
1616
import com.linkedin.venice.kafka.protocol.Put;
1717
import com.linkedin.venice.kafka.protocol.enums.MessageType;
1818
import com.linkedin.venice.message.KafkaKey;
19-
import com.linkedin.venice.pubsub.PubSubTopicPartitionImpl;
2019
import com.linkedin.venice.pubsub.PubSubTopicRepository;
2120
import com.linkedin.venice.pubsub.adapter.kafka.consumer.ApacheKafkaConsumerAdapterFactory;
2221
import com.linkedin.venice.pubsub.api.PubSubConsumerAdapter;
2322
import com.linkedin.venice.pubsub.api.PubSubMessage;
2423
import com.linkedin.venice.pubsub.api.PubSubMessageDeserializer;
25-
import com.linkedin.venice.pubsub.api.PubSubTopic;
2624
import com.linkedin.venice.pubsub.api.PubSubTopicPartition;
2725
import com.linkedin.venice.utils.Utils;
2826
import com.linkedin.venice.utils.pools.LandFillObjectPool;
@@ -37,7 +35,6 @@
3735
import org.apache.hadoop.mapred.InputSplit;
3836
import org.apache.hadoop.mapred.JobConf;
3937
import org.apache.hadoop.mapred.RecordReader;
40-
import org.apache.kafka.common.TopicPartition;
4138
import org.apache.logging.log4j.LogManager;
4239
import org.apache.logging.log4j.Logger;
4340

@@ -70,8 +67,7 @@ public class KafkaInputRecordReader implements RecordReader<KafkaInputMapperKey,
7067
private static final PubSubTopicRepository PUBSUB_TOPIC_REPOSITORY = new PubSubTopicRepository();
7168

7269
private final PubSubConsumerAdapter consumer;
73-
private final TopicPartition topicPartition;
74-
private final PubSubTopicPartition pubSubTopicPartition;
70+
private final PubSubTopicPartition topicPartition;
7571
private final long maxNumberOfRecords;
7672
private final long startingOffset;
7773
private long currentOffset;
@@ -129,8 +125,6 @@ public KafkaInputRecordReader(
129125
KafkaInputSplit inputSplit = (KafkaInputSplit) split;
130126
this.consumer = consumer;
131127
this.topicPartition = inputSplit.getTopicPartition();
132-
PubSubTopic pubSubTopic = pubSubTopicRepository.getTopic(topicPartition.topic());
133-
this.pubSubTopicPartition = new PubSubTopicPartitionImpl(pubSubTopic, topicPartition.partition());
134128
this.startingOffset = inputSplit.getStartingOffset();
135129
this.currentOffset = inputSplit.getStartingOffset() - 1;
136130
this.endingOffset = inputSplit.getEndingOffset();
@@ -148,7 +142,7 @@ public KafkaInputRecordReader(
148142
if (!ownedConsumer) {
149143
this.consumer.batchUnsubscribe(this.consumer.getAssignment());
150144
}
151-
this.consumer.subscribe(pubSubTopicPartition, currentOffset);
145+
this.consumer.subscribe(topicPartition, currentOffset);
152146
this.taskTracker = taskTracker;
153147
LOGGER.info(
154148
"KafkaInputRecordReader started for TopicPartition: {} starting offset: {} ending offset: {}",

clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/kafka/KafkaInputSplit.java

Lines changed: 19 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -1,43 +1,51 @@
11
package com.linkedin.venice.hadoop.input.kafka;
22

3+
import com.linkedin.venice.pubsub.PubSubTopicPartitionImpl;
4+
import com.linkedin.venice.pubsub.PubSubTopicRepository;
5+
import com.linkedin.venice.pubsub.api.PubSubTopicPartition;
36
import java.io.DataInput;
47
import java.io.DataOutput;
58
import java.io.IOException;
69
import org.apache.hadoop.mapred.InputSplit;
7-
import org.apache.kafka.common.TopicPartition;
810

911

1012
/**
1113
* We borrowed some idea from the open-sourced attic-crunch lib:
1214
* https://github.com/apache/attic-crunch/blob/master/crunch-kafka/src/main/java/org/apache/crunch/kafka/record/KafkaInputSplit.java
1315
*
14-
* InputSplit that represent retrieving data from a single {@link TopicPartition} between the specified start
16+
* InputSplit that represent retrieving data from a single {@link PubSubTopicPartition} between the specified start
1517
* and end offsets.
1618
*/
1719
public class KafkaInputSplit implements InputSplit {
1820
private long startingOffset;
1921
private long endingOffset;
20-
private TopicPartition topicPartition;
22+
private PubSubTopicPartition topicPartition;
23+
private final PubSubTopicRepository topicRepository;
2124

2225
/**
2326
* Nullary Constructor for creating the instance inside the Mapper instance.
2427
*/
2528
public KafkaInputSplit() {
29+
topicRepository = new PubSubTopicRepository();
2630
}
2731

2832
/**
2933
* Constructs an input split for the provided {@param topic} and {@param partition} restricting data to be between
3034
* the {@param startingOffset} and {@param endingOffset}
3135
*
32-
* @param topic the topic for the split
33-
* @param partition the partition for the topic
36+
* @param topicPartition the topic-partition for the split
3437
* @param startingOffset the start of the split
3538
* @param endingOffset the end of the split
3639
*/
37-
public KafkaInputSplit(String topic, int partition, long startingOffset, long endingOffset) {
40+
public KafkaInputSplit(
41+
PubSubTopicRepository topicRepository,
42+
PubSubTopicPartition topicPartition,
43+
long startingOffset,
44+
long endingOffset) {
45+
this.topicRepository = topicRepository;
3846
this.startingOffset = startingOffset;
3947
this.endingOffset = endingOffset;
40-
topicPartition = new TopicPartition(topic, partition);
48+
this.topicPartition = topicPartition;
4149
}
4250

4351
@Override
@@ -57,7 +65,7 @@ public String[] getLocations() throws IOException {
5765
*
5866
* @return the topic and partition for the split
5967
*/
60-
public TopicPartition getTopicPartition() {
68+
public PubSubTopicPartition getTopicPartition() {
6169
return topicPartition;
6270
}
6371

@@ -81,8 +89,8 @@ public long getEndingOffset() {
8189

8290
@Override
8391
public void write(DataOutput dataOutput) throws IOException {
84-
dataOutput.writeUTF(topicPartition.topic());
85-
dataOutput.writeInt(topicPartition.partition());
92+
dataOutput.writeUTF(topicPartition.getTopicName());
93+
dataOutput.writeInt(topicPartition.getPartitionNumber());
8694
dataOutput.writeLong(startingOffset);
8795
dataOutput.writeLong(endingOffset);
8896
}
@@ -94,7 +102,7 @@ public void readFields(DataInput dataInput) throws IOException {
94102
startingOffset = dataInput.readLong();
95103
endingOffset = dataInput.readLong();
96104

97-
topicPartition = new TopicPartition(topic, partition);
105+
topicPartition = new PubSubTopicPartitionImpl(topicRepository.getTopic(topic), partition);
98106
}
99107

100108
@Override

clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/input/kafka/KafkaInputRecordReaderTest.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -80,8 +80,8 @@ public void testNext() throws IOException {
8080
Map<PubSubTopicPartition, List<PubSubMessage<KafkaKey, KafkaMessageEnvelope, Long>>> recordsMap = new HashMap<>();
8181
recordsMap.put(pubSubTopicPartition, consumerRecordList);
8282
when(consumer.poll(anyLong())).thenReturn(recordsMap, new HashMap<>());
83-
84-
KafkaInputSplit split = new KafkaInputSplit(topic, 0, 0, 102);
83+
PubSubTopicPartition topicPartition = new PubSubTopicPartitionImpl(pubSubTopicRepository.getTopic(topic), 0);
84+
KafkaInputSplit split = new KafkaInputSplit(pubSubTopicRepository, topicPartition, 0, 102);
8585
DataWriterTaskTracker taskTracker = new ReporterBackedMapReduceDataWriterTaskTracker(Reporter.NULL);
8686
try (KafkaInputRecordReader reader =
8787
new KafkaInputRecordReader(split, conf, taskTracker, consumer, pubSubTopicRepository)) {

clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/input/kafka/TestKafkaInputDictTrainer.java

Lines changed: 20 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -16,7 +16,10 @@
1616
import com.linkedin.venice.exceptions.VeniceException;
1717
import com.linkedin.venice.hadoop.input.kafka.avro.KafkaInputMapperKey;
1818
import com.linkedin.venice.hadoop.input.kafka.avro.KafkaInputMapperValue;
19+
import com.linkedin.venice.pubsub.PubSubTopicPartitionImpl;
20+
import com.linkedin.venice.pubsub.PubSubTopicRepository;
1921
import com.linkedin.venice.pubsub.api.PubSubConsumerAdapter;
22+
import com.linkedin.venice.pubsub.api.PubSubTopicPartition;
2023
import java.io.IOException;
2124
import java.nio.ByteBuffer;
2225
import java.util.ArrayList;
@@ -30,6 +33,8 @@
3033

3134

3235
public class TestKafkaInputDictTrainer {
36+
private final static PubSubTopicRepository PUB_SUB_TOPIC_REPOSITORY = new PubSubTopicRepository();
37+
3338
private KafkaInputDictTrainer.CompressorBuilder getCompressorBuilder(VeniceCompressor mockCompressor) {
3439
return (compressorFactory, compressionStrategy, kafkaUrl, topic, props) -> mockCompressor;
3540
}
@@ -52,7 +57,9 @@ private KafkaInputDictTrainer.Param getParam(int sampleSize, CompressionStrategy
5257
@Test(expectedExceptions = VeniceException.class, expectedExceptionsMessageRegExp = "No record.*")
5358
public void testEmptyTopic() throws IOException {
5459
KafkaInputFormat mockFormat = mock(KafkaInputFormat.class);
55-
InputSplit[] splits = new KafkaInputSplit[] { new KafkaInputSplit("test_topic", 0, 0, 0) };
60+
PubSubTopicPartition topicPartition =
61+
new PubSubTopicPartitionImpl(PUB_SUB_TOPIC_REPOSITORY.getTopic("test_topic"), 0);
62+
InputSplit[] splits = new KafkaInputSplit[] { new KafkaInputSplit(PUB_SUB_TOPIC_REPOSITORY, topicPartition, 0, 0) };
5663
doReturn(splits).when(mockFormat).getSplitsByRecordsPerSplit(any(), anyLong());
5764
RecordReader<KafkaInputMapperKey, KafkaInputMapperValue> mockRecordReader = mock(RecordReader.class);
5865
doReturn(false).when(mockRecordReader).next(any(), any());
@@ -143,8 +150,10 @@ private ResettableRecordReader<KafkaInputMapperKey, KafkaInputMapperValue> mockR
143150
@Test
144151
public void testSamplingFromMultiplePartitions() throws IOException {
145152
KafkaInputFormat mockFormat = mock(KafkaInputFormat.class);
146-
InputSplit[] splits = new KafkaInputSplit[] { new KafkaInputSplit("test_topic", 0, 0, 2),
147-
new KafkaInputSplit("test_topic", 0, 0, 2) };
153+
PubSubTopicPartition topicPartition =
154+
new PubSubTopicPartitionImpl(PUB_SUB_TOPIC_REPOSITORY.getTopic("test_topic"), 0);
155+
InputSplit[] splits = new KafkaInputSplit[] { new KafkaInputSplit(PUB_SUB_TOPIC_REPOSITORY, topicPartition, 0, 2),
156+
new KafkaInputSplit(PUB_SUB_TOPIC_REPOSITORY, topicPartition, 0, 2) };
148157
doReturn(splits).when(mockFormat).getSplitsByRecordsPerSplit(any(), anyLong());
149158

150159
// Return 3 records
@@ -194,8 +203,10 @@ public void testSamplingFromMultiplePartitions() throws IOException {
194203
@Test
195204
public void testSamplingFromMultiplePartitionsWithSourceVersionCompressionEnabled() throws IOException {
196205
KafkaInputFormat mockFormat = mock(KafkaInputFormat.class);
197-
InputSplit[] splits = new KafkaInputSplit[] { new KafkaInputSplit("test_topic", 0, 0, 2),
198-
new KafkaInputSplit("test_topic", 0, 0, 2) };
206+
PubSubTopicPartition topicPartition =
207+
new PubSubTopicPartitionImpl(PUB_SUB_TOPIC_REPOSITORY.getTopic("test_topic"), 0);
208+
InputSplit[] splits = new KafkaInputSplit[] { new KafkaInputSplit(PUB_SUB_TOPIC_REPOSITORY, topicPartition, 0, 2),
209+
new KafkaInputSplit(PUB_SUB_TOPIC_REPOSITORY, topicPartition, 0, 2) };
199210
doReturn(splits).when(mockFormat).getSplitsByRecordsPerSplit(any(), anyLong());
200211

201212
// Return 3 records
@@ -244,8 +255,10 @@ public void testSamplingFromMultiplePartitionsWithSourceVersionCompressionEnable
244255
@Test
245256
public void testSamplingFromMultiplePartitionsWithSourceVersionCompressionEnabledWithChunking() throws IOException {
246257
KafkaInputFormat mockFormat = mock(KafkaInputFormat.class);
247-
InputSplit[] splits = new KafkaInputSplit[] { new KafkaInputSplit("test_topic", 0, 0, 2),
248-
new KafkaInputSplit("test_topic", 0, 0, 2) };
258+
PubSubTopicPartition topicPartition =
259+
new PubSubTopicPartitionImpl(PUB_SUB_TOPIC_REPOSITORY.getTopic("test_topic"), 0);
260+
InputSplit[] splits = new KafkaInputSplit[] { new KafkaInputSplit(PUB_SUB_TOPIC_REPOSITORY, topicPartition, 0, 2),
261+
new KafkaInputSplit(PUB_SUB_TOPIC_REPOSITORY, topicPartition, 0, 2) };
249262
doReturn(splits).when(mockFormat).getSplitsByRecordsPerSplit(any(), anyLong());
250263

251264
// Return 3 records

0 commit comments

Comments
 (0)