AHeise commented on a change in pull request #17406:
URL: https://github.com/apache/flink/pull/17406#discussion_r721318188



##########
File path: 
flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaUtil.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.kafka.sink;
+
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.PartitionInfo;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/** Collection of methods to interact with a Kafka cluster. */
+public class KafkaUtil {
+
+    private static final Logger LOG = LoggerFactory.getLogger(KafkaUtil.class);
+    private static final Duration CONSUMER_POLL_DURATION = 
Duration.ofSeconds(1);
+
+    private KafkaUtil() {}
+
+    /**
+     * Drain all records available from the given topic from the beginning 
until the current highest
+     * offset.
+     *
+     * <p>This method will fetch the latest offsets for the partitions once 
and only return records
+     * until that point.
+     *
+     * @param topic to fetch from
+     * @param properties used to configure the created {@link KafkaConsumer}
+     * @param committed determines the mode {@link 
ConsumerConfig#ISOLATION_LEVEL_CONFIG} with which
+     *     the consumer reads the records.
+     * @return all {@link ConsumerRecord} in the topic
+     * @throws KafkaException
+     */
+    public static List<ConsumerRecord<byte[], byte[]>> 
drainAllRecordsFromTopic(
+            String topic, Properties properties, boolean committed) throws 
KafkaException {

Review comment:
       What's in `properties` except the URL at this point?

##########
File path: 
flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java
##########
@@ -279,32 +279,16 @@ public String getVersion() {
     }
 
     @Override
+    @SuppressWarnings("unchecked")
     public <K, V> Collection<ConsumerRecord<K, V>> getAllRecordsFromTopic(
-            Properties properties, String topic, int partition, long timeout) {
-        List<ConsumerRecord<K, V>> result = new ArrayList<>();
-
-        try (KafkaConsumer<K, V> consumer = new KafkaConsumer<>(properties)) {
-            consumer.assign(Arrays.asList(new TopicPartition(topic, 
partition)));
-
-            while (true) {
-                boolean processedAtLeastOneRecord = false;
-
-                // wait for new records with timeout and break the loop if we 
didn't get any
-                Iterator<ConsumerRecord<K, V>> iterator = 
consumer.poll(timeout).iterator();
-                while (iterator.hasNext()) {
-                    ConsumerRecord<K, V> record = iterator.next();
-                    result.add(record);
-                    processedAtLeastOneRecord = true;
-                }
-
-                if (!processedAtLeastOneRecord) {
-                    break;
-                }
-            }
-            consumer.commitSync();
-        }
-
-        return UnmodifiableList.decorate(result);
+            Properties properties, String topic) {
+        final boolean committed =
+                Objects.equals(
+                        properties.getProperty(
+                                ConsumerConfig.ISOLATION_LEVEL_CONFIG, 
"read_uncommitted"),
+                        "read_committed");
+        return UnmodifiableList.decorate(
+                KafkaUtil.drainAllRecordsFromTopic(topic, properties, 
committed));

Review comment:
       Maybe offer 2 overloads of `drainAllRecordsFromTopic` one with 
properties, one with URL+committed flag?

##########
File path: 
flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaUtil.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.kafka.sink;
+
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.PartitionInfo;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/** Collection of methods to interact with a Kafka cluster. */
+public class KafkaUtil {
+
+    private static final Logger LOG = LoggerFactory.getLogger(KafkaUtil.class);
+    private static final Duration CONSUMER_POLL_DURATION = 
Duration.ofSeconds(1);
+
+    private KafkaUtil() {}
+
+    /**
+     * Drain all records available from the given topic from the beginning 
until the current highest
+     * offset.
+     *
+     * <p>This method will fetch the latest offsets for the partitions once 
and only return records
+     * until that point.
+     *
+     * @param topic to fetch from
+     * @param properties used to configure the created {@link KafkaConsumer}
+     * @param committed determines the mode {@link 
ConsumerConfig#ISOLATION_LEVEL_CONFIG} with which
+     *     the consumer reads the records.
+     * @return all {@link ConsumerRecord} in the topic
+     * @throws KafkaException
+     */
+    public static List<ConsumerRecord<byte[], byte[]>> 
drainAllRecordsFromTopic(
+            String topic, Properties properties, boolean committed) throws 
KafkaException {
+        final Properties consumerConfig = new Properties();
+        consumerConfig.putAll(properties);
+        consumerConfig.put("key.deserializer", 
ByteArrayDeserializer.class.getName());
+        consumerConfig.put("value.deserializer", 
ByteArrayDeserializer.class.getName());
+        consumerConfig.put(
+                ConsumerConfig.ISOLATION_LEVEL_CONFIG,
+                committed ? "read_committed" : "read_uncommitted");
+        try (KafkaConsumer<byte[], byte[]> consumer = new 
KafkaConsumer<>(consumerConfig)) {
+            Map<Integer, TopicPartition> assignments = 
getAllPartitions(consumer, topic);
+            Map<TopicPartition, Long> endOffsets = 
consumer.endOffsets(assignments.values());
+            consumer.assign(assignments.values());
+            consumer.seekToBeginning(assignments.values());
+
+            final List<ConsumerRecord<byte[], byte[]>> consumerRecords = new 
ArrayList<>();
+            while (!assignments.isEmpty()) {
+                consumer.assign(assignments.values());
+                ConsumerRecords<byte[], byte[]> records = 
consumer.poll(CONSUMER_POLL_DURATION);
+                LOG.info("Fetched {} records from topic {}.", records.count(), 
topic);
+
+                // Remove partitions from polling which have reached its end.
+                final Iterator<Map.Entry<Integer, TopicPartition>> 
assignmentIterator =
+                        assignments.entrySet().iterator();
+                while (assignmentIterator.hasNext()) {
+                    final Map.Entry<Integer, TopicPartition> assignment = 
assignmentIterator.next();
+                    final TopicPartition topicPartition = 
assignment.getValue();
+                    final long position = consumer.position(topicPartition);
+                    final long endOffset = endOffsets.get(topicPartition);
+                    LOG.info(
+                            "Endoffset {} and current position {} for 
partition {}",
+                            endOffset,
+                            position,
+                            assignment.getKey());
+                    if (endOffset - position > 0) {
+                        continue;
+                    }
+                    assignmentIterator.remove();
+                }

Review comment:
       We could have a `finishedPartitions` list in each iteration. Then we 
don't need the Iterator as we would only modify assignments in a batch way at 
the end of the iteration. Further, we would only call `consumer.assign` on a 
change.

##########
File path: 
flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaUtil.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.kafka.sink;
+
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.PartitionInfo;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/** Collection of methods to interact with a Kafka cluster. */
+public class KafkaUtil {
+
+    private static final Logger LOG = LoggerFactory.getLogger(KafkaUtil.class);
+    private static final Duration CONSUMER_POLL_DURATION = 
Duration.ofSeconds(1);
+
+    private KafkaUtil() {}
+
+    /**
+     * Drain all records available from the given topic from the beginning 
until the current highest
+     * offset.
+     *
+     * <p>This method will fetch the latest offsets for the partitions once 
and only return records
+     * until that point.
+     *
+     * @param topic to fetch from
+     * @param properties used to configure the created {@link KafkaConsumer}
+     * @param committed determines the mode {@link 
ConsumerConfig#ISOLATION_LEVEL_CONFIG} with which
+     *     the consumer reads the records.
+     * @return all {@link ConsumerRecord} in the topic
+     * @throws KafkaException
+     */
+    public static List<ConsumerRecord<byte[], byte[]>> 
drainAllRecordsFromTopic(
+            String topic, Properties properties, boolean committed) throws 
KafkaException {
+        final Properties consumerConfig = new Properties();
+        consumerConfig.putAll(properties);
+        consumerConfig.put("key.deserializer", 
ByteArrayDeserializer.class.getName());
+        consumerConfig.put("value.deserializer", 
ByteArrayDeserializer.class.getName());
+        consumerConfig.put(
+                ConsumerConfig.ISOLATION_LEVEL_CONFIG,
+                committed ? "read_committed" : "read_uncommitted");
+        try (KafkaConsumer<byte[], byte[]> consumer = new 
KafkaConsumer<>(consumerConfig)) {
+            Map<Integer, TopicPartition> assignments = 
getAllPartitions(consumer, topic);
+            Map<TopicPartition, Long> endOffsets = 
consumer.endOffsets(assignments.values());
+            consumer.assign(assignments.values());
+            consumer.seekToBeginning(assignments.values());
+
+            final List<ConsumerRecord<byte[], byte[]>> consumerRecords = new 
ArrayList<>();
+            while (!assignments.isEmpty()) {
+                consumer.assign(assignments.values());
+                ConsumerRecords<byte[], byte[]> records = 
consumer.poll(CONSUMER_POLL_DURATION);
+                LOG.info("Fetched {} records from topic {}.", records.count(), 
topic);
+
+                // Remove partitions from polling which have reached its end.
+                final Iterator<Map.Entry<Integer, TopicPartition>> 
assignmentIterator =
+                        assignments.entrySet().iterator();
+                while (assignmentIterator.hasNext()) {
+                    final Map.Entry<Integer, TopicPartition> assignment = 
assignmentIterator.next();
+                    final TopicPartition topicPartition = 
assignment.getValue();
+                    final long position = consumer.position(topicPartition);
+                    final long endOffset = endOffsets.get(topicPartition);
+                    LOG.info(
+                            "Endoffset {} and current position {} for 
partition {}",
+                            endOffset,
+                            position,
+                            assignment.getKey());

Review comment:
       Is this the only place where we need the key?

##########
File path: 
flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaUtil.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.kafka.sink;
+
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.PartitionInfo;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/** Collection of methods to interact with a Kafka cluster. */
+public class KafkaUtil {
+
+    private static final Logger LOG = LoggerFactory.getLogger(KafkaUtil.class);
+    private static final Duration CONSUMER_POLL_DURATION = 
Duration.ofSeconds(1);
+
+    private KafkaUtil() {}
+
+    /**
+     * Drain all records available from the given topic from the beginning 
until the current highest
+     * offset.
+     *
+     * <p>This method will fetch the latest offsets for the partitions once 
and only return records
+     * until that point.
+     *
+     * @param topic to fetch from
+     * @param properties used to configure the created {@link KafkaConsumer}
+     * @param committed determines the mode {@link 
ConsumerConfig#ISOLATION_LEVEL_CONFIG} with which
+     *     the consumer reads the records.
+     * @return all {@link ConsumerRecord} in the topic
+     * @throws KafkaException
+     */
+    public static List<ConsumerRecord<byte[], byte[]>> 
drainAllRecordsFromTopic(
+            String topic, Properties properties, boolean committed) throws 
KafkaException {
+        final Properties consumerConfig = new Properties();
+        consumerConfig.putAll(properties);
+        consumerConfig.put("key.deserializer", 
ByteArrayDeserializer.class.getName());
+        consumerConfig.put("value.deserializer", 
ByteArrayDeserializer.class.getName());
+        consumerConfig.put(
+                ConsumerConfig.ISOLATION_LEVEL_CONFIG,
+                committed ? "read_committed" : "read_uncommitted");
+        try (KafkaConsumer<byte[], byte[]> consumer = new 
KafkaConsumer<>(consumerConfig)) {
+            Map<Integer, TopicPartition> assignments = 
getAllPartitions(consumer, topic);
+            Map<TopicPartition, Long> endOffsets = 
consumer.endOffsets(assignments.values());
+            consumer.assign(assignments.values());
+            consumer.seekToBeginning(assignments.values());
+
+            final List<ConsumerRecord<byte[], byte[]>> consumerRecords = new 
ArrayList<>();
+            while (!assignments.isEmpty()) {
+                consumer.assign(assignments.values());
+                ConsumerRecords<byte[], byte[]> records = 
consumer.poll(CONSUMER_POLL_DURATION);
+                LOG.info("Fetched {} records from topic {}.", records.count(), 
topic);

Review comment:
       debug




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