KAFKA-14708: Use Java thread instead of kafka library for example purpose (#13238)

Remove "kafka.examples.Consumer" dependency on ShutdownableThread. "examples" module should be dependent only on public APIs but not to be dependent upon server common/internal components.

Reviewers: Luke Chen <showuon@gmail.com>
This commit is contained in:
Philip Nee 2023-02-15 19:28:32 -08:00 committed by GitHub
parent 958bc0601c
commit 6d64075230
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
1 changed files with 16 additions and 15 deletions

View File

@ -16,7 +16,6 @@
*/
package kafka.examples;
import kafka.utils.ShutdownableThread;
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.consumer.ConsumerRecords;
@ -28,7 +27,7 @@ import java.util.Optional;
import java.util.Properties;
import java.util.concurrent.CountDownLatch;
public class Consumer extends ShutdownableThread {
public class Consumer extends Thread {
private final KafkaConsumer<Integer, String> consumer;
private final String topic;
private final String groupId;
@ -42,7 +41,7 @@ public class Consumer extends ShutdownableThread {
final boolean readCommitted,
final int numMessageToConsume,
final CountDownLatch latch) {
super("KafkaConsumerExample", false);
super("KafkaConsumerExample");
this.groupId = groupId;
Properties props = new Properties();
props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, KafkaProperties.KAFKA_SERVER_URL + ":" + KafkaProperties.KAFKA_SERVER_PORT);
@ -70,6 +69,18 @@ public class Consumer extends ShutdownableThread {
}
@Override
public void run() {
try {
do {
doWork();
} while (messageRemaining > 0);
System.out.println(groupId + " finished reading " + numMessageToConsume + " messages");
} catch (Exception e) {
System.out.println("Unexpected termination, exception thrown:" + e);
} finally {
shutdown();
}
}
public void doWork() {
consumer.subscribe(Collections.singletonList(this.topic));
ConsumerRecords<Integer, String> records = consumer.poll(Duration.ofSeconds(1));
@ -77,19 +88,9 @@ public class Consumer extends ShutdownableThread {
System.out.println(groupId + " received message : from partition " + record.partition() + ", (" + record.key() + ", " + record.value() + ") at offset " + record.offset());
}
messageRemaining -= records.count();
if (messageRemaining <= 0) {
System.out.println(groupId + " finished reading " + numMessageToConsume + " messages");
latch.countDown();
}
}
@Override
public String name() {
return null;
}
@Override
public boolean isInterruptible() {
return false;
public void shutdown() {
latch.countDown();
}
}