Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[BP-2.0][FLINK-37206][Runtime] Fix initialization of batching timer service in async state operators #26092

Merged
merged 1 commit into from
Feb 5, 2025
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
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,87 @@
/*
* 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.annotation.Internal;
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.
*/
@Internal
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