You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by ar...@apache.org on 2021/11/11 10:14:04 UTC

[flink] branch release-1.14 updated: [FLINK-24733][connector/pulsar] Data loss in pulsar source when using shared mode

This is an automated email from the ASF dual-hosted git repository.

arvid pushed a commit to branch release-1.14
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/release-1.14 by this push:
     new 6a319e4  [FLINK-24733][connector/pulsar] Data loss in pulsar source when using shared mode
6a319e4 is described below

commit 6a319e421b848f20ea67f54ee2eaa9e1c73d91c4
Author: wsz94 <wa...@163.com>
AuthorDate: Wed Nov 10 18:30:33 2021 +0800

    [FLINK-24733][connector/pulsar] Data loss in pulsar source when using shared mode
---
 .../split/PulsarOrderedPartitionSplitReader.java   |   7 +-
 .../split/PulsarPartitionSplitReaderBase.java      |   9 +-
 .../split/PulsarUnorderedPartitionSplitReader.java |  11 +-
 .../PulsarOrderedPartitionSplitReaderTest.java     |  37 ++++
 .../split/PulsarPartitionSplitReaderTestBase.java  | 188 +++++++++++++++++++++
 .../PulsarUnorderedPartitionSplitReaderTest.java   |  38 +++++
 6 files changed, 281 insertions(+), 9 deletions(-)

diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReader.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReader.java
index af650ea..8643891 100644
--- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReader.java
+++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReader.java
@@ -37,9 +37,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.time.Duration;
-import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
 
 import static org.apache.flink.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyClient;
 import static org.apache.flink.connector.pulsar.source.config.CursorVerification.FAIL_ON_MISMATCH;
@@ -65,9 +63,8 @@ public class PulsarOrderedPartitionSplitReader<OUT> extends PulsarPartitionSplit
     }
 
     @Override
-    protected Message<byte[]> pollMessage(Duration timeout)
-            throws ExecutionException, InterruptedException, TimeoutException {
-        return pulsarConsumer.receiveAsync().get(timeout.toMillis(), TimeUnit.MILLISECONDS);
+    protected Message<byte[]> pollMessage(Duration timeout) throws PulsarClientException {
+        return pulsarConsumer.receive(Math.toIntExact(timeout.toMillis()), TimeUnit.MILLISECONDS);
     }
 
     @Override
diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderBase.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderBase.java
index 906c626..69c7b5e 100644
--- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderBase.java
+++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderBase.java
@@ -40,11 +40,14 @@ import org.apache.pulsar.client.api.ConsumerBuilder;
 import org.apache.pulsar.client.api.KeySharedPolicy;
 import org.apache.pulsar.client.api.Message;
 import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
 import org.apache.pulsar.client.api.Schema;
 import org.apache.pulsar.client.api.SubscriptionType;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import javax.annotation.Nullable;
+
 import java.io.IOException;
 import java.time.Duration;
 import java.util.List;
@@ -114,6 +117,9 @@ abstract class PulsarPartitionSplitReaderBase<OUT>
             try {
                 Duration timeout = deadline.timeLeftIfAny();
                 Message<byte[]> message = pollMessage(timeout);
+                if (message == null) {
+                    break;
+                }
 
                 // Deserialize message.
                 collector.setMessage(message);
@@ -189,8 +195,9 @@ abstract class PulsarPartitionSplitReaderBase<OUT>
         }
     }
 
+    @Nullable
     protected abstract Message<byte[]> pollMessage(Duration timeout)
-            throws ExecutionException, InterruptedException, TimeoutException;
+            throws ExecutionException, InterruptedException, PulsarClientException;
 
     protected abstract void finishedPollMessage(Message<byte[]> message);
 
diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReader.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReader.java
index 9c8e3d7..846101d 100644
--- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReader.java
+++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReader.java
@@ -31,6 +31,7 @@ import org.apache.pulsar.client.admin.PulsarAdmin;
 import org.apache.pulsar.client.api.Consumer;
 import org.apache.pulsar.client.api.Message;
 import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
 import org.apache.pulsar.client.api.transaction.Transaction;
 import org.apache.pulsar.client.api.transaction.TransactionCoordinatorClient;
 import org.apache.pulsar.client.api.transaction.TransactionCoordinatorClientException;
@@ -44,7 +45,6 @@ import java.time.Duration;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
 
 import static org.apache.flink.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyClient;
 
