You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2020/10/13 09:54:32 UTC

[GitHub] [flink] kl0u commented on a change in pull request #13596: [FLINK-19475] Implement a time service for the batch execution mode

kl0u commented on a change in pull request #13596:
URL: https://github.com/apache/flink/pull/13596#discussion_r503813006



##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java
##########
@@ -174,6 +176,14 @@ public StateBackend getStateBackend() {
 		return this.stateBackend;
 	}
 
+	public InternalTimeServiceManager.Provider getTimerServiceProvider() {
+		return timerServiceProvider;
+	}
+
+	public void setTimerServiceProvider(InternalTimeServiceManager.Provider timerServiceProvider) {
+		this.timerServiceProvider = timerServiceProvider;

Review comment:
       I would add a `checkNotNull()`.

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/sorted/state/BatchExecutionInternalTimeServiceManager.java
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.api.operators.sorted.state;
+
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.runtime.state.CheckpointableKeyedStateBackend;
+import org.apache.flink.runtime.state.KeyGroupStatePartitionStreamProvider;
+import org.apache.flink.runtime.state.KeyedStateBackend;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.operators.InternalTimeServiceManager;
+import org.apache.flink.streaming.api.operators.InternalTimerService;
+import org.apache.flink.streaming.api.operators.KeyContext;
+import org.apache.flink.streaming.api.operators.Triggerable;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
+import org.apache.flink.util.WrappingRuntimeException;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * An implementation of a {@link InternalTimeServiceManager} that manages timers with a single active key at a time.
+ * Can be used in a BATCH execution mode.
+ */
+public class BatchExecutionInternalTimeServiceManager<K> implements InternalTimeServiceManager<K>,
+		KeyedStateBackend.KeySelectionListener<K> {
+
+	private final ProcessingTimeService processingTimeService;
+	private final Map<String, BatchExecutionInternalTimeService<K, ?>> timerServices = new HashMap<>();
+
+	public BatchExecutionInternalTimeServiceManager(ProcessingTimeService processingTimeService) {
+		this.processingTimeService = processingTimeService;
+	}
+
+	@Override
+	@SuppressWarnings("unchecked")
+	public <N> InternalTimerService<N> getInternalTimerService(
+			String name,
+			TypeSerializer<K> keySerializer,
+			TypeSerializer<N> namespaceSerializer,
+			Triggerable<K, N> triggerable) {
+		BatchExecutionInternalTimeService<K, N> timerService =
+			(BatchExecutionInternalTimeService<K, N>) timerServices.get(name);
+		if (timerService == null) {
+			timerService = new BatchExecutionInternalTimeService<>(
+				processingTimeService,
+				triggerable
+			);
+			timerServices.put(name, timerService);
+		}
+
+		return timerService;
+	}
+
+	@Override
+	public void advanceWatermark(Watermark watermark) {
+		if (watermark.getTimestamp() == Long.MAX_VALUE) {
+			keySelected(null);
+		}
+	}
+
+	@Override
+	public void snapshotState(
+			StateSnapshotContext context,
+			String operatorName) throws Exception {
+		throw new UnsupportedOperationException("Checkpoints are not supported in BATCH execution");
+	}
+
+	public static <K> InternalTimeServiceManager<K> create(
+			CheckpointableKeyedStateBackend<K> keyedStatedBackend,
+			ClassLoader userClassloader,
+			KeyContext keyContext, //the operator
+			ProcessingTimeService processingTimeService,
+			Iterable<KeyGroupStatePartitionStreamProvider> rawKeyedStates) {
+		if (!(keyedStatedBackend instanceof BatchExecutionKeyedStateBackend)) {

Review comment:
       This could be a `checkState` right?

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/sorted/state/BatchExecutionInternalTimeService.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.api.operators.sorted.state;
+
+import org.apache.flink.runtime.state.KeyGroupedInternalPriorityQueue;
+import org.apache.flink.runtime.state.PriorityComparator;
+import org.apache.flink.streaming.api.operators.InternalTimer;
+import org.apache.flink.streaming.api.operators.InternalTimerService;
+import org.apache.flink.streaming.api.operators.TimerHeapInternalTimer;
+import org.apache.flink.streaming.api.operators.Triggerable;
+import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
+import org.apache.flink.util.function.BiConsumerWithException;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * An implementation of a {@link InternalTimerService} that manages timers with a single active key at a time.
+ * Can be used in a BATCH execution mode.
+ */
+public class BatchExecutionInternalTimeService<K, N> implements InternalTimerService<N> {
+
+	private final ProcessingTimeService processingTimeService;
+
+	/**
+	 * Processing time timers that are currently in-flight.
+	 */
+	private final KeyGroupedInternalPriorityQueue<TimerHeapInternalTimer<K, N>> processingTimeTimersQueue;
+
+	/**
+	 * Event time timers that are currently in-flight.
+	 */
+	private final KeyGroupedInternalPriorityQueue<TimerHeapInternalTimer<K, N>> eventTimeTimersQueue;
+
+	/**
+	 * The local event time, as denoted by the last received
+	 * {@link org.apache.flink.streaming.api.watermark.Watermark Watermark}.
+	 */
+	private long currentWatermark = Long.MIN_VALUE;
+
+	private final Triggerable<K, N> triggerTarget;
+
+	private K currentKey;
+
+	BatchExecutionInternalTimeService(
+			ProcessingTimeService processingTimeService,
+			Triggerable<K, N> triggerTarget) {
+
+		this.processingTimeService = checkNotNull(processingTimeService);
+		this.triggerTarget = checkNotNull(triggerTarget);
+
+		this.processingTimeTimersQueue = new BatchExecutionInternalPriorityQueueSet<>(
+			PriorityComparator.forPriorityComparableObjects(),
+			128
+		);
+		this.eventTimeTimersQueue = new BatchExecutionInternalPriorityQueueSet<>(
+			PriorityComparator.forPriorityComparableObjects(),
+			128
+		);
+	}
+
+	@Override
+	public long currentProcessingTime() {
+		return processingTimeService.getCurrentProcessingTime();
+	}
+
+	@Override
+	public long currentWatermark() {
+		return currentWatermark;
+	}
+
+	@Override
+	public void registerProcessingTimeTimer(N namespace, long time) {
+		processingTimeTimersQueue.add(new TimerHeapInternalTimer<>(time, currentKey, namespace));
+	}
+
+	@Override
+	public void registerEventTimeTimer(N namespace, long time) {
+		eventTimeTimersQueue.add(new TimerHeapInternalTimer<>(time, currentKey, namespace));
+	}
+
+	@Override
+	public void deleteProcessingTimeTimer(N namespace, long time) {
+		processingTimeTimersQueue.remove(new TimerHeapInternalTimer<>(time, currentKey, namespace));
+	}
+
+	@Override
+	public void deleteEventTimeTimer(N namespace, long time) {
+		eventTimeTimersQueue.remove(new TimerHeapInternalTimer<>(time, currentKey, namespace));
+	}
+
+	@Override
+	public void forEachEventTimeTimer(BiConsumerWithException<N, Long, Exception> consumer) {
+		throw new UnsupportedOperationException(
+			"The BatchExecutionInternalTimeService should not be used in State Processor API.");
+	}
+
+	@Override
+	public void forEachProcessingTimeTimer(BiConsumerWithException<N, Long, Exception> consumer) {
+		throw new UnsupportedOperationException(
+			"The BatchExecutionInternalTimeService should not be used in State Processor API.");
+	}
+
+	public void setCurrentKey(K currentKey) throws Exception {

Review comment:
       Had the same comment as @aljoscha plus, shouldn't we here prevent the user-code from registering any new timers? If not, it looks like we risk ending up in a loop where every fired timer will register a new one, right?
   We can simply add in the register timer `if(watermark ==Long.MAX) -> ignore` or sth like that. Same for processing time (and I think it is ok to check again the watermark).
   
   In addition, we are relying on each key being processed in its entirety and then move to the next, which is totally valid. In this case, I think we should somehow disable the `DataStreamUtils.reinterpretAsKeyedStream()` for batch because, if I am not mistaken, it messes things up, right? Maybe at another ticket @dawidwys ?

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/sorted/state/BatchExecutionInternalTimeServiceManager.java
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.api.operators.sorted.state;
+
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.runtime.state.CheckpointableKeyedStateBackend;
+import org.apache.flink.runtime.state.KeyGroupStatePartitionStreamProvider;
+import org.apache.flink.runtime.state.KeyedStateBackend;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.operators.InternalTimeServiceManager;
+import org.apache.flink.streaming.api.operators.InternalTimerService;
+import org.apache.flink.streaming.api.operators.KeyContext;
+import org.apache.flink.streaming.api.operators.Triggerable;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
+import org.apache.flink.util.WrappingRuntimeException;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * An implementation of a {@link InternalTimeServiceManager} that manages timers with a single active key at a time.
+ * Can be used in a BATCH execution mode.
+ */
+public class BatchExecutionInternalTimeServiceManager<K> implements InternalTimeServiceManager<K>,
+		KeyedStateBackend.KeySelectionListener<K> {
+
+	private final ProcessingTimeService processingTimeService;
+	private final Map<String, BatchExecutionInternalTimeService<K, ?>> timerServices = new HashMap<>();
+
+	public BatchExecutionInternalTimeServiceManager(ProcessingTimeService processingTimeService) {
+		this.processingTimeService = processingTimeService;

Review comment:
       What about a `checkNotNull`?




----------------------------------------------------------------
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:
users@infra.apache.org