You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/07/14 14:28:13 UTC

[GitHub] [flink] tisonkun commented on a diff in pull request #19972: [FLINK-27399][Connector/Pulsar] Change initial consuming position setting logic for better handle the checkpoint.

tisonkun commented on code in PR #19972:
URL: https://github.com/apache/flink/pull/19972#discussion_r921051194


##########
flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReader.java:
##########
@@ -80,12 +80,18 @@ protected void startConsumer(PulsarPartitionSplit split, Consumer<byte[]> consum
 
         // Reset the start position for ordered pulsar consumer.
         if (latestConsumedId != null) {
-            StartCursor startCursor = StartCursor.fromMessageId(latestConsumedId, false);
-            TopicPartition partition = split.getPartition();
-
+            LOG.debug("Start seeking from the checkpoint {}", latestConsumedId);
             try {
-                startCursor.seekPosition(
-                        partition.getTopic(), partition.getPartitionId(), consumer);
+                MessageId initialPosition;
+                if (latestConsumedId == MessageId.latest
+                        || latestConsumedId == MessageId.earliest) {
+                    // This logic is added only the compatible.

Review Comment:
   ```suggestion
                       //  for compatibility
   ```



##########
flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/MessageIdUtils.java:
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.enumerator.cursor;
+
+import org.apache.flink.annotation.Internal;
+
+import org.apache.pulsar.client.api.MessageId;
+import org.apache.pulsar.client.impl.BatchMessageIdImpl;
+import org.apache.pulsar.client.impl.MessageIdImpl;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/** The helper class for Pulsar's message id. */
+@Internal
+public final class MessageIdUtils {
+
+    private MessageIdUtils() {
+        // No public constructor.
+    }
+
+    /**
+     * The implementation from the <a
+     * href="https://github.com/apache/pulsar/blob/7c8dc3201baad7d02d886dbc26db5c03abce77d6/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/PositionImpl.java#L85">this
+     * code</a> to get the next message id.
+     */

Review Comment:
   ```suggestion
       /**
        * The implementation from <a
        * href="https://github.com/apache/pulsar/blob/7c8dc3201baad7d02d886dbc26db5c03abce77d6/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/PositionImpl.java#L85">this
        * code snippet</a> to get next message id.
        */
   ```



##########
flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SplitAssignerFactory.java:
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.enumerator.assigner;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.pulsar.source.config.SourceConfiguration;
+import org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumState;
+import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor;
+
+import org.apache.pulsar.client.api.SubscriptionType;
+
+import static org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumState.initialState;
+import static org.apache.pulsar.client.api.SubscriptionType.Exclusive;
+import static org.apache.pulsar.client.api.SubscriptionType.Failover;
+import static org.apache.pulsar.client.api.SubscriptionType.Key_Shared;
+import static org.apache.pulsar.client.api.SubscriptionType.Shared;
+
+/** The factory for creating split assigner. */
+@Internal
+public final class SplitAssignerFactory {
+
+    private SplitAssignerFactory() {
+        // No public constructor.
+    }
+
+    /** Create blank assigner. */
+    public static SplitAssigner create(
+            StopCursor stopCursor, SourceConfiguration sourceConfiguration) {
+        return create(stopCursor, sourceConfiguration, initialState());
+    }
+
+    /** Create assigner from checkpoint state. */
+    public static SplitAssigner create(
+            StopCursor stopCursor,
+            SourceConfiguration sourceConfiguration,
+            PulsarSourceEnumState sourceEnumState) {
+        SubscriptionType subscriptionType = sourceConfiguration.getSubscriptionType();
+        if (subscriptionType == Exclusive
+                || subscriptionType == Failover
+                || subscriptionType == Key_Shared) {
+            return new NormalSplitAssigner(stopCursor, sourceConfiguration, sourceEnumState);
+        } else if (subscriptionType == Shared) {
+            return new SharedSplitAssigner(stopCursor, sourceConfiguration, sourceEnumState);
+        } else {
+            throw new IllegalArgumentException(
+                    "We don't support this subscription type: " + subscriptionType);
+        }

Review Comment:
   ```suggestion
           switch (subscriptionType) {
               case Exclusive:
               case Failover:
               case Key_Shared:
                   return new NormalSplitAssigner(stopCursor, sourceConfiguration, sourceEnumState);
               case Shared:
                   return new SharedSplitAssigner(stopCursor, sourceConfiguration, sourceEnumState);
               default:
                   throw new IllegalArgumentException("Unknown subscription type: " + subscriptionType);
           }
   ```



##########
flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/PulsarSourceEnumerator.java:
##########
@@ -248,17 +230,19 @@ private void assignPendingPartitionSplits(List<Integer> pendingReaders) {
                 });
 
         // Assign splits to downstream readers.
-        assignmentState.assignSplits(pendingReaders).ifPresent(context::assignSplits);
+        splitAssigner.createAssignment(pendingReaders).ifPresent(context::assignSplits);
 
         // If periodically partition discovery is disabled and the initializing discovery has done,
-        // signal NoMoreSplitsEvent to pending readers
-        if (assignmentState.noMoreNewPartitionSplits()) {
-            LOG.debug(
-                    "No more PulsarPartitionSplits to assign."
-                            + " Sending NoMoreSplitsEvent to reader {} in subscription {}.",
-                    pendingReaders,
-                    sourceConfiguration.getSubscriptionDesc());
-            pendingReaders.forEach(this.context::signalNoMoreSplits);
+        // signal NoMoreSplitsEvent to pending readers.
+        for (Integer reader : pendingReaders) {
+            if (splitAssigner.noMoreSplits(reader)) {
+                LOG.debug(
+                        "No more PulsarPartitionSplits to assign."
+                                + " Sending NoMoreSplitsEvent to reader {} in subscription {}.",
+                        reader,
+                        sourceConfiguration.getSubscriptionDesc());
+                context.signalNoMoreSplits(reader);

Review Comment:
   Here is no explicit happens-before relation. The event will be queued into source operator event thread and when it gets processed the source reader sets its internal state, which will be tested and then generates `InputStatus.END_OF_INPUT`.



##########
flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/MessageIdStartCursor.java:
##########
@@ -43,23 +44,16 @@ public class MessageIdStartCursor implements StartCursor {
      * code</a> for understanding pulsar internal logic.
      *
      * @param messageId The message id for start position.
-     * @param inclusive Should we include the start message id in consuming result.
+     * @param inclusive Whether we include the start message id in consuming result. This works only
+     *     if we provide a specified message id instead of {@link MessageId#earliest} or {@link
+     *     MessageId#latest}.
      */
     public MessageIdStartCursor(MessageId messageId, boolean inclusive) {
-        if (inclusive) {
-            this.messageId = messageId;
+        MessageIdImpl idImpl = unwrapMessageId(messageId);
+        if (MessageId.earliest.equals(idImpl) || MessageId.latest.equals(idImpl) || inclusive) {
+            this.messageId = idImpl;
         } else {
-            checkState(
-                    messageId instanceof MessageIdImpl,
-                    "We only support normal message id and batch message id.");
-            MessageIdImpl id = (MessageIdImpl) messageId;
-            if (MessageId.earliest.equals(messageId) || MessageId.latest.equals(messageId)) {
-                this.messageId = messageId;
-            } else {
-                this.messageId =
-                        new MessageIdImpl(
-                                id.getLedgerId(), id.getEntryId() + 1, id.getPartitionIndex());
-            }
+            this.messageId = nextMessageId(idImpl);

Review Comment:
   @syhily IIRC you ever told me that we change the logic to get next message id in this PR. But it seems just the same before and after? Or `if (idImpl.getEntryId() < 0)` significant?



##########
flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/LatestMessageStopCursor.java:
##########
@@ -30,13 +30,26 @@
 /**
  * A stop cursor that initialize the position to the latest message id. The offsets initialization
  * are taken care of by the {@code PulsarPartitionSplitReaderBase} instead of by the {@code
- * PulsarSourceEnumerator}.
+ * PulsarSourceEnumerator}. We would include the latest message available in Pulsar by default.
  */
 public class LatestMessageStopCursor implements StopCursor {
     private static final long serialVersionUID = 1702059838323965723L;
 
     private MessageId messageId;
 
+    /**
+     * Set this to false would include the latest available message when the flink pipeline start.
+     */
+    private final boolean exclusive;
+
+    public LatestMessageStopCursor() {
+        this.exclusive = false;
+    }

Review Comment:
   ```suggestion
   ```
   
   I tend to remove this constructor as `exclusive = false` is not an intuitive default.



##########
flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/CursorPosition.java:
##########
@@ -52,22 +52,19 @@ public CursorPosition(@Nullable Long timestamp) {
         this.timestamp = timestamp;

Review Comment:
   In these two constructors, the argument should be `Nonnull`?



##########
flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SplitAssignerTestBase.java:
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.enumerator.assigner;
+
+import org.apache.flink.api.connector.source.SplitsAssignment;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.pulsar.source.config.SourceConfiguration;
+import org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumState;
+import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor;
+import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition;
+import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+
+import static java.util.Collections.emptyList;
+import static java.util.Collections.singleton;
+import static java.util.Collections.singletonList;
+import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_PARTITION_DISCOVERY_INTERVAL_MS;
+import static org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumState.initialState;
+import static org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor.defaultStopCursor;
+import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange.createFullRange;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Test utils for split assigners. */
+abstract class SplitAssignerTestBase<T extends SplitAssigner> {

Review Comment:
   ```suggestion
   abstract class SplitAssignerTestBase<T extends SplitAssigner> extends TestLogger {
   ```



##########
flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/MessageIdStartCursor.java:
##########
@@ -43,23 +44,16 @@ public class MessageIdStartCursor implements StartCursor {
      * code</a> for understanding pulsar internal logic.
      *
      * @param messageId The message id for start position.
-     * @param inclusive Should we include the start message id in consuming result.
+     * @param inclusive Whether we include the start message id in consuming result. This works only
+     *     if we provide a specified message id instead of {@link MessageId#earliest} or {@link
+     *     MessageId#latest}.
      */
     public MessageIdStartCursor(MessageId messageId, boolean inclusive) {

Review Comment:
   Can we keep consistency to take an `exclusive` argument?



##########
flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursor.java:
##########
@@ -65,28 +65,48 @@ static StopCursor latest() {
     }
 
     /**
-     * Stop when the messageId is equal or greater than the specified messageId. Message that is
-     * equal to the specified messageId will not be consumed.
+     * Stop consuming when the messageId is equal or greater than the specified messageId. Message
+     * that is equal to the specified messageId will not be consumed.
      */
     static StopCursor atMessageId(MessageId messageId) {
-        return new MessageIdStopCursor(messageId);
+        if (MessageId.latest.equals(messageId)) {
+            return new LatestMessageStopCursor(true);
+        } else {
+            return new MessageIdStopCursor(messageId);
+        }
     }
 
     /**
-     * Stop when the messageId is greater than the specified messageId. Message that is equal to the
-     * specified messageId will be consumed.
+     * Stop consuming when the messageId is greater than the specified messageId. Message that is
+     * equal to the specified messageId will be consumed.
      */
     static StopCursor afterMessageId(MessageId messageId) {
-        return new MessageIdStopCursor(messageId, false);
+        if (MessageId.latest.equals(messageId)) {
+            return new LatestMessageStopCursor(false);
+        } else {
+            return new MessageIdStopCursor(messageId, false);
+        }
     }
 
-    @Deprecated
+    /** Stop consuming when message eventTime is greater than or equals the specified timestamp. */
     static StopCursor atEventTime(long timestamp) {
         return new EventTimestampStopCursor(timestamp);
     }
 
-    /** Stop when message publishTime is greater than the specified timestamp. */
+    /** Stop consuming when message eventTime is greater than the specified timestamp. */
+    static StopCursor afterEventTime(long timestamp) {
+        return new EventTimestampStopCursor(timestamp + 1);
+    }
+
+    /**
+     * Stop consuming when message publishTime is greater than or equals the specified timestamp.
+     */
     static StopCursor atPublishTime(long timestamp) {
         return new PublishTimestampStopCursor(timestamp);
     }
+
+    /** Stop consuming when message publishTime is greater than the specified timestamp. */
+    static StopCursor afterPublishTime(long timestamp) {
+        return new PublishTimestampStopCursor(timestamp + 1);
+    }

Review Comment:
   May these constructors have an `exclusive` field?



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org