[ https://issues.apache.org/jira/browse/FLINK-9311?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16683894#comment-16683894 ]
ASF GitHub Bot commented on FLINK-9311: --------------------------------------- nielsbasjes 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_r232687817 ########## File path: flink-connectors/flink-connector-pubsub/src/main/java/org/apache/flink/streaming/connectors/pubsub/Bound.java ########## @@ -0,0 +1,130 @@ +/* + * 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.api.functions.source.SourceFunction; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Serializable; +import java.util.Timer; +import java.util.TimerTask; + +/** + * This class defines a bound based on messages received or time since last received message. + * Using start(SourceFunction) starts the bound. Everytime a message is received the sourceFunction should call receivedMessage(). + * When the bound is reached, the sourcefunction gets closed by calling sourceFunction.close() + * See {@link BoundedPubSubSource}. + * + * @param <OUT> type of message that is received by the SourceFunction. + */ +class Bound<OUT> implements Serializable { + private static final Logger LOG = LoggerFactory.getLogger(Bound.class); + + private final Bound.Mode mode; + private final long maxMessagedReceived; + private final long maxTimeBetweenMessages; + + private SourceFunction<OUT> sourceFunction; + private transient Timer timer; + private long messagesReceived; + private long lastReceivedMessage; + private boolean cancelled = false; + + private Bound(Bound.Mode mode, long maxMessagedReceived, long maxTimeBetweenMessages) { + this.mode = mode; + this.maxMessagedReceived = maxMessagedReceived; + this.maxTimeBetweenMessages = maxTimeBetweenMessages; + this.messagesReceived = 0L; + } + + static <OUT> Bound<OUT> boundByAmountOfMessages(long maxMessagedReceived) { + return new Bound<>(Mode.COUNTER, maxMessagedReceived, 0L); + } + + static <OUT> Bound<OUT> boundByTimeSinceLastMessage(long maxTimeBetweenMessages) { + return new Bound<>(Mode.TIMER, 0L, maxTimeBetweenMessages); + } + + static <OUT> Bound<OUT> boundByAmountOfMessagesOrTimeSinceLastMessage(long maxMessagedReceived, long maxTimeBetweenMessages) { + return new Bound<>(Mode.COUNTER_OR_TIMER, maxMessagedReceived, maxTimeBetweenMessages); + } + + private TimerTask shutdownPubSubSource() { + return new TimerTask() { + @Override + public void run() { + if (maxTimeBetweenMessagesElapsed()) { + cancelPubSubSource("BoundedSourceFunction: Idle timeout --> canceling source"); + timer.cancel(); + } + } + }; + } + + private synchronized boolean maxTimeBetweenMessagesElapsed() { + return System.currentTimeMillis() - lastReceivedMessage > maxTimeBetweenMessages; + } + + private synchronized void cancelPubSubSource(String logMessage) { + if (!cancelled) { + cancelled = true; + sourceFunction.cancel(); + LOG.info(logMessage); + } + } + + void start(SourceFunction<OUT> sourceFunction) { + if (this.sourceFunction != null) { + throw new IllegalStateException("start() already called"); + } + + this.sourceFunction = sourceFunction; + messagesReceived = 0; + + if (mode == Mode.TIMER || mode == Mode.COUNTER_OR_TIMER) { + lastReceivedMessage = System.currentTimeMillis(); + timer = new Timer(); + timer.schedule(shutdownPubSubSource(), 0, 100); + } + } + + synchronized void receivedMessage() { + if (sourceFunction == null) { + throw new IllegalStateException("start() not called"); + } + + lastReceivedMessage = System.currentTimeMillis(); + messagesReceived++; + + if ((mode == Mode.COUNTER || mode == Mode.COUNTER_OR_TIMER) && messagesReceived >= maxMessagedReceived) { + cancelPubSubSource("BoundedSourceFunction: Max received messages --> canceling source"); + } + } + + /** + * There are 3 types of Bounds. + * COUNTER - A maximum amount of received messages per SourceFunction + * TIMER - A maximum amount of idle time between messages per SourceFunction + * COUNTER_OR_TIMER - Maximum amount of messages OR maximum amount of idle time. + */ + private enum Mode { + COUNTER, TIMER, COUNTER_OR_TIMER Review comment: I do not see any immediate situations on how to extend the idea of the Bound within the PubSub connector. The Bound is package private and this enum is even private within the class. So we could make this much more generic but I think for now this implementation is fine. I do see value in (later) making this idea of Bounding reusable by all Source implementations. At that time it should be moved out of the PubSub module and into a more generic place. I prefer to do this later. ---------------------------------------------------------------- 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 > PubSub connector > ---------------- > > Key: FLINK-9311 > URL: https://issues.apache.org/jira/browse/FLINK-9311 > Project: Flink > Issue Type: New Feature > Components: Streaming Connectors > Reporter: Richard Deurwaarder > Assignee: Niels Basjes > Priority: Minor > Labels: pull-request-available > > I would like start adding some google cloud connectors starting with a PubSub > Source. I have a basic implementation ready but I want it to be able to: > * easily scale up (should I have it extend RichParallelSourceFunction for > this?) > * Make it easier to provide the google cloud credentials. This would require > being able to send some json string / ServiceAccount to the nodes when > starting up this source. > CouldĀ this be something that would be useful for others and added to the > flink connectors repo? -- This message was sent by Atlassian JIRA (v7.6.3#76005)