Xeli commented on a change in pull request #6594: [FLINK-9311] [pubsub] Added 
PubSub source connector with support for checkpointing (ATLEAST_ONCE)
URL: https://github.com/apache/flink/pull/6594#discussion_r271059377
 
 

 ##########
 File path: 
flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/PubSubSource.java
 ##########
 @@ -0,0 +1,296 @@
+/*
+ * 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.streaming.connectors.gcp.pubsub;
+
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.functions.StoppableFunction;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.configuration.Configuration;
+import 
org.apache.flink.streaming.api.functions.source.MultipleIdsMessageAcknowledgingSourceBase;
+import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction;
+import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
+import 
org.apache.flink.streaming.connectors.gcp.pubsub.common.PubSubSubscriberFactory;
+import org.apache.flink.util.Preconditions;
+
+import com.google.api.core.ApiFuture;
+import com.google.auth.Credentials;
+import com.google.cloud.pubsub.v1.Subscriber;
+import com.google.cloud.pubsub.v1.stub.SubscriberStub;
+import com.google.pubsub.v1.AcknowledgeRequest;
+import com.google.pubsub.v1.ProjectSubscriptionName;
+import com.google.pubsub.v1.PubsubMessage;
+import com.google.pubsub.v1.PullRequest;
+import com.google.pubsub.v1.PullResponse;
+import com.google.pubsub.v1.ReceivedMessage;
+import io.grpc.netty.shaded.io.netty.channel.EventLoopGroup;
+import io.grpc.netty.shaded.io.netty.channel.nio.NioEventLoopGroup;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.TimeUnit;
+
+import static 
com.google.cloud.pubsub.v1.SubscriptionAdminSettings.defaultCredentialsProviderBuilder;
+
+/**
+ * PubSub Source, this Source will consume PubSub messages from a subscription 
and Acknowledge them on the next checkpoint.
+ * This ensures every message will get acknowledged at least once.
+ */
+public class PubSubSource<OUT> extends 
MultipleIdsMessageAcknowledgingSourceBase<OUT, String, String>
+               implements ResultTypeQueryable<OUT>, 
ParallelSourceFunction<OUT>, StoppableFunction {
+       private static final Logger LOG = 
LoggerFactory.getLogger(PubSubSource.class);
+       protected final DeserializationSchema<OUT> deserializationSchema;
+       protected final PubSubSubscriberFactory pubSubSubscriberFactory;
+       protected final Credentials credentials;
+       protected final String projectSubscriptionName;
+       protected final int maxMessagesPerPull;
+
+       protected transient boolean deduplicateMessages;
+       protected transient SubscriberStub subscriber;
+       protected transient PullRequest pullRequest;
+       protected transient EventLoopGroup eventLoopGroup;
+
+       protected transient volatile boolean isRunning;
+       protected transient volatile ApiFuture<PullResponse> messagesFuture;
+
+       PubSubSource(DeserializationSchema<OUT> deserializationSchema, 
PubSubSubscriberFactory pubSubSubscriberFactory, Credentials credentials, 
String projectSubscriptionName, int maxMessagesPerPull) {
+               super(String.class);
+               this.deserializationSchema = deserializationSchema;
+               this.pubSubSubscriberFactory = pubSubSubscriberFactory;
+               this.credentials = credentials;
+               this.projectSubscriptionName = projectSubscriptionName;
+               this.maxMessagesPerPull = maxMessagesPerPull;
+       }
+
+       @Override
+       public void open(Configuration configuration) throws Exception {
+               super.open(configuration);
+               if (hasNoCheckpointingEnabled(getRuntimeContext())) {
+                       throw new IllegalArgumentException("The PubSubSource 
REQUIRES Checkpointing to be enabled and " +
+                               "the checkpointing frequency must be MUCH lower 
than the PubSub timeout for it to retry a message.");
+               }
+
+               
getRuntimeContext().getMetricGroup().gauge("PubSubMessagesProcessedNotAcked", 
this::getOutstandingMessagesToAck);
+
+               this.eventLoopGroup = new NioEventLoopGroup();
+               this.subscriber = 
pubSubSubscriberFactory.getSubscriber(eventLoopGroup, credentials);
+               this.deduplicateMessages = 
getRuntimeContext().getNumberOfParallelSubtasks() == 1;
+               this.isRunning = true;
+               this.pullRequest = PullRequest.newBuilder()
+                                                                               
.setMaxMessages(100)
+                                                                               
.setReturnImmediately(false)
+                                                                               
.setSubscription(projectSubscriptionName)
+                                                                               
.build();
+       }
+
+       private boolean hasNoCheckpointingEnabled(RuntimeContext 
runtimeContext) {
+               return !(runtimeContext instanceof StreamingRuntimeContext && 
((StreamingRuntimeContext) runtimeContext).isCheckpointingEnabled());
+       }
+
+       @Override
+       protected void acknowledgeSessionIDs(List<String> acknowledgementIds) {
+               if (acknowledgementIds.isEmpty() || !isRunning) {
+                       return;
+               }
+
+               AcknowledgeRequest acknowledgeRequest =
+                       AcknowledgeRequest.newBuilder()
+                                                               
.setSubscription(projectSubscriptionName)
+                                                               
.addAllAckIds(acknowledgementIds)
+                                                               .build();
+               subscriber.acknowledgeCallable().call(acknowledgeRequest);
+       }
+
+       @Override
+       public void run(SourceContext<OUT> sourceContext) throws Exception {
+               while (isRunning) {
+                       messagesFuture = 
subscriber.pullCallable().futureCall(pullRequest);
+                       try {
+                               processMessage(sourceContext, 
messagesFuture.get().getReceivedMessagesList());
+                       } catch (InterruptedException | CancellationException 
e) {
+                               awaitSubscriberTermination();
+                               return;
+                       }
+               }
+
+               awaitSubscriberTermination();
+       }
+
+       void processMessage(SourceContext<OUT> sourceContext, 
List<ReceivedMessage> messages) throws IOException {
+               synchronized (sourceContext.getCheckpointLock()) {
+                       for (ReceivedMessage message : messages) {
+                               sessionIds.add(message.getAckId());
+
+                               PubsubMessage pubsubMessage = 
message.getMessage();
+                               if (deduplicateMessages && 
!addId(pubsubMessage.getMessageId())) {
+                                       // message is duplicate so just ignore 
it
+                                       return;
+                               }
+
+                               OUT deserializedMessage = 
deserializeMessage(pubsubMessage);
+                               if 
(deserializationSchema.isEndOfStream(deserializedMessage)) {
+                                       stop();
+                                       return;
+                               }
+
+                               sourceContext.collect(deserializedMessage);
+                       }
+
+               }
+       }
+
+       private Integer getOutstandingMessagesToAck() {
+               return this.sessionIdsPerSnapshot
+                               .stream()
+                               .mapToInt(tuple -> tuple.f1.size())
+                               .sum() + this.sessionIds.size();
+       }
+
+       @Override
+       public void cancel() {
+               stop();
+       }
+
+       @Override
+       public void stop() {
 
 Review comment:
   I've removed the StoppableFunction interface as I noticed it might get 
deprecated soonish in FLINK-11889 and have changed the `cancel()` to just set a 
boolean to false as per your suggestion.
   
   Could you have another look?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

Reply via email to