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_r257502453
########## File path: flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/DefaultPubSubSubscriberFactory.java ########## @@ -0,0 +1,67 @@ +/* + * 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.streaming.connectors.pubsub.common.PubSubSubscriberFactory; + +import com.google.api.gax.batching.FlowControlSettings; +import com.google.api.gax.batching.FlowController; +import com.google.api.gax.core.FixedCredentialsProvider; +import com.google.api.gax.grpc.GrpcTransportChannel; +import com.google.api.gax.rpc.FixedTransportChannelProvider; +import com.google.api.gax.rpc.TransportChannel; +import com.google.auth.Credentials; +import com.google.cloud.pubsub.v1.MessageReceiver; +import com.google.cloud.pubsub.v1.Subscriber; +import com.google.pubsub.v1.ProjectSubscriptionName; +import io.grpc.ManagedChannel; +import io.grpc.ManagedChannelBuilder; + +final class DefaultPubSubSubscriberFactory implements PubSubSubscriberFactory { + private String hostAndPort; + + void withHostAndPort(String hostAndPort) { + this.hostAndPort = hostAndPort; + } + + @Override + public Subscriber getSubscriber(Credentials credentials, ProjectSubscriptionName projectSubscriptionName, MessageReceiver messageReceiver) { + FlowControlSettings flowControlSettings = FlowControlSettings.newBuilder() + .setMaxOutstandingElementCount(10000L) + .setMaxOutstandingRequestBytes(100000L) Review comment: In my own applications I change/tune 3 parameters: - setMaxOutstandingElementCount - setMaxOutstandingRequestBytes - setMaxAckExtensionPeriod The first two go hand in hand. They are what you would tune when dealing with backpressure (see the flink docs pubsub section I added for a bit of explanation). The third indicates how long a message is kept unacknowledged. After this time PubSub might start sending the message to other consumers. This is relevant when the time between checkpoints is high. My thought was to not expose this through the Source builder because it's kinda tricky to properly explain what they do, while in the context of a `Subscriber` the google docs speak for themselves. I do see your point though: it would increase usability a lot if people can just tune the number a bit without having to go through the hassle of figuring out how to create a `Subscriber`. So I now added: /** * Tune how many messages the connector will buffer when the Flink pipeline cannot keep up (backpressure). * * @param maxMessagesReceivedNotProcessed This indicates how many messages will be read and buffered until when the flink pipeline can't handle the messages fast enough. * @param maxBytesReceivedNotProcessed This indicates how many bytes will be read and buffered. A good pick would be: maxBytesReceivedNotProcessed = maxMessagesReceivedNotProcessed * averageBytesSizePerMessage */ public PubSubSourceBuilder<OUT> withBackpressureParameters(long maxMessagesReceivedNotProcessed, long maxBytesReceivedNotProcessed) { this.maxMessagesReceivedNotProcessed = maxMessagesReceivedNotProcessed; this.maxBytesReceivedNotProcessed = maxBytesReceivedNotProcessed; return this; } Do you think this is clear enough? ---------------------------------------------------------------- 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