Skip to content

Commit 82baf88

Browse files
authored
[FLINK-37084][python] Fix TimerRegistration concurrency issue in PyFlink (#26004)
1 parent d8cd559 commit 82baf88

File tree

33 files changed

+133
-1
lines changed

33 files changed

+133
-1
lines changed

flink-python/src/main/java/org/apache/flink/python/PythonFunctionRunner.java

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -44,6 +44,8 @@ public interface PythonFunctionRunner extends AutoCloseable {
4444
/** Send the triggered timer to the Python function. */
4545
void processTimer(byte[] timerData) throws Exception;
4646

47+
void drainUnregisteredTimers();
48+
4749
/**
4850
* Retrieves the Python function result.
4951
*

flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/AbstractPythonFunctionOperator.java

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -268,17 +268,25 @@ public Configuration getConfiguration() {
268268

269269
protected abstract PythonEnvironmentManager createPythonEnvironmentManager();
270270

271+
protected void drainUnregisteredTimers() {}
272+
271273
/**
272274
* Advances the watermark of all managed timer services, potentially firing event time timers.
273275
* It also ensures that the fired timers are processed in the Python user-defined functions.
274276
*/
275277
private void advanceWatermark(Watermark watermark) throws Exception {
276278
if (getTimeServiceManager().isPresent()) {
277279
InternalTimeServiceManager<?> timeServiceManager = getTimeServiceManager().get();
280+
// make sure the registered timer are processed before advancing the watermark to
281+
// ensure the timers could be triggered
282+
drainUnregisteredTimers();
278283
timeServiceManager.advanceWatermark(watermark);
279284

280285
while (!isBundleFinished()) {
281286
invokeFinishBundle();
287+
// make sure the registered timer are processed before advancing the watermark to
288+
// ensure the timers could be triggered
289+
drainUnregisteredTimers();
282290
timeServiceManager.advanceWatermark(watermark);
283291
}
284292
}

flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/process/AbstractExternalPythonFunctionOperator.java

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -136,6 +136,11 @@ protected ProcessPythonEnvironmentManager createPythonEnvironmentManager() {
136136
}
137137
}
138138

139+
@Override
140+
protected void drainUnregisteredTimers() {
141+
pythonFunctionRunner.drainUnregisteredTimers();
142+
}
143+
139144
protected void emitResults() throws Exception {
140145
Tuple3<String, byte[], Integer> resultTuple;
141146
while ((resultTuple = pythonFunctionRunner.pollResult()) != null && resultTuple.f2 != 0) {

flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/process/ExternalPythonCoProcessOperator.java

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -72,6 +72,7 @@ public void open() throws Exception {
7272
@Override
7373
public PythonFunctionRunner createPythonFunctionRunner() throws Exception {
7474
return new BeamDataStreamPythonFunctionRunner(
75+
getContainingTask().getEnvironment(),
7576
getRuntimeContext().getTaskInfo().getTaskName(),
7677
createPythonEnvironmentManager(),
7778
STATELESS_FUNCTION_URN,

flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/process/ExternalPythonKeyedCoProcessOperator.java

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -113,6 +113,7 @@ public void open() throws Exception {
113113
@Override
114114
public PythonFunctionRunner createPythonFunctionRunner() throws Exception {
115115
return new BeamDataStreamPythonFunctionRunner(
116+
getContainingTask().getEnvironment(),
116117
getRuntimeContext().getTaskInfo().getTaskName(),
117118
createPythonEnvironmentManager(),
118119
STATEFUL_FUNCTION_URN,

flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/process/ExternalPythonKeyedProcessOperator.java

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -139,6 +139,7 @@ public void onProcessingTime(InternalTimer<Row, Object> timer) throws Exception
139139
@Override
140140
public PythonFunctionRunner createPythonFunctionRunner() throws Exception {
141141
return new BeamDataStreamPythonFunctionRunner(
142+
getContainingTask().getEnvironment(),
142143
getRuntimeContext().getTaskInfo().getTaskName(),
143144
createPythonEnvironmentManager(),
144145
STATEFUL_FUNCTION_URN,

flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/process/ExternalPythonProcessOperator.java

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -68,6 +68,7 @@ public void open() throws Exception {
6868
@Override
6969
public PythonFunctionRunner createPythonFunctionRunner() throws Exception {
7070
return new BeamDataStreamPythonFunctionRunner(
71+
getContainingTask().getEnvironment(),
7172
getRuntimeContext().getTaskInfo().getTaskName(),
7273
createPythonEnvironmentManager(),
7374
STATELESS_FUNCTION_URN,
Lines changed: 42 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,42 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
19+
package org.apache.flink.streaming.api.operators.python.process.timer;
20+
21+
public class TimerRegistrationAction {
22+
23+
private final TimerRegistration timerRegistration;
24+
25+
private final byte[] serializedTimerData;
26+
27+
private boolean isRegistered;
28+
29+
public TimerRegistrationAction(
30+
TimerRegistration timerRegistration, byte[] serializedTimerData) {
31+
this.timerRegistration = timerRegistration;
32+
this.serializedTimerData = serializedTimerData;
33+
this.isRegistered = false;
34+
}
35+
36+
public void run() {
37+
if (!isRegistered) {
38+
timerRegistration.setTimer(serializedTimerData);
39+
isRegistered = true;
40+
}
41+
}
42+
}

flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/BeamDataStreamPythonFunctionRunner.java

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,7 @@
2424
import org.apache.flink.python.env.process.ProcessPythonEnvironmentManager;
2525
import org.apache.flink.python.metric.process.FlinkMetricContainer;
2626
import org.apache.flink.python.util.ProtoUtils;
27+
import org.apache.flink.runtime.execution.Environment;
2728
import org.apache.flink.runtime.memory.MemoryManager;
2829
import org.apache.flink.runtime.state.KeyedStateBackend;
2930
import org.apache.flink.runtime.state.OperatorStateBackend;
@@ -69,6 +70,7 @@ public class BeamDataStreamPythonFunctionRunner extends BeamPythonFunctionRunner
6970
private final List<FlinkFnApi.UserDefinedDataStreamFunction> userDefinedDataStreamFunctions;
7071

7172
public BeamDataStreamPythonFunctionRunner(
73+
Environment environment,
7274
String taskName,
7375
ProcessPythonEnvironmentManager environmentManager,
7476
String headOperatorFunctionUrn,
@@ -86,6 +88,7 @@ public BeamDataStreamPythonFunctionRunner(
8688
@Nullable FlinkFnApi.CoderInfoDescriptor timerCoderDescriptor,
8789
Map<String, FlinkFnApi.CoderInfoDescriptor> sideOutputCoderDescriptors) {
8890
super(
91+
environment,
8992
taskName,
9093
environmentManager,
9194
flinkMetricContainer,

flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/BeamPythonFunctionRunner.java

Lines changed: 28 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -31,11 +31,13 @@
3131
import org.apache.flink.python.env.process.ProcessPythonEnvironment;
3232
import org.apache.flink.python.env.process.ProcessPythonEnvironmentManager;
3333
import org.apache.flink.python.metric.process.FlinkMetricContainer;
34+
import org.apache.flink.runtime.execution.Environment;
3435
import org.apache.flink.runtime.memory.MemoryManager;
3536
import org.apache.flink.runtime.memory.OpaqueMemoryResource;
3637
import org.apache.flink.runtime.state.KeyedStateBackend;
3738
import org.apache.flink.runtime.state.OperatorStateBackend;
3839
import org.apache.flink.streaming.api.operators.python.process.timer.TimerRegistration;
40+
import org.apache.flink.streaming.api.operators.python.process.timer.TimerRegistrationAction;
3941
import org.apache.flink.streaming.api.runners.python.beam.state.BeamStateRequestHandler;
4042
import org.apache.flink.util.Preconditions;
4143
import org.apache.flink.util.ShutdownHookUtil;
@@ -85,6 +87,7 @@
8587
import java.util.ArrayList;
8688
import java.util.Collection;
8789
import java.util.Collections;
90+
import java.util.LinkedList;
8891
import java.util.List;
8992
import java.util.Map;
9093
import java.util.Optional;
@@ -190,7 +193,12 @@ public abstract class BeamPythonFunctionRunner implements PythonFunctionRunner {
190193

191194
private transient Thread shutdownHook;
192195

196+
private transient Environment environment;
197+
198+
private transient List<TimerRegistrationAction> unregisteredTimers;
199+
193200
public BeamPythonFunctionRunner(
201+
Environment environment,
194202
String taskName,
195203
ProcessPythonEnvironmentManager environmentManager,
196204
@Nullable FlinkMetricContainer flinkMetricContainer,
@@ -204,6 +212,7 @@ public BeamPythonFunctionRunner(
204212
FlinkFnApi.CoderInfoDescriptor inputCoderDescriptor,
205213
FlinkFnApi.CoderInfoDescriptor outputCoderDescriptor,
206214
Map<String, FlinkFnApi.CoderInfoDescriptor> sideOutputCoderDescriptors) {
215+
this.environment = environment;
207216
this.taskName = Preconditions.checkNotNull(taskName);
208217
this.environmentManager = Preconditions.checkNotNull(environmentManager);
209218
this.flinkMetricContainer = flinkMetricContainer;
@@ -301,6 +310,8 @@ public void open(ReadableConfig config) throws Exception {
301310
shutdownHook =
302311
ShutdownHookUtil.addShutdownHook(
303312
this, BeamPythonFunctionRunner.class.getSimpleName(), LOG);
313+
314+
unregisteredTimers = new LinkedList<>();
304315
}
305316

306317
@Override
@@ -339,6 +350,14 @@ public void process(byte[] data) throws Exception {
339350
mainInputReceiver.accept(WindowedValue.valueInGlobalWindow(data));
340351
}
341352

353+
@Override
354+
public void drainUnregisteredTimers() {
355+
for (TimerRegistrationAction timerRegistrationAction : unregisteredTimers) {
356+
timerRegistrationAction.run();
357+
}
358+
unregisteredTimers.clear();
359+
}
360+
342361
@Override
343362
public void processTimer(byte[] timerData) throws Exception {
344363
if (timerInputReceiver == null) {
@@ -681,7 +700,15 @@ public void onCompleted(BeamFnApi.ProcessBundleResponse response) {
681700

682701
private TimerReceiverFactory createTimerReceiverFactory() {
683702
BiConsumer<Timer<?>, TimerInternals.TimerData> timerDataConsumer =
684-
(timer, timerData) -> timerRegistration.setTimer((byte[]) timer.getUserKey());
703+
(timer, timerData) -> {
704+
TimerRegistrationAction timerRegistrationAction =
705+
new TimerRegistrationAction(
706+
timerRegistration, (byte[]) timer.getUserKey());
707+
unregisteredTimers.add(timerRegistrationAction);
708+
environment
709+
.getMainMailboxExecutor()
710+
.execute(timerRegistrationAction::run, "PythonTimerRegistration");
711+
};
685712
return new TimerReceiverFactory(stageBundleFactory, timerDataConsumer, null);
686713
}
687714

0 commit comments

Comments
 (0)