Skip to content
Draft
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
1 change: 1 addition & 0 deletions .github/trigger_files/beam_PostCommit_Go_VR_Flink.json
Original file line number Diff line number Diff line change
Expand Up @@ -2,4 +2,5 @@
"comment": "Modify this file in a trivial way to cause this test suite to run",
"modification": 2,
"https://github.com/apache/beam/pull/32440": "testing datastream optimizations",
"pr": "37640"
}
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.net.HostAndPort;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.RuntimeExecutionMode;
import org.apache.flink.api.common.serialization.SerializerConfigImpl;
import org.apache.flink.configuration.CheckpointingOptions;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.CoreOptions;
Expand Down Expand Up @@ -270,6 +271,7 @@ public static StreamExecutionEnvironment createStreamExecutionEnvironment(
flinkStreamEnv.getConfig().setAutoWatermarkInterval(options.getAutoWatermarkInterval());
}
configureWebUIOptions(flinkStreamEnv.getConfig(), options.as(PipelineOptions.class));
configureCustomKryoSerializers(flinkStreamEnv.getConfig());

return flinkStreamEnv;
}
Expand All @@ -294,6 +296,14 @@ private static void configureWebUIOptions(
}
}

private static void configureCustomKryoSerializers(ExecutionConfig config) {
SerializerConfigImpl serializerConfig = (SerializerConfigImpl) config.getSerializerConfig();
// Force Beam schema to use JavaSerializer to fix serialization involving ImmutableMap
serializerConfig.registerTypeWithKryoSerializer(
org.apache.beam.sdk.schemas.Schema.class,
com.esotericsoftware.kryo.serializers.JavaSerializer.class);
}

private static class GlobalJobParametersImpl extends ExecutionConfig.GlobalJobParameters {
private final Map<String, String> jobOptions;

Expand Down
7 changes: 7 additions & 0 deletions runners/flink/flink_runner.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -230,6 +230,13 @@ dependencies {
// configuration (https://issues.apache.org/jira/browse/BEAM-11732).
permitUnusedDeclared "org.apache.flink:flink-clients:$flink_version"

// align with Flink's kryo version (groupId changed)
if (flink_major.startsWith('2')) {
implementation "com.esotericsoftware:kryo:5.6.2"
} else {
implementation "com.esotericsoftware.kryo:kryo:2.24.0"
}

implementation "org.apache.flink:flink-streaming-java:$flink_version"
testImplementation "org.apache.flink:flink-statebackend-rocksdb:$flink_version"
testImplementation "org.apache.flink:flink-streaming-java:$flink_version:tests"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -298,6 +298,7 @@ public static StreamExecutionEnvironment createStreamExecutionEnvironment(
configureStateBackend(options, flinkStreamEnv);

configureWebUIOptions(flinkStreamEnv.getConfig(), options.as(PipelineOptions.class));
configureCustomKryoSerializers(flinkStreamEnv.getConfig());

return flinkStreamEnv;
}
Expand All @@ -322,6 +323,13 @@ private static void configureWebUIOptions(
}
}

private static void configureCustomKryoSerializers(ExecutionConfig config) {
// Force Beam schema to use JavaSerializer to fix serialization involving ImmutableMap
config.registerTypeWithKryoSerializer(
org.apache.beam.sdk.schemas.Schema.class,
com.esotericsoftware.kryo.serializers.JavaSerializer.class);
}

private static class GlobalJobParametersImpl extends ExecutionConfig.GlobalJobParameters {
private final Map<String, String> jobOptions;

Expand Down
3 changes: 1 addition & 2 deletions sdks/go/test/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -79,8 +79,7 @@ task dataflowValidatesRunnerARM64() {
task flinkValidatesRunner {
group = "Verification"

// TODO(https://github.com/apache/beam/issues/37600) use project.ext.latestFlinkVersion after resolved
def flinkVersion = '1.20'
def flinkVersion = project.ext.latestFlinkVersion

dependsOn ":sdks:go:test:goBuild"
dependsOn ":sdks:go:container:docker"
Expand Down
Loading