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_r257501427
########## File path: flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/PubSubSource.java ########## @@ -0,0 +1,294 @@ +/* + * 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.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.tuple.Tuple2; +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.pubsub.common.PubSubSubscriberFactory; + +import com.google.auth.Credentials; +import com.google.cloud.pubsub.v1.AckReplyConsumer; +import com.google.cloud.pubsub.v1.Subscriber; +import com.google.pubsub.v1.ProjectSubscriptionName; +import com.google.pubsub.v1.PubsubMessage; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.List; +import java.util.Optional; + +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, AckReplyConsumer> + implements ResultTypeQueryable<OUT>, ParallelSourceFunction<OUT>, StoppableFunction { + private static final Logger LOG = LoggerFactory.getLogger(PubSubSource.class); + protected DeserializationSchema<OUT> deserializationSchema; + protected SubscriberWrapper subscriberWrapper; + + protected boolean running = true; + protected transient volatile SourceContext<OUT> sourceContext = null; + + protected PubSubSource() { + super(String.class); + } + + protected void setDeserializationSchema(DeserializationSchema<OUT> deserializationSchema) { + this.deserializationSchema = deserializationSchema; + } + + protected void setSubscriberWrapper(SubscriberWrapper subscriberWrapper) { + this.subscriberWrapper = subscriberWrapper; + } + + @Override + public void open(Configuration configuration) throws Exception { + super.open(configuration); + subscriberWrapper.initialize(); + 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); + getRuntimeContext().getMetricGroup().gauge("PubSubMessagesReceivedNotProcessed", subscriberWrapper::amountOfMessagesInBuffer); + } + + private boolean hasNoCheckpointingEnabled(RuntimeContext runtimeContext) { + return !(runtimeContext instanceof StreamingRuntimeContext && ((StreamingRuntimeContext) runtimeContext).isCheckpointingEnabled()); + } + + @Override + protected void acknowledgeSessionIDs(List<AckReplyConsumer> ackReplyConsumers) { + ackReplyConsumers.forEach(AckReplyConsumer::ack); + } + + @Override + public void run(SourceContext<OUT> sourceContext) throws Exception { Review comment: It now gracefully shuts down when isEndOfStream() returns true. I wonder though, how does this work for parallel instances of this source function? Does flink make sure once one of them calls stop() or cancel() the others get shut down as well? On your second comment: I'm a bit on the fence on this one. `run()` would become quite large and with lines being nested quite deep (some lines would be indented/nested 5 times). Also the `processMessage()` proved quite useful to make the unit tests a bit easier too read, this prevented me having to create threads and such in the unit test like the RabbitMQ unit tests have. ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on 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