mas-chen commented on code in PR #19456:
URL: https://github.com/apache/flink/pull/19456#discussion_r850797500


##########
flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/KafkaPartitionSplitReader.java:
##########
@@ -131,12 +138,7 @@ public RecordsWithSplitIds<ConsumerRecord<byte[], byte[]>> 
fetch() throws IOExce
             kafkaSourceReaderMetrics.maybeAddRecordsLagMetric(consumer, tp);
         }
 
-        // Some splits are discovered as empty when handling split additions. 
These splits should be
-        // added to finished splits to clean up states in split fetcher and 
source reader.
-        if (!emptySplits.isEmpty()) {
-            recordsBySplits.finishedSplits.addAll(emptySplits);
-            emptySplits.clear();
-        }
+        markEmptySplitsAsFinished(recordsBySplits);

Review Comment:
   We should only finish empty splits in bounded mode, right? Empty partitions 
may eventually contain data in streaming mode.



##########
flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/KafkaPartitionSplitReader.java:
##########
@@ -98,9 +98,16 @@ public RecordsWithSplitIds<ConsumerRecord<byte[], byte[]>> 
fetch() throws IOExce
         ConsumerRecords<byte[], byte[]> consumerRecords;
         try {
             consumerRecords = consumer.poll(Duration.ofMillis(POLL_TIMEOUT));
-        } catch (WakeupException we) {
-            return new KafkaPartitionSplitRecords(
-                    ConsumerRecords.empty(), kafkaSourceReaderMetrics);
+        } catch (WakeupException | IllegalStateException e) {
+            // IllegalStateException will be thrown if the consumer is not 
assigned any partitions.
+            // This happens if all assigned partitions are invalid or empty 
(starting offset >=

Review Comment:
   The design makes sense, but I was thinking in terms of the specified offsets 
initializer. Looks like it is handled properly by the enumerator in that case 
though.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to