leonardBang commented on code in PR #19456:
URL: https://github.com/apache/flink/pull/19456#discussion_r892310026
##########
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:
> all assigned partitions are invalid
'all' should be 'some of'?
##########
flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceITCase.java:
##########
@@ -443,13 +502,15 @@ public void invoke(PartitionAndValue value, Context
context) {
.add(partitionAndValue.value));
resultPerPartition.forEach(
(tp, values) -> {
- int firstExpectedValue =
Integer.parseInt(tp.substring(tp.indexOf('-') + 1));
+ int firstExpectedValue =
+ Integer.parseInt(tp.substring(tp.lastIndexOf('-')
+ 1));
Review Comment:
we can add a note here:
the elements of partition is an automatic integer sequence start from the
partition number.
##########
flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceITCase.java:
##########
@@ -309,6 +309,65 @@ public void processElement(
});
env.execute();
}
+
+ @Test
+ public void testConsumingEmptyTopic() throws Throwable {
+ String emptyTopic = "emptyTopic-" + UUID.randomUUID();
+ KafkaSourceTestEnv.createTestTopic(emptyTopic, 3, 1);
+ KafkaSource<PartitionAndValue> source =
+ KafkaSource.<PartitionAndValue>builder()
+
.setBootstrapServers(KafkaSourceTestEnv.brokerConnectionStrings)
+ .setTopics(emptyTopic)
+ .setGroupId("empty-topic-test")
+ .setDeserializer(new
TestingKafkaRecordDeserializationSchema(false))
+ .setStartingOffsets(OffsetsInitializer.earliest())
+ .setBounded(OffsetsInitializer.latest())
+ .build();
+ StreamExecutionEnvironment env =
StreamExecutionEnvironment.getExecutionEnvironment();
+ env.setParallelism(1);
+ try (CloseableIterator<PartitionAndValue> iterator =
+ env.fromSource(
+ source,
+ WatermarkStrategy.noWatermarks(),
+ "testConsumingEmptyTopic")
+ .executeAndCollect()) {
+ assertThat(iterator.hasNext()).isFalse();
+ }
+ }
+
+ @Test
+ public void testConsumingTopicWithEmptyPartitions() throws Throwable {
+ String topicWithEmptyPartitions = "topicWithEmptyPartitions-" +
UUID.randomUUID();
+ KafkaSourceTestEnv.createTestTopic(
+ topicWithEmptyPartitions,
KafkaSourceTestEnv.NUM_PARTITIONS, 1);
+ List<ProducerRecord<String, Integer>> records =
+
KafkaSourceTestEnv.getRecordsForTopicWithoutTimestamp(topicWithEmptyPartitions);
+ // Only keep records in partition 5
+ int partitionWithRecords = 5;
+ records.removeIf(record -> record.partition() !=
partitionWithRecords);
+ KafkaSourceTestEnv.produceToKafka(records);
+ KafkaSourceTestEnv.setupEarliestOffsets(
+ Collections.singletonList(
+ new TopicPartition(topicWithEmptyPartitions,
partitionWithRecords)));
+
+ KafkaSource<PartitionAndValue> source =
+ KafkaSource.<PartitionAndValue>builder()
+
.setBootstrapServers(KafkaSourceTestEnv.brokerConnectionStrings)
+ .setTopics(topicWithEmptyPartitions)
+ .setGroupId("topic-with-empty-partition-test")
+ .setDeserializer(new
TestingKafkaRecordDeserializationSchema(false))
+ .setStartingOffsets(OffsetsInitializer.earliest())
+ .setBounded(OffsetsInitializer.latest())
+ .build();
+ StreamExecutionEnvironment env =
StreamExecutionEnvironment.getExecutionEnvironment();
+ env.setParallelism(2);
Review Comment:
Could we use default parallelism?
What's the purpose to set different parallelism in these test cases?
--
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]