You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2019/01/06 20:58:38 UTC

[GitHub] sijie commented on a change in pull request #3310: Add a running flag for Kafka source connector

sijie commented on a change in pull request #3310: Add a running flag for Kafka source connector
URL: https://github.com/apache/pulsar/pull/3310#discussion_r245522208
 
 

 ##########
 File path: pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaAbstractSource.java
 ##########
 @@ -106,32 +107,42 @@ public void close() throws InterruptedException {
 
     public void start() {
         runnerThread = new Thread(() -> {
-            LOG.info("Starting kafka source");
-            consumer = new KafkaConsumer<>(beforeCreateConsumer(props));
-            consumer.subscribe(Arrays.asList(kafkaSourceConfig.getTopic()));
-            LOG.info("Kafka source started.");
-            ConsumerRecords<String, byte[]> consumerRecords;
-            while(true){
-                consumerRecords = consumer.poll(1000);
-                CompletableFuture<?>[] futures = new CompletableFuture<?>[consumerRecords.count()];
-                int index = 0;
-                for (ConsumerRecord<String, byte[]> consumerRecord : consumerRecords) {
-                    LOG.debug("Record received from kafka, key: {}. value: {}", consumerRecord.key(), consumerRecord.value());
-                    KafkaRecord<V> record = new KafkaRecord<>(consumerRecord, extractValue(consumerRecord));
-                    consume(record);
-                    futures[index] = record.getCompletableFuture();
-                    index++;
-                }
-                if (!kafkaSourceConfig.isAutoCommitEnabled()) {
-                    try {
-                        CompletableFuture.allOf(futures).get();
-                        consumer.commitSync();
-                    } catch (ExecutionException | InterruptedException ex) {
-                        break;
+            try {
+                consumer = new KafkaConsumer<>(beforeCreateConsumer(props));
+            } catch (Exception ex) {
+                LOG.error("Unable to instantiate Kafka consumer", ex);
+                return;
+            }
+            while (running) {
+                LOG.info("Starting kafka source");
+                try {
+                    consumer.subscribe(Collections.singletonList(kafkaSourceConfig.getTopic()));
+                    LOG.info("Kafka source started.");
+                    ConsumerRecords<String, byte[]> consumerRecords;
+                    while (running) {
+                        consumerRecords = consumer.poll(1000);
+                        CompletableFuture<?>[] futures = new CompletableFuture<?>[consumerRecords.count()];
+                        int index = 0;
+                        for (ConsumerRecord<String, byte[]> consumerRecord : consumerRecords) {
+                            LOG.debug("Record received from kafka, key: {}. value: {}", consumerRecord.key(), consumerRecord.value());
+                            KafkaRecord<V> record = new KafkaRecord<>(consumerRecord, extractValue(consumerRecord));
+                            consume(record);
+                            futures[index] = record.getCompletableFuture();
+                            index++;
+                        }
+                        if (!kafkaSourceConfig.isAutoCommitEnabled()) {
+                            try {
+                                CompletableFuture.allOf(futures).get();
+                                consumer.commitSync();
+                            } catch (InterruptedException ex) {
+                                break;
+                            }
+                        }
                     }
+                } catch (Exception ex) {
+                    LOG.error("Error while consuming data from Kafka", ex);
 
 Review comment:
   if we fail to consume, the logic here will cause the connector going into a retry loop and use a same consumer instance to subscribe. does kafka client guarantee the consumer instance usable after exceptions occur?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services