Skip to content

Commit

Permalink
[FLINK-37206][Runtime] Fix initialization of batching timer service i…
Browse files Browse the repository at this point in the history
…n async state operators
  • Loading branch information
Zakelly committed Jan 24, 2025
1 parent 9912774 commit d215b61
Show file tree
Hide file tree
Showing 7 changed files with 507 additions and 21 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,7 @@
import org.apache.flink.streaming.api.operators.InternalTimerServiceAsyncImpl;
import org.apache.flink.streaming.api.operators.Triggerable;
import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
import org.apache.flink.streaming.api.operators.sorted.state.BatchExecutionInternalTimeServiceWithAsyncState;
import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.runtime.operators.asyncprocessing.AsyncStateProcessing;
import org.apache.flink.streaming.runtime.operators.asyncprocessing.AsyncStateProcessingOperator;
Expand Down Expand Up @@ -307,7 +308,12 @@ public <K, N> InternalTimerService<N> getInternalTimerService(
InternalTimerService<N> service =
keyedTimeServiceHandler.getInternalTimerService(
name, keySerializer, namespaceSerializer, triggerable);
((InternalTimerServiceAsyncImpl<K, N>) service).setup(asyncExecutionController);
if (service instanceof InternalTimerServiceAsyncImpl) {
((InternalTimerServiceAsyncImpl<K, N>) service).setup(asyncExecutionController);
} else if (service instanceof BatchExecutionInternalTimeServiceWithAsyncState) {
((BatchExecutionInternalTimeServiceWithAsyncState<K, N>) service)
.setup(asyncExecutionController);
}
return service;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@
import org.apache.flink.streaming.api.operators.InternalTimerServiceAsyncImpl;
import org.apache.flink.streaming.api.operators.StreamOperatorParameters;
import org.apache.flink.streaming.api.operators.Triggerable;
import org.apache.flink.streaming.api.operators.sorted.state.BatchExecutionInternalTimeServiceWithAsyncState;
import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.runtime.operators.asyncprocessing.AsyncStateProcessing;
import org.apache.flink.streaming.runtime.operators.asyncprocessing.AsyncStateProcessingOperator;
Expand Down Expand Up @@ -286,7 +287,12 @@ public <K, N> InternalTimerService<N> getInternalTimerService(
InternalTimerService<N> service =
keyedTimeServiceHandler.getInternalTimerService(
name, keySerializer, namespaceSerializer, triggerable);
((InternalTimerServiceAsyncImpl<K, N>) service).setup(asyncExecutionController);
if (service instanceof InternalTimerServiceAsyncImpl) {
((InternalTimerServiceAsyncImpl<K, N>) service).setup(asyncExecutionController);
} else if (service instanceof BatchExecutionInternalTimeServiceWithAsyncState) {
((BatchExecutionInternalTimeServiceWithAsyncState<K, N>) service)
.setup(asyncExecutionController);
}
return service;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@

package org.apache.flink.runtime.state.v2.adaptor;

import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.api.common.state.CheckpointListener;
import org.apache.flink.api.common.state.InternalCheckpointListener;
import org.apache.flink.api.common.state.v2.State;
Expand Down Expand Up @@ -199,7 +198,6 @@ public boolean isSafeToReuseKVState() {
return keyedStateBackend.isSafeToReuseKVState();
}

@VisibleForTesting
public CheckpointableKeyedStateBackend<K> getKeyedStateBackend() {
return keyedStateBackend;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,25 +40,23 @@ public class BatchExecutionInternalTimeService<K, N> implements InternalTimerSer
private static final Logger LOG =
LoggerFactory.getLogger(BatchExecutionInternalTimeService.class);

private final ProcessingTimeService processingTimeService;
final ProcessingTimeService processingTimeService;

/** Processing time timers that are currently in-flight. */
private final KeyGroupedInternalPriorityQueue<TimerHeapInternalTimer<K, N>>
processingTimeTimersQueue;
final KeyGroupedInternalPriorityQueue<TimerHeapInternalTimer<K, N>> processingTimeTimersQueue;

/** Event time timers that are currently in-flight. */
private final KeyGroupedInternalPriorityQueue<TimerHeapInternalTimer<K, N>>
eventTimeTimersQueue;
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;
long currentWatermark = Long.MIN_VALUE;

private final Triggerable<K, N> triggerTarget;
final Triggerable<K, N> triggerTarget;

private K currentKey;
K currentKey;

BatchExecutionInternalTimeService(
ProcessingTimeService processingTimeService, Triggerable<K, N> triggerTarget) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import org.apache.flink.runtime.state.KeyedStateBackend;
import org.apache.flink.runtime.state.KeyedStateCheckpointOutputStream;
import org.apache.flink.runtime.state.PriorityQueueSetFactory;
import org.apache.flink.runtime.state.v2.adaptor.AsyncKeyedStateBackendAdaptor;
import org.apache.flink.streaming.api.operators.InternalTimeServiceManager;
import org.apache.flink.streaming.api.operators.InternalTimerService;
import org.apache.flink.streaming.api.operators.KeyContext;
Expand All @@ -38,7 +39,6 @@
import java.util.Map;

import static org.apache.flink.util.Preconditions.checkNotNull;
import static org.apache.flink.util.Preconditions.checkState;

/**
* An implementation of a {@link InternalTimeServiceManager} that manages timers with a single
Expand All @@ -51,8 +51,14 @@ public class BatchExecutionInternalTimeServiceManager<K>
private final Map<String, BatchExecutionInternalTimeService<K, ?>> timerServices =
new HashMap<>();

public BatchExecutionInternalTimeServiceManager(ProcessingTimeService processingTimeService) {
// In batch mode, there is a chance that the operator is {@link AsyncStateProcessing} and we
// should perform correctly when the timer fires.
private final boolean asyncStateProcessingMode;

public BatchExecutionInternalTimeServiceManager(
ProcessingTimeService processingTimeService, boolean asyncStateProcessingMode) {
this.processingTimeService = checkNotNull(processingTimeService);
this.asyncStateProcessingMode = asyncStateProcessingMode;
}

@Override
Expand All @@ -66,7 +72,11 @@ public <N> InternalTimerService<N> getInternalTimerService(
(BatchExecutionInternalTimeService<K, N>) timerServices.get(name);
if (timerService == null) {
timerService =
new BatchExecutionInternalTimeService<>(processingTimeService, triggerable);
asyncStateProcessingMode
? new BatchExecutionInternalTimeServiceWithAsyncState<>(
processingTimeService, triggerable)
: new BatchExecutionInternalTimeService<>(
processingTimeService, triggerable);
timerServices.put(name, timerService);
}

Expand All @@ -93,6 +103,7 @@ public void snapshotToRawKeyedState(
throw new UnsupportedOperationException("Checkpoints are not supported in BATCH execution");
}

@SuppressWarnings("unchecked")
public static <K> InternalTimeServiceManager<K> create(
TaskIOMetricGroup taskIOMetricGroup,
PriorityQueueSetFactory factory,
Expand All @@ -102,14 +113,27 @@ public static <K> InternalTimeServiceManager<K> create(
ProcessingTimeService processingTimeService,
Iterable<KeyGroupStatePartitionStreamProvider> rawKeyedStates,
StreamTaskCancellationContext cancellationContext) {
checkState(
factory instanceof BatchExecutionKeyedStateBackend,
"Batch execution specific time service can work only with BatchExecutionKeyedStateBackend");
BatchExecutionKeyedStateBackend<K> theFactory = null;
boolean asyncStateProcessingMode = false;
if (factory instanceof BatchExecutionKeyedStateBackend) {
theFactory = (BatchExecutionKeyedStateBackend<K>) factory;
} else if (factory instanceof AsyncKeyedStateBackendAdaptor) {
KeyedStateBackend<K> keyedStateBackend =
((AsyncKeyedStateBackendAdaptor<K>) factory).getKeyedStateBackend();
if (keyedStateBackend instanceof BatchExecutionKeyedStateBackend) {
theFactory = (BatchExecutionKeyedStateBackend<K>) keyedStateBackend;
asyncStateProcessingMode = true;
}
}
if (theFactory == null) {
throw new IllegalStateException(
"Batch execution specific time service can work only with BatchExecutionKeyedStateBackend");
}

BatchExecutionInternalTimeServiceManager<K> timeServiceManager =
new BatchExecutionInternalTimeServiceManager<>(processingTimeService);
((BatchExecutionKeyedStateBackend) factory)
.registerKeySelectionListener(timeServiceManager);
new BatchExecutionInternalTimeServiceManager<>(
processingTimeService, asyncStateProcessingMode);
theFactory.registerKeySelectionListener(timeServiceManager);
return timeServiceManager;
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,85 @@
/*
* 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.asyncprocessing.AsyncExecutionController;
import org.apache.flink.runtime.asyncprocessing.RecordContext;
import org.apache.flink.streaming.api.operators.InternalTimer;
import org.apache.flink.streaming.api.operators.InternalTimerService;
import org.apache.flink.streaming.api.operators.Triggerable;
import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
import org.apache.flink.util.function.ThrowingRunnable;

/**
* 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 cooperating with async state operators.
*/
public class BatchExecutionInternalTimeServiceWithAsyncState<K, N>
extends BatchExecutionInternalTimeService<K, N> {

private AsyncExecutionController<K> asyncExecutionController;

BatchExecutionInternalTimeServiceWithAsyncState(
ProcessingTimeService processingTimeService, Triggerable<K, N> triggerTarget) {
super(processingTimeService, triggerTarget);
}

/** Set up the async execution controller. */
public void setup(AsyncExecutionController<K> asyncExecutionController) {
if (asyncExecutionController != null) {
this.asyncExecutionController = asyncExecutionController;
}
}

/**
* Sets the current key. Timers that are due to be fired are collected and will be triggered.
*/
@Override
public void setCurrentKey(K currentKey) throws Exception {
if (currentKey != null && currentKey.equals(this.currentKey)) {
return;
}
currentWatermark = Long.MAX_VALUE;
InternalTimer<K, N> timer;
while ((timer = eventTimeTimersQueue.poll()) != null) {
final InternalTimer<K, N> timerToTrigger = timer;
maintainContextAndProcess(
timerToTrigger, () -> triggerTarget.onEventTime(timerToTrigger));
}
while ((timer = processingTimeTimersQueue.poll()) != null) {
final InternalTimer<K, N> timerToTrigger = timer;
maintainContextAndProcess(
timerToTrigger, () -> triggerTarget.onProcessingTime(timerToTrigger));
}
currentWatermark = Long.MIN_VALUE;
this.currentKey = currentKey;
}

private void maintainContextAndProcess(
InternalTimer<K, N> timer, ThrowingRunnable<Exception> runnable) {
// Since we are in middle of processing a record, we need to maintain the context.
final RecordContext<K> previousContext = asyncExecutionController.getCurrentContext();
RecordContext<K> recordCtx = asyncExecutionController.buildContext(timer, timer.getKey());
recordCtx.retain();
asyncExecutionController.setCurrentContext(recordCtx);
asyncExecutionController.syncPointRequestWithCallback(runnable, true);
recordCtx.release();
asyncExecutionController.setCurrentContext(previousContext);
}
}
Loading

0 comments on commit d215b61

Please sign in to comment.