lindong28 commented on code in PR #21589:
URL: https://github.com/apache/flink/pull/21589#discussion_r1096967718


##########
flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaSourceReaderTest.java:
##########
@@ -479,6 +486,125 @@ public void testSupportsPausingOrResumingSplits() throws 
Exception {
         }
     }
 
+    @Test
+    public void testSupportsUsingRecordEvaluatorWithSplitsFinishedAtMiddle() 
throws Exception {
+        final Set<String> finishedSplits = new HashSet<>();
+        try (final KafkaSourceReader<Integer> reader =
+                (KafkaSourceReader<Integer>)
+                        createReader(
+                                Boundedness.BOUNDED,
+                                "groupId",
+                                new TestingReaderContext(),
+                                finishedSplits::addAll,
+                                r -> r == 7 || r == NUM_RECORDS_PER_SPLIT + 
5)) {

Review Comment:
   I suppose this means that we should receive 7 messages from the 1st 
partition (not including the message whose value is 7) and 5 messages from the 
2nd partition, right?
   
   Then why the test below expects to receive 14 messages?



##########
flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaSourceReaderTest.java:
##########
@@ -479,6 +486,125 @@ public void testSupportsPausingOrResumingSplits() throws 
Exception {
         }
     }
 
+    @Test
+    public void testSupportsUsingRecordEvaluatorWithSplitsFinishedAtMiddle() 
throws Exception {
+        final Set<String> finishedSplits = new HashSet<>();
+        try (final KafkaSourceReader<Integer> reader =
+                (KafkaSourceReader<Integer>)
+                        createReader(
+                                Boundedness.BOUNDED,
+                                "groupId",
+                                new TestingReaderContext(),
+                                finishedSplits::addAll,
+                                r -> r == 7 || r == NUM_RECORDS_PER_SPLIT + 
5)) {
+            KafkaPartitionSplit split1 =
+                    new KafkaPartitionSplit(new TopicPartition(TOPIC, 0), 0, 
Integer.MAX_VALUE);
+            KafkaPartitionSplit split2 =
+                    new KafkaPartitionSplit(new TopicPartition(TOPIC, 1), 0, 
Integer.MAX_VALUE);
+            reader.addSplits(Arrays.asList(split1, split2));
+            reader.notifyNoMoreSplits();
+
+            TestingReaderOutput<Integer> output = new TestingReaderOutput<>();
+            pollUntil(
+                    reader,
+                    output,
+                    () -> output.getEmittedRecords().size() == 14,

Review Comment:
   Instead of having the test stop after receiving 14 messages, would it be 
better to let the test keep pulling until both partitions are in 
`finishedSplits`? Then we can see whether the `eofRecordEvaluator` can stop the 
consumption at the right point.



##########
flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaSourceReaderTest.java:
##########
@@ -479,6 +486,125 @@ public void testSupportsPausingOrResumingSplits() throws 
Exception {
         }
     }
 
+    @Test
+    public void testSupportsUsingRecordEvaluatorWithSplitsFinishedAtMiddle() 
throws Exception {
+        final Set<String> finishedSplits = new HashSet<>();
+        try (final KafkaSourceReader<Integer> reader =
+                (KafkaSourceReader<Integer>)
+                        createReader(
+                                Boundedness.BOUNDED,
+                                "groupId",
+                                new TestingReaderContext(),
+                                finishedSplits::addAll,
+                                r -> r == 7 || r == NUM_RECORDS_PER_SPLIT + 
5)) {
+            KafkaPartitionSplit split1 =
+                    new KafkaPartitionSplit(new TopicPartition(TOPIC, 0), 0, 
Integer.MAX_VALUE);
+            KafkaPartitionSplit split2 =
+                    new KafkaPartitionSplit(new TopicPartition(TOPIC, 1), 0, 
Integer.MAX_VALUE);
+            reader.addSplits(Arrays.asList(split1, split2));
+            reader.notifyNoMoreSplits();
+
+            TestingReaderOutput<Integer> output = new TestingReaderOutput<>();
+            pollUntil(
+                    reader,
+                    output,
+                    () -> output.getEmittedRecords().size() == 14,
+                    "The reader cannot get the excepted result before 
timeout.");
+            InputStatus status;
+            while (true) {

Review Comment:
   What is the purpose of this while loop?



##########
flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceITCase.java:
##########
@@ -167,6 +174,58 @@ public void testBasicRead(boolean enableObjectReuse) 
throws Exception {
             executeAndVerify(env, stream);
         }
 
+        @ParameterizedTest(name = "Object reuse in deserializer = {arguments}")
+        @ValueSource(booleans = {false, true})
+        public void testEndWithRecordEvaluator(boolean enableObjectReuse) 
throws Exception {
+            KafkaSource<PartitionAndValue> source =
+                    KafkaSource.<PartitionAndValue>builder()
+                            
.setBootstrapServers(KafkaSourceTestEnv.brokerConnectionStrings)
+                            .setGroupId("testEndWithRecordEvaluator")
+                            .setTopics(Arrays.asList(TOPIC1, TOPIC3))

Review Comment:
   Would it be simpler to just use `TOPIC2` instead of creating a new `TOPIC3`?



##########
flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicSource.java:
##########
@@ -486,6 +493,12 @@ protected KafkaSource<RowData> createKafkaSource(
                 break;
         }
 
+        if (recordEvaluator != null) {
+            kafkaSourceBuilder
+                    .setBounded(new NoStoppingOffsetsInitializer())

Review Comment:
   Why do we need this change? I suppose the boundedness of kafka source should 
be independent of whether `recordEvaluator` is specified.



##########
flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/mocks/MockRecordEvaluator.java:
##########
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.base.source.reader.mocks;
+
+import org.apache.flink.connector.base.source.reader.RecordEvaluator;
+
+import java.util.function.Function;
+
+/** A mock RecordEvaluator class. */
+public class MockRecordEvaluator implements RecordEvaluator<Integer> {

Review Comment:
   Would it be simpler to define it as a private class 
in`SourceReaderBaseTest`, similar to the existing private class 
SourceReaderBaseTest#OnEventWatermarkGenerator?



##########
flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableTestBase.java:
##########
@@ -147,6 +153,46 @@ public void createTestTopic(String topic, int 
numPartitions, int replicationFact
         }
     }
 
+    public <K, V> void produceToKafka(

Review Comment:
   Should we produce data to Kafka topic similar to the existing tests such as 
`KafkaTableITCase#testPerPartitionWatermarkWithIdleSource` for consistency?



##########
flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/MockRecordEvaluator.java:
##########
@@ -0,0 +1,12 @@
+package org.apache.flink.streaming.connectors.kafka.testutils;
+
+import org.apache.flink.connector.base.source.reader.RecordEvaluator;
+import org.apache.flink.table.data.RowData;
+
+/** A mock record evaluator. */
+public class MockRecordEvaluator implements RecordEvaluator<RowData> {

Review Comment:
   Would it be simpler to define it as a public class in `KafkaTableITCase`, 
similar to the existing public class `KafkaTableITCase#TestPartitioner`?



##########
flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactoryTest.java:
##########
@@ -400,6 +407,49 @@ public void testTableSourceSetOffsetResetWithException() {
                                 ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, 
errorStrategy));
     }
 
+    @Test
+    public void testTableSourceWithRecordEvaluator() {
+        Map<String, String> tableOptions = getBasicSourceOptions();
+        tableOptions.put(
+                "scan.record.evaluator.class",
+                
"org.apache.flink.streaming.connectors.kafka.testutils.MockRecordEvaluator");

Review Comment:
   Would it be better to pass the value as 
`MockRecordEvaluator.class.getName()` similar to what 
`testPerPartitionWatermarkKafka` does to get TestPartitioner's class name?



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