PatrickRen commented on code in PR #19456:
URL: https://github.com/apache/flink/pull/19456#discussion_r894256191


##########
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:
   I'm afraid the default parallelism will be quite large on CI (# CPU cores) 
and occupy too much resources. Setting parallelism to 2 is enough to reproduce 
the issue. 
   
   This case is for testing the case that only some partitions are empty, so 
that readers assigned with empty partitions can be closed without exception and 
other readers can process messages as expected. 



-- 
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