@@ -79,9 +79,14 @@ public class PulsarUnorderedPartitionSplitReader<OUT> extends PulsarPartitionSpl
 
     @Override
     protected Message<byte[]> pollMessage(Duration timeout)
-            throws ExecutionException, InterruptedException, TimeoutException {
+            throws ExecutionException, InterruptedException, PulsarClientException {
         Message<byte[]> message =
-                pulsarConsumer.receiveAsync().get(timeout.toMillis(), TimeUnit.MILLISECONDS);
+                pulsarConsumer.receive(Math.toIntExact(timeout.toMillis()), TimeUnit.MILLISECONDS);
+
+        // Skip the message when receive timeout
+        if (message == null) {
+            return null;
+        }
 
         if (!sourceConfiguration.isEnableAutoAcknowledgeMessage()) {
             if (uncommittedTransaction == null) {
diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReaderTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReaderTest.java
new file mode 100644
index 0000000..a94504a
--- /dev/null
+++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReaderTest.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.pulsar.source.reader.split;
+
+import org.apache.flink.api.common.serialization.SimpleStringSchema;
+
+import static org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema.flinkSchema;
+
+/** Unit tests for {@link PulsarOrderedPartitionSplitReaderTest}. */
+class PulsarOrderedPartitionSplitReaderTest extends PulsarPartitionSplitReaderTestBase {
+
+    @Override
+    protected PulsarPartitionSplitReaderBase<String> splitReader() {
+        return new PulsarOrderedPartitionSplitReader<>(
+                operator().client(),
+                operator().admin(),
+                readerConfig(),
+                sourceConfig(),
+                flinkSchema(new SimpleStringSchema()));
+    }
+}
diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderTestBase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderTestBase.java
new file mode 100644
index 0000000..fb9fa01
--- /dev/null
+++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderTestBase.java
@@ -0,0 +1,188 @@
+/*
+ * 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.pulsar.source.reader.split;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitsAddition;
+import org.apache.flink.connector.pulsar.source.config.SourceConfiguration;
+import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition;
+import org.apache.flink.connector.pulsar.source.reader.message.PulsarMessage;
+import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit;
+import org.apache.flink.connector.pulsar.testutils.PulsarTestSuiteBase;
+
+import org.junit.jupiter.api.DisplayName;
+import org.junit.jupiter.api.TestTemplate;
+import org.junit.jupiter.api.extension.Extension;
+import org.junit.jupiter.api.extension.ExtensionContext;
+import org.junit.jupiter.api.extension.ParameterContext;
+import org.junit.jupiter.api.extension.ParameterResolutionException;
+import org.junit.jupiter.api.extension.ParameterResolver;
+import org.junit.jupiter.api.extension.RegisterExtension;
+import org.junit.jupiter.api.extension.TestTemplateInvocationContext;
+import org.junit.jupiter.api.extension.TestTemplateInvocationContextProvider;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Stream;
+
+import static java.time.Duration.ofSeconds;
+import static java.util.Collections.singletonList;
+import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic;
+import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_ENABLE_AUTO_ACKNOWLEDGE_MESSAGE;
+import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_MAX_FETCH_RECORDS;
+import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_MAX_FETCH_TIME;
+import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_NAME;
+import static org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor.never;
+import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicNameWithPartition;
+import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange.createFullRange;
+import static org.apache.flink.core.testutils.CommonTestUtils.waitUtil;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.pulsar.client.api.Schema.STRING;
+import static org.junit.jupiter.api.Assertions.assertNull;
+
+/** Test utils for split readers. */
+public abstract class PulsarPartitionSplitReaderTestBase extends PulsarTestSuiteBase {
+
+    @RegisterExtension
+    PulsarSplitReaderInvocationContextProvider provider =
+            new PulsarSplitReaderInvocationContextProvider();
+
+    protected Configuration readerConfig() {
+        Configuration config = operator().config();
+        config.set(PULSAR_MAX_FETCH_RECORDS, 1);
+        config.set(PULSAR_MAX_FETCH_TIME, 1000L);
+        config.set(PULSAR_SUBSCRIPTION_NAME, randomAlphabetic(10));
+        config.set(PULSAR_ENABLE_AUTO_ACKNOWLEDGE_MESSAGE, true);
+
+        return config;
+    }
+
+    protected SourceConfiguration sourceConfig() {
+        return new SourceConfiguration(readerConfig());
+    }
+
+    protected SplitsAddition<PulsarPartitionSplit> createSplit(String topicName, int partitionId) {
+        TopicPartition partition = new TopicPartition(topicName, partitionId, createFullRange());
+        PulsarPartitionSplit split = new PulsarPartitionSplit(partition, never());
+        return new SplitsAddition<>(singletonList(split));
+    }
+
+    protected <T> PulsarMessage<T> fetchedMessage(PulsarPartitionSplitReaderBase<T> splitReader) {
+        try {
+            RecordsWithSplitIds<PulsarMessage<T>> records = splitReader.fetch();
+            if (records.nextSplit() != null) {
+                return records.nextRecordFromSplit();
+            } else {
+                return null;
+            }
+        } catch (IOException e) {
+            return null;
+        }
+    }
+
+    @TestTemplate
+    @DisplayName("Retrieve message after timeout by using given split reader")
+    void pollMessageAfterTimeout(PulsarPartitionSplitReaderBase<String> splitReader)
+            throws InterruptedException, TimeoutException {
+        String topicName = randomAlphabetic(10);
+
+        // Add a split
+        splitReader.handleSplitsChanges(createSplit(topicName, 0));
+
+        // Poll once with a null message
+        PulsarMessage<String> message1 = fetchedMessage(splitReader);
+        assertNull(message1);
+
+        // Send a message to pulsar
+        String topic = topicNameWithPartition(topicName, 0);
+        operator().sendMessage(topic, STRING, randomAlphabetic(10));
+
+        // Poll this message again
+        waitUtil(
+                () -> {
+                    PulsarMessage<String> message2 = fetchedMessage(splitReader);
+                    return message2 != null;
+                },
+                ofSeconds(10),
+                "Couldn't poll message from Pulsar.");
+    }
+
+    /** Create a split reader with max message 1, fetch timeout 1s. */
+    protected abstract PulsarPartitionSplitReaderBase<String> splitReader();
+
+    /** JUnit5 extension for all the TestTemplate methods in this class. */
+    public class PulsarSplitReaderInvocationContextProvider
+            implements TestTemplateInvocationContextProvider {
+
+        @Override
+        public boolean supportsTestTemplate(ExtensionContext context) {
+            return true;
+        }
+
+        @Override
+        public Stream<TestTemplateInvocationContext> provideTestTemplateInvocationContexts(
+                ExtensionContext context) {
+            return Stream.of(new PulsarSplitReaderInvocationContext(splitReader()));
+        }
+    }
+
+    /** Parameter resolver for Split Reader. */
+    public static class PulsarSplitReaderInvocationContext
+            implements TestTemplateInvocationContext {
+
+        private final PulsarPartitionSplitReaderBase<?> splitReader;
+
+        public PulsarSplitReaderInvocationContext(PulsarPartitionSplitReaderBase<?> splitReader) {
+            this.splitReader = checkNotNull(splitReader);
+        }
+
+        @Override
+        public String getDisplayName(int invocationIndex) {
+            return splitReader.getClass().getSimpleName();
+        }
+
+        @Override
+        public List<Extension> getAdditionalExtensions() {
+            return Collections.singletonList(
+                    new ParameterResolver() {
+                        @Override
+                        public boolean supportsParameter(
+                                ParameterContext parameterContext,
+                                ExtensionContext extensionContext)
+                                throws ParameterResolutionException {
+                            return parameterContext
+                                    .getParameter()
+                                    .getType()
+                                    .equals(PulsarPartitionSplitReaderBase.class);
+                        }
+
+                        @Override
+                        public Object resolveParameter(
+                                ParameterContext parameterContext,
+                                ExtensionContext extensionContext)
+                                throws ParameterResolutionException {
+                            return splitReader;
+                        }
+                    });
+        }
+    }
+}
diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReaderTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReaderTest.java
new file mode 100644
index 0000000..917cacd
--- /dev/null
+++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReaderTest.java
@@ -0,0 +1,38 @@
+/*
+ * 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.pulsar.source.reader.split;
+
+import org.apache.flink.api.common.serialization.SimpleStringSchema;
+
+import static org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema.flinkSchema;
+
+/** Unit tests for {@link PulsarUnorderedPartitionSplitReaderTest}. */
+class PulsarUnorderedPartitionSplitReaderTest extends PulsarPartitionSplitReaderTestBase {
+
+    @Override
+    protected PulsarPartitionSplitReaderBase<String> splitReader() {
+        return new PulsarUnorderedPartitionSplitReader<>(
+                operator().client(),
+                operator().admin(),
+                readerConfig(),
+                sourceConfig(),
+                flinkSchema(new SimpleStringSchema()),
+                null);
+    }
+}