1717 */
1818package org .apache .beam .runners .flink ;
1919
20- import static org .apache .beam .sdk .util .Preconditions .checkStateNotNull ;
2120import static org .apache .beam .sdk .util .construction .resources .PipelineResources .detectClassPathResourcesToStage ;
2221
2322import java .util .UUID ;
3231import org .apache .beam .vendor .grpc .v1p69p0 .com .google .protobuf .Struct ;
3332import org .apache .beam .vendor .guava .v32_1_2_jre .com .google .common .base .Strings ;
3433import org .apache .beam .vendor .guava .v32_1_2_jre .com .google .common .util .concurrent .ListeningExecutorService ;
34+ import org .checkerframework .checker .nullness .qual .Nullable ;
3535import org .slf4j .Logger ;
3636import org .slf4j .LoggerFactory ;
3737
3838/** Job Invoker for the {@link FlinkRunner}. */
39+ @ SuppressWarnings ({
40+ "nullness" // TODO(https://github.com/apache/beam/issues/20497)
41+ })
3942public class FlinkJobInvoker extends JobInvoker {
4043 private static final Logger LOG = LoggerFactory .getLogger (FlinkJobInvoker .class );
4144
@@ -54,7 +57,7 @@ protected FlinkJobInvoker(FlinkJobServerDriver.FlinkServerConfiguration serverCo
5457 protected JobInvocation invokeWithExecutor (
5558 RunnerApi .Pipeline pipeline ,
5659 Struct options ,
57- String retrievalToken ,
60+ @ Nullable String retrievalToken ,
5861 ListeningExecutorService executorService ) {
5962
6063 // TODO: How to make Java/Python agree on names of keys and their values?
@@ -71,22 +74,20 @@ protected JobInvocation invokeWithExecutor(
7174
7275 PortablePipelineOptions portableOptions = flinkOptions .as (PortablePipelineOptions .class );
7376
74- ClassLoader thisClassLoader =
75- checkStateNotNull (
76- FlinkJobInvoker .class .getClassLoader (),
77- "FlinkJobInvoker class loader is null - this means it was loaded by the bootstrap classloader, which should be impossible" );
78-
7977 PortablePipelineRunner pipelineRunner ;
8078 if (Strings .isNullOrEmpty (portableOptions .getOutputExecutablePath ())) {
8179 pipelineRunner =
8280 new FlinkPipelineRunner (
8381 flinkOptions ,
8482 serverConfig .getFlinkConfDir (),
85- detectClassPathResourcesToStage (thisClassLoader , flinkOptions ));
83+ detectClassPathResourcesToStage (
84+ FlinkJobInvoker .class .getClassLoader (), flinkOptions ));
8685 } else {
8786 pipelineRunner = new PortablePipelineJarCreator (FlinkPipelineRunner .class );
8887 }
8988
89+ flinkOptions .setRunner (null );
90+
9091 LOG .info ("Invoking job {} with pipeline runner {}" , invocationId , pipelineRunner );
9192 return createJobInvocation (
9293 invocationId , retrievalToken , executorService , pipeline , flinkOptions , pipelineRunner );
0 commit comments