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

 ##########
 File path: 
flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/PubSubSource.java
 ##########
 @@ -0,0 +1,298 @@
+/*
+ * 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.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.runtime.state.CheckpointListener;
+import org.apache.flink.streaming.api.checkpoint.ListCheckpointed;
+import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
+import 
org.apache.flink.streaming.connectors.gcp.pubsub.common.AcknowledgeIdsForCheckpoint;
+import 
org.apache.flink.streaming.connectors.gcp.pubsub.common.AcknowledgeOnCheckpoint;
+import org.apache.flink.streaming.connectors.gcp.pubsub.common.Acknowledger;
+import 
org.apache.flink.streaming.connectors.gcp.pubsub.common.PubSubDeserializationSchema;
+import 
org.apache.flink.streaming.connectors.gcp.pubsub.common.PubSubSubscriber;
+import 
org.apache.flink.streaming.connectors.gcp.pubsub.common.PubSubSubscriberFactory;
+import org.apache.flink.util.Preconditions;
+
+import com.google.auth.Credentials;
+import com.google.cloud.pubsub.v1.Subscriber;
+import com.google.pubsub.v1.ProjectSubscriptionName;
+import com.google.pubsub.v1.PubsubMessage;
+import com.google.pubsub.v1.ReceivedMessage;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.List;
+import java.util.concurrent.CancellationException;
+
+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 RichSourceFunction<OUT>
+       implements Acknowledger<String>, ResultTypeQueryable<OUT>, 
ParallelSourceFunction<OUT>, CheckpointListener, 
ListCheckpointed<AcknowledgeIdsForCheckpoint<String>> {
+       public static final int NO_MAX_MESSAGES_TO_ACKNOWLEDGE_LIMIT = -1;
+       private static final Logger LOG = 
LoggerFactory.getLogger(PubSubSource.class);
+       protected final PubSubDeserializationSchema<OUT> deserializationSchema;
+       protected final PubSubSubscriberFactory pubSubSubscriberFactory;
+       protected final Credentials credentials;
+       protected final int maxMessagesToAcknowledge;
+       protected final AcknowledgeOnCheckpointFactory 
acknowledgeOnCheckpointFactory;
+
+       protected transient AcknowledgeOnCheckpoint<String> 
acknowledgeOnCheckpoint;
+       protected transient PubSubSubscriber subscriber;
+
+       protected transient volatile boolean isRunning;
+
+       PubSubSource(PubSubDeserializationSchema<OUT> deserializationSchema,
+                               PubSubSubscriberFactory pubSubSubscriberFactory,
+                               Credentials credentials,
+                               int maxMessagesToAcknowledge,
+                               AcknowledgeOnCheckpointFactory 
acknowledgeOnCheckpointFactory) {
+               this.deserializationSchema = deserializationSchema;
+               this.pubSubSubscriberFactory = pubSubSubscriberFactory;
+               this.credentials = credentials;
+               this.maxMessagesToAcknowledge = maxMessagesToAcknowledge;
+               this.acknowledgeOnCheckpointFactory = 
acknowledgeOnCheckpointFactory;
+       }
+
+       @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);
+
+               createAndSetAcknowledgeOnCheckpoint();
+               this.subscriber = 
pubSubSubscriberFactory.getSubscriber(credentials);
+               this.isRunning = true;
+       }
+
+       private boolean hasNoCheckpointingEnabled(RuntimeContext 
runtimeContext) {
+               return !(runtimeContext instanceof StreamingRuntimeContext && 
((StreamingRuntimeContext) runtimeContext).isCheckpointingEnabled());
+       }
+
+       @Override
+       public void acknowledge(List<String> acknowledgementIds) {
+               if (!isRunning) {
+                       return;
+               }
+
+               subscriber.acknowledge(acknowledgementIds);
+       }
+
+       @Override
+       public void run(SourceContext<OUT> sourceContext) throws Exception {
+               while (isRunning) {
+                       try {
+                               blockIfMaxMessagesToAcknowledgeLimitReached();
+
+                               processMessage(sourceContext, 
subscriber.pull());
+                       } catch (InterruptedException | CancellationException 
e) {
+                               isRunning = false;
+                       }
+               }
+               subscriber.close();
+       }
+
+       void processMessage(SourceContext<OUT> sourceContext, 
List<ReceivedMessage> messages) throws Exception {
+               synchronized (sourceContext.getCheckpointLock()) {
+                       for (ReceivedMessage message : messages) {
+                               
acknowledgeOnCheckpoint.addAcknowledgeId(message.getAckId());
+
+                               PubsubMessage pubsubMessage = 
message.getMessage();
+
+                               OUT deserializedMessage = 
deserializationSchema.deserialize(pubsubMessage);
+                               if 
(deserializationSchema.isEndOfStream(deserializedMessage)) {
+                                       cancel();
+                                       return;
+                               }
+
+                               sourceContext.collect(deserializedMessage);
+                       }
+
+               }
+       }
+
+       private void blockIfMaxMessagesToAcknowledgeLimitReached() throws 
Exception {
+               while (maxMessagesToAcknowledge != 
NO_MAX_MESSAGES_TO_ACKNOWLEDGE_LIMIT && getOutstandingMessagesToAck() > 
maxMessagesToAcknowledge) {
+                       LOG.debug("Sleeping because there are {} messages 
waiting to be ack'ed but limit is {}", getOutstandingMessagesToAck(), 
maxMessagesToAcknowledge);
+                       Thread.sleep(100);
+               }
+       }
+
+       private Integer getOutstandingMessagesToAck() {
+               return 
acknowledgeOnCheckpoint.numberOfOutstandingAcknowledgements();
+       }
+
+       @Override
+       public void cancel() {
+               isRunning = false;
+       }
+
+       @Override
+       public TypeInformation<OUT> getProducedType() {
+               return deserializationSchema.getProducedType();
+       }
+
+       public static <OUT> PubSubSourceBuilder<OUT> 
newBuilder(DeserializationSchema<OUT> deserializationSchema,
 
 Review comment:
   These two public methods to get builders are not really a builder pattern. 
See the comments for the `PubSubSink`

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