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_r300069515
 
 

 ##########
 File path: 
flink-connectors/flink-connector-gcp-pubsub/src/main/java/org/apache/flink/streaming/connectors/gcp/pubsub/BlockingGrpcPubSubSubscriber.java
 ##########
 @@ -0,0 +1,133 @@
+/*
+ * 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.java.tuple.Tuple2;
+import 
org.apache.flink.streaming.connectors.gcp.pubsub.common.PubSubSubscriber;
+
+import com.google.pubsub.v1.AcknowledgeRequest;
+import com.google.pubsub.v1.PullRequest;
+import com.google.pubsub.v1.ReceivedMessage;
+import com.google.pubsub.v1.SubscriberGrpc;
+import io.grpc.ManagedChannel;
+import io.grpc.StatusRuntimeException;
+
+import java.time.Duration;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import static java.util.Collections.emptyList;
+import static java.util.concurrent.TimeUnit.SECONDS;
+
+/**
+ * Implementation for {@link PubSubSubscriber}.
+ * This Grpc PubSubSubscriber allows for flexible timeouts and retries.
+ */
+public class BlockingGrpcPubSubSubscriber implements PubSubSubscriber {
+       private final String projectSubscriptionName;
+       private final ManagedChannel channel;
+       private final SubscriberGrpc.SubscriberBlockingStub stub;
+       private final int retries;
+       private final Duration timeout;
+       private final PullRequest pullRequest;
+
+       public BlockingGrpcPubSubSubscriber(String projectSubscriptionName, 
ManagedChannel channel, SubscriberGrpc.SubscriberBlockingStub stub, PullRequest 
pullRequest, int retries, Duration timeout) {
+               this.projectSubscriptionName = projectSubscriptionName;
+               this.channel = channel;
+               this.stub = stub;
+               this.retries = retries;
+               this.timeout = timeout;
+               this.pullRequest = pullRequest;
+       }
+
+       @Override
+       public List<ReceivedMessage> pull() {
+               return pull(retries);
+       }
+
+       private List<ReceivedMessage> pull(int retriesRemaining) {
+               try {
+                       return stub.withDeadlineAfter(timeout.toMillis(), 
TimeUnit.MILLISECONDS)
+                                       .pull(pullRequest)
+                                       .getReceivedMessagesList();
+               } catch (StatusRuntimeException e) {
+                       if (retriesRemaining > 0) {
+                               return pull(retriesRemaining - 1);
+                       }
+
+                       throw e;
+               }
+       }
+
+       @Override
+       public void acknowledge(List<String> acknowledgementIds) {
+               if (acknowledgementIds.isEmpty()) {
+                       return;
+               }
+
+               //grpc servers won't accept acknowledge requests that are too 
large so we split the ackIds
+               Tuple2<List<String>, List<String>> splittedAckIds = 
splitAckIds(acknowledgementIds);
+               while (!splittedAckIds.f0.isEmpty()) {
+                       AcknowledgeRequest acknowledgeRequest =
+                                       AcknowledgeRequest.newBuilder()
+                                                                       
.setSubscription(projectSubscriptionName)
+                                                                       
.addAllAckIds(splittedAckIds.f0)
+                                                                       
.build();
+
+                       stub.withDeadlineAfter(60, 
SECONDS).acknowledge(acknowledgeRequest);
 
 Review comment:
   Well, the `timeout` value is set alongside the number of retries. So, for 
instance, in my project we set it to 5 seconds with 12 retries, resulting in 60 
seconds total. If we were to use timeout for acknowledgements it would go from 
60s to 5s without retries which might be unexpected.
   
   I think we should add retries to the acknowledgment and then (re)use this 
timeout value. I've created a new jira issue and added this. Shall we leave it 
at 60 for now?

----------------------------------------------------------------
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:
[email protected]


With regards,
Apache Git Services

Reply via email to