Skip to content

Commit 9801c43

Browse files
authored
Fixed integration tests (#22)
### Motivation KafkaApiTest is failing ### Modifications 1. conversion between Pulsar topic name and Kafka TopicPartition ended up with TopicPartition using name with "-partition-<partition idx>" 2. Seek was not working correctly: PulsarKafkaConsumer seeks to beginning, as asked. Clears lastReceivedOffset in the process. on poll it checks ``` if (lastReceivedOffset.get(tp) == null && !unpolledPartitions.contains(tp)) { log.info("When polling offsets, invalid offsets were detected. Resetting topic partition {}", tp); resetOffsets(tp); } ``` seek didn't update unpolledPartitions - reset offset uses default strategy to reset => seeks to the end ### Verifying this change - [ ] Make sure that the change passes the CI checks. *(Please pick either of the following options)* This change is already covered by existing tests, such as KafkaApiTest. ### Does this pull request potentially affect one of the following parts: *If `yes` was chosen, please highlight the changes* No ### Documentation - Does this pull request introduce a new feature? NO
1 parent 74c0dd8 commit 9801c43

File tree

3 files changed

+44
-27
lines changed

3 files changed

+44
-27
lines changed

pulsar-client-kafka-compat/pulsar-client-kafka/src/main/java/org/apache/kafka/clients/consumer/PulsarKafkaConsumer.java

Lines changed: 30 additions & 16 deletions
Original file line numberDiff line numberDiff line change
@@ -47,7 +47,6 @@
4747
import org.apache.pulsar.client.util.MessageIdUtils;
4848
import org.apache.pulsar.common.naming.TopicName;
4949
import org.apache.pulsar.common.util.FutureUtil;
50-
import org.apache.pulsar.functions.utils.FunctionCommon;
5150

5251
import java.time.Duration;
5352
import java.util.AbstractMap;
@@ -59,7 +58,6 @@
5958
import java.util.HashSet;
6059
import java.util.List;
6160
import java.util.Map;
62-
import java.util.Objects;
6361
import java.util.Properties;
6462
import java.util.Set;
6563
import java.util.concurrent.ArrayBlockingQueue;
@@ -277,9 +275,7 @@ public void subscribe(Collection<String> topics, ConsumerRebalanceListener callb
277275
CompletableFuture<org.apache.pulsar.client.api.Consumer<byte[]>> future = consumerBuilder.clone()
278276
.topic(partitionName).subscribeAsync();
279277
int partitionIndex = i;
280-
TopicPartition tp = new TopicPartition(
281-
TopicName.get(topic).getLocalName(),
282-
partitionIndex);
278+
TopicPartition tp = normalizedTopicPartition(topic, partitionIndex);
283279
futures.add(future.thenApply(consumer -> {
284280
log.info("Add consumer {} for partition {}", consumer, tp);
285281
consumers.putIfAbsent(tp, consumer);
@@ -291,9 +287,7 @@ public void subscribe(Collection<String> topics, ConsumerRebalanceListener callb
291287
// Topic has a single partition
292288
CompletableFuture<org.apache.pulsar.client.api.Consumer<byte[]>> future = consumerBuilder.topic(topic)
293289
.subscribeAsync();
294-
TopicPartition tp = new TopicPartition(
295-
TopicName.get(topic).getLocalName(),
296-
0);
290+
TopicPartition tp = normalizedTopicPartition(topic, 0);
297291
futures.add(future.thenApply(consumer -> {
298292
log.info("Add consumer {} for partition {}", consumer, tp);
299293
consumers.putIfAbsent(tp, consumer);
@@ -327,6 +321,15 @@ public void subscribe(Collection<String> topics, ConsumerRebalanceListener callb
327321
}
328322
}
329323

324+
private TopicPartition normalizedTopicPartition(TopicPartition tp) {
325+
return normalizedTopicPartition(tp.topic(), tp.partition());
326+
}
327+
328+
private TopicPartition normalizedTopicPartition(String topic, int partition) {
329+
String name = TopicName.get(topic).getPartitionedTopicName();
330+
return new TopicPartition(name, partition);
331+
}
332+
330333
@Override
331334
public void assign(Collection<TopicPartition> partitions) {
332335
Set<String> topics = partitions.stream().map(p -> p.topic()).collect(Collectors.toSet());
@@ -372,7 +375,7 @@ public ConsumerRecords<K, V> poll(long timeoutMillis) {
372375

373376
while (item != null) {
374377
TopicName topicName = TopicName.get(item.consumer.getTopic());
375-
String topic = topicName.getLocalName();
378+
String topic = topicName.getPartitionedTopicName();
376379
int partition = topicName.isPartitioned() ? topicName.getPartitionIndex() : 0;
377380
Message<byte[]> msg = item.message;
378381
MessageIdImpl msgId = (MessageIdImpl) msg.getMessageId();
@@ -504,12 +507,15 @@ public void commitAsync(Map<TopicPartition, OffsetAndMetadata> offsets, OffsetCo
504507
}
505508

506509
private CompletableFuture<Void> doCommitOffsets(Map<TopicPartition, OffsetAndMetadata> offsets) {
510+
Preconditions.checkNotNull(offsets);
507511
List<CompletableFuture<Void>> futures = new ArrayList<>();
508512

509513
applyConsumerInterceptorsOnCommit(interceptors, offsets);
510-
offsets.forEach((topicPartition, offsetAndMetadata) -> {
514+
offsets.forEach((tp, offsetAndMetadata) -> {
515+
TopicPartition topicPartition = normalizedTopicPartition(tp);
511516
org.apache.pulsar.client.api.Consumer<byte[]> consumer = consumers.get(topicPartition);
512-
lastCommittedOffset.put(topicPartition, offsetAndMetadata);
517+
518+
lastCommittedOffset.put(tp, offsetAndMetadata);
513519
futures.add(consumer.acknowledgeCumulativeAsync(MessageIdUtils.getMessageId(offsetAndMetadata.offset())));
514520
});
515521

@@ -566,7 +572,8 @@ private void applyConsumerInterceptorsOnCommit(List<ConsumerInterceptor<K, V>> i
566572
@Override
567573
public void seek(TopicPartition partition, long offset) {
568574
MessageId msgId = MessageIdUtils.getMessageId(offset);
569-
org.apache.pulsar.client.api.Consumer<byte[]> c = consumers.get(partition);
575+
TopicPartition topicPartition = normalizedTopicPartition(partition);
576+
org.apache.pulsar.client.api.Consumer<byte[]> c = consumers.get(topicPartition);
570577
if (c == null) {
571578
throw new IllegalArgumentException("Cannot seek on a partition where we are not subscribed");
572579
}
@@ -594,12 +601,14 @@ public void seekToBeginning(Collection<TopicPartition> partitions) {
594601
lastReceivedOffset.clear();
595602

596603
for (TopicPartition tp : partitions) {
597-
org.apache.pulsar.client.api.Consumer<byte[]> c = consumers.get(tp);
604+
TopicPartition normalizedTp = normalizedTopicPartition(tp);
605+
org.apache.pulsar.client.api.Consumer<byte[]> c = consumers.get(normalizedTp);
598606
if (c == null) {
599607
futures.add(FutureUtil.failedFuture(
600608
new IllegalArgumentException("Cannot seek on a partition where we are not subscribed")));
601609
} else {
602610
futures.add(c.seekAsync(MessageId.earliest));
611+
unpolledPartitions.add(tp);
603612
}
604613
}
605614

@@ -617,12 +626,15 @@ public void seekToEnd(Collection<TopicPartition> partitions) {
617626
lastReceivedOffset.clear();
618627

619628
for (TopicPartition tp : partitions) {
620-
org.apache.pulsar.client.api.Consumer<byte[]> c = consumers.get(tp);
629+
TopicPartition normalizedTp = normalizedTopicPartition(tp);
630+
org.apache.pulsar.client.api.Consumer<byte[]> c = consumers.get(normalizedTp);
631+
621632
if (c == null) {
622633
futures.add(FutureUtil.failedFuture(
623634
new IllegalArgumentException("Cannot seek on a partition where we are not subscribed")));
624635
} else {
625636
futures.add(c.seekAsync(MessageId.latest));
637+
unpolledPartitions.add(tp);
626638
}
627639
}
628640

@@ -712,7 +724,8 @@ public Set<TopicPartition> paused() {
712724
@Override
713725
public void pause(Collection<TopicPartition> partitions) {
714726
partitions.forEach(p -> {
715-
org.apache.pulsar.client.api.Consumer<byte[]> c = consumers.get(p);
727+
TopicPartition topicPartition = normalizedTopicPartition(p);
728+
org.apache.pulsar.client.api.Consumer<byte[]> c = consumers.get(topicPartition);
716729
if (c != null) {
717730
c.pause();
718731
}
@@ -722,7 +735,8 @@ public void pause(Collection<TopicPartition> partitions) {
722735
@Override
723736
public void resume(Collection<TopicPartition> partitions) {
724737
partitions.forEach(p -> {
725-
org.apache.pulsar.client.api.Consumer<byte[]> c = consumers.get(p);
738+
TopicPartition topicPartition = normalizedTopicPartition(p);
739+
org.apache.pulsar.client.api.Consumer<byte[]> c = consumers.get(topicPartition);
726740
if (c != null) {
727741
c.resume();
728742
}

pulsar-client-kafka-compat/pulsar-client-kafka_0_8/src/test/java/org/apache/pulsar/client/kafka/test/KafkaProducerSimpleConsumerTest.java

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -141,6 +141,7 @@ public void testPulsarKafkaProducerWithSerializer(int partitions) throws Excepti
141141
producer.send(message);
142142
}
143143
producer.close();
144+
Thread.sleep(500);
144145

145146
// (2) Consume using simple consumer
146147
PulsarKafkaSimpleConsumer consumer = new PulsarKafkaSimpleConsumer(serviceUrl, 0, 0, 0, "clientId");
@@ -158,6 +159,7 @@ public void testPulsarKafkaProducerWithSerializer(int partitions) throws Excepti
158159
.build();
159160
FetchResponse fetchResponse = consumer.fetch(fReq);
160161

162+
Thread.sleep(500);
161163
long lastOffset = 0;
162164
MessageId offset = null;
163165
for (MessageAndOffset messageAndOffset : fetchResponse.messageSet(topicName, partition)) {

tests/pulsar-spark-test/src/test/java/org/apache/pulsar/spark/SparkStreamingPulsarReceiverTest.java

Lines changed: 12 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -27,6 +27,7 @@
2727
import java.util.HashSet;
2828
import java.util.Map;
2929
import java.util.Set;
30+
import java.util.function.Supplier;
3031

3132
import org.apache.commons.lang3.mutable.MutableInt;
3233
import org.apache.pulsar.client.api.Consumer;
@@ -48,7 +49,7 @@ public class SparkStreamingPulsarReceiverTest extends PulsarTestSuite {
4849
private static final String EXPECTED_MESSAGE = "pulsar-spark test message";
4950

5051
@Test(dataProvider = "ServiceUrls")
51-
public void testReceivedMessage(String serviceUrl) throws Exception {
52+
public void testReceivedMessage(Supplier<String> serviceUrl) throws Exception {
5253
ConsumerConfigurationData<byte[]> consConf = new ConsumerConfigurationData<>();
5354

5455
Set<String> set = new HashSet<>();
@@ -68,14 +69,14 @@ public void received(Consumer consumer, Message msg) {
6869
consConf.setMessageListener(msgListener);
6970

7071
SparkStreamingPulsarReceiver receiver = new SparkStreamingPulsarReceiver(
71-
serviceUrl,
72+
serviceUrl.get(),
7273
consConf,
7374
new AuthenticationDisabled());
7475

7576
receiver.onStart();
7677
waitForTransmission();
7778

78-
PulsarClient client = PulsarClient.builder().serviceUrl(serviceUrl).build();
79+
PulsarClient client = PulsarClient.builder().serviceUrl(serviceUrl.get()).build();
7980
Producer<byte[]> producer = client.newProducer().topic(TOPIC).create();
8081
producer.send(EXPECTED_MESSAGE.getBytes());
8182

@@ -85,7 +86,7 @@ public void received(Consumer consumer, Message msg) {
8586
}
8687

8788
@Test(dataProvider = "ServiceUrls")
88-
public void testDefaultSettingsOfReceiver(String serviceUrl) {
89+
public void testDefaultSettingsOfReceiver(Supplier<String> serviceUrl) {
8990
ConsumerConfigurationData<byte[]> consConf = new ConsumerConfigurationData<>();
9091

9192
Set<String> set = new HashSet<>();
@@ -94,7 +95,7 @@ public void testDefaultSettingsOfReceiver(String serviceUrl) {
9495
consConf.setSubscriptionName(SUBS);
9596

9697
SparkStreamingPulsarReceiver receiver = new SparkStreamingPulsarReceiver(
97-
serviceUrl,
98+
serviceUrl.get(),
9899
consConf,
99100
new AuthenticationDisabled());
100101

@@ -103,7 +104,7 @@ public void testDefaultSettingsOfReceiver(String serviceUrl) {
103104
}
104105

105106
@Test(dataProvider = "ServiceUrls")
106-
public void testSharedSubscription(String serviceUrl) throws Exception {
107+
public void testSharedSubscription(Supplier<String> serviceUrl) throws Exception {
107108
ConsumerConfigurationData<byte[]> consConf = new ConsumerConfigurationData<>();
108109

109110
Set<String> set = new HashSet<>();
@@ -120,20 +121,20 @@ public void testSharedSubscription(String serviceUrl) throws Exception {
120121
});
121122

122123
SparkStreamingPulsarReceiver receiver1 = new SparkStreamingPulsarReceiver(
123-
serviceUrl,
124+
serviceUrl.get(),
124125
consConf,
125126
new AuthenticationDisabled());
126127

127128
SparkStreamingPulsarReceiver receiver2 = new SparkStreamingPulsarReceiver(
128-
serviceUrl,
129+
serviceUrl.get(),
129130
consConf,
130131
new AuthenticationDisabled());
131132

132133
receiver1.onStart();
133134
receiver2.onStart();
134135
waitForTransmission();
135136

136-
PulsarClient client = PulsarClient.builder().serviceUrl(serviceUrl).build();
137+
PulsarClient client = PulsarClient.builder().serviceUrl(serviceUrl.get()).build();
137138
Producer<byte[]> producer = client.newProducer().topic(TOPIC).create();
138139
for (int i = 0; i < 10; i++) {
139140
producer.send(EXPECTED_MESSAGE.getBytes());
@@ -149,8 +150,8 @@ public void testSharedSubscription(String serviceUrl) throws Exception {
149150
@Test(expectedExceptions = NullPointerException.class,
150151
expectedExceptionsMessageRegExp = "ConsumerConfigurationData must not be null",
151152
dataProvider = "ServiceUrls")
152-
public void testReceiverWhenClientConfigurationIsNull(String serviceUrl) {
153-
new SparkStreamingPulsarReceiver(serviceUrl, null, new AuthenticationDisabled());
153+
public void testReceiverWhenClientConfigurationIsNull(Supplier<String> serviceUrl) {
154+
new SparkStreamingPulsarReceiver(serviceUrl.get(), null, new AuthenticationDisabled());
154155
}
155156

156157
private static void waitForTransmission() {

0 commit comments

Comments
 (0)