Skip to content

Commit

Permalink
[flink] add support for Flink 1.17 (apache#29939)
Browse files Browse the repository at this point in the history
  • Loading branch information
je-ik committed Feb 2, 2024
1 parent 1f69271 commit 5b13d19
Show file tree
Hide file tree
Showing 10 changed files with 233 additions and 32 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -78,7 +78,7 @@ jobs:
- name: run validatesRunner script
uses: ./.github/actions/gradle-command-self-hosted-action
with:
gradle-command: :runners:flink:1.15:validatesRunner
gradle-command: :runners:flink:1.17:validatesRunner
- name: Archive JUnit Test Results
uses: actions/upload-artifact@v4
if: ${{ !success() }}
Expand All @@ -92,4 +92,4 @@ jobs:
large_files: true
commit: '${{ env.prsha || env.GITHUB_SHA }}'
comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }}
files: '**/build/test-results/**/*.xml'
files: '**/build/test-results/**/*.xml'
2 changes: 1 addition & 1 deletion gradle.properties
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,6 @@ docker_image_default_repo_root=apache
docker_image_default_repo_prefix=beam_

# supported flink versions
flink_versions=1.12,1.13,1.14,1.15,1.16
flink_versions=1.12,1.13,1.14,1.15,1.16,1.17
# supported python versions
python_versions=3.8,3.9,3.10,3.11
Original file line number Diff line number Diff line change
@@ -0,0 +1,50 @@
/*
* 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.beam.runners.flink.translation.types.compat;

import org.apache.beam.runners.flink.translation.types.CoderTypeSerializer;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility;
import org.apache.flink.core.io.VersionedIOReadableWritable;

/** A legacy snapshot which does not care about schema compatibility. */
@SuppressWarnings("allcheckers")
public class UnversionedTypeSerializerSnapshot<T> extends TypeSerializerConfigSnapshot<T> {

/** Needs to be public to work with {@link VersionedIOReadableWritable}. */
public UnversionedTypeSerializerSnapshot() {}

@SuppressWarnings("initialization")
public UnversionedTypeSerializerSnapshot(CoderTypeSerializer<T> serializer) {
super.setPriorSerializer(serializer);
}

@Override
public int getVersion() {
// We always return the same version
return 1;
}

@Override
public TypeSerializerSchemaCompatibility<T> resolveSchemaCompatibility(
TypeSerializer<T> newSerializer) {
// We assume compatibility because we don't have a way of checking schema compatibility
return TypeSerializerSchemaCompatibility.compatibleAsIs();
}
}
34 changes: 34 additions & 0 deletions runners/flink/1.17/build.gradle
Original file line number Diff line number Diff line change
@@ -0,0 +1,34 @@
/*
* 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.
*/

def basePath = '..'

/* All properties required for loading the Flink build script */
project.ext {
// Set the version of all Flink-related dependencies here.
flink_version = '1.17.0'
// Version specific code overrides.
main_source_overrides = ["${basePath}/1.14/src/main/java", "${basePath}/1.15/src/main/java", "${basePath}/1.16/src/main/java", './src/main/java']
test_source_overrides = ["${basePath}/1.14/src/test/java", "${basePath}/1.15/src/test/java", "${basePath}/1.16/src/test/java", './src/test/java']
main_resources_overrides = []
test_resources_overrides = []
archives_base_name = 'beam-runners-flink-1.17'
}

// Load the main build script which contains all build logic.
apply from: "$basePath/flink_runner.gradle"
26 changes: 26 additions & 0 deletions runners/flink/1.17/job-server-container/build.gradle
Original file line number Diff line number Diff line change
@@ -0,0 +1,26 @@
/*
* 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.
*/

def basePath = '../../job-server-container'

project.ext {
resource_path = basePath
}

// Load the main build script which contains all build logic.
apply from: "$basePath/flink_job_server_container.gradle"
31 changes: 31 additions & 0 deletions runners/flink/1.17/job-server/build.gradle
Original file line number Diff line number Diff line change
@@ -0,0 +1,31 @@
/*
* 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.
*/

def basePath = '../../job-server'

project.ext {
// Look for the source code in the parent module
main_source_dirs = ["$basePath/src/main/java"]
test_source_dirs = ["$basePath/src/test/java"]
main_resources_dirs = ["$basePath/src/main/resources"]
test_resources_dirs = ["$basePath/src/test/resources"]
archives_base_name = 'beam-runners-flink-1.17-job-server'
}

// Load the main build script which contains all build logic.
apply from: "$basePath/flink_job_server.gradle"
Original file line number Diff line number Diff line change
@@ -0,0 +1,81 @@
/*
* 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.beam.runners.flink.translation.types.compat;

import java.io.IOException;
import org.apache.beam.runners.flink.translation.types.CoderTypeSerializer;
import org.apache.beam.sdk.util.SerializableUtils;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility;
import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot;
import org.apache.flink.core.memory.DataInputView;
import org.apache.flink.core.memory.DataOutputView;

/** A legacy snapshot which does not care about schema compatibility. */
@SuppressWarnings("allcheckers")
public class UnversionedTypeSerializerSnapshot<T> implements TypeSerializerSnapshot<T> {

private CoderTypeSerializer<T> serializer = null;

/** Needs to be public to work with {@link VersionedIOReadableWritable}. */
public UnversionedTypeSerializerSnapshot() {}

@SuppressWarnings("initialization")
public UnversionedTypeSerializerSnapshot(CoderTypeSerializer<T> serializer) {
this.serializer = serializer;
}

@Override
public int getCurrentVersion() {
// We always return the same version
return 1;
}

@Override
public void writeSnapshot(DataOutputView dataOutputView) throws IOException {
byte[] bytes = SerializableUtils.serializeToByteArray(serializer);
dataOutputView.writeInt(getCurrentVersion());
dataOutputView.writeInt(bytes.length);
dataOutputView.write(bytes);
}

@Override
public void readSnapshot(int i, DataInputView dataInputView, ClassLoader classLoader)
throws IOException {
// discard version
dataInputView.readInt();
int length = dataInputView.readInt();
byte[] bytes = new byte[length];
dataInputView.readFully(bytes);
this.serializer =
(CoderTypeSerializer<T>)
SerializableUtils.deserializeFromByteArray(bytes, "CoderTypeSerializer");
}

@Override
public TypeSerializer<T> restoreSerializer() {
return serializer;
}

@Override
public TypeSerializerSchemaCompatibility<T> resolveSchemaCompatibility(
TypeSerializer<T> newSerializer) {
// We assume compatibility because we don't have a way of checking schema compatibility
return TypeSerializerSchemaCompatibility.compatibleAsIs();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,17 +21,15 @@
import java.io.IOException;
import org.apache.beam.runners.core.construction.SerializablePipelineOptions;
import org.apache.beam.runners.flink.FlinkPipelineOptions;
import org.apache.beam.runners.flink.translation.types.compat.UnversionedTypeSerializerSnapshot;
import org.apache.beam.runners.flink.translation.wrappers.DataInputViewWrapper;
import org.apache.beam.runners.flink.translation.wrappers.DataOutputViewWrapper;
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.coders.CoderException;
import org.apache.beam.sdk.util.CoderUtils;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility;
import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot;
import org.apache.flink.core.io.VersionedIOReadableWritable;
import org.apache.flink.core.memory.DataInputView;
import org.apache.flink.core.memory.DataOutputView;
import org.checkerframework.checker.nullness.qual.Nullable;
Expand Down Expand Up @@ -155,31 +153,7 @@ public int hashCode() {

@Override
public TypeSerializerSnapshot<T> snapshotConfiguration() {
return new LegacySnapshot<>(this);
}

/** A legacy snapshot which does not care about schema compatibility. */
public static class LegacySnapshot<T> extends TypeSerializerConfigSnapshot<T> {

/** Needs to be public to work with {@link VersionedIOReadableWritable}. */
public LegacySnapshot() {}

public LegacySnapshot(CoderTypeSerializer<T> serializer) {
setPriorSerializer(serializer);
}

@Override
public int getVersion() {
// We always return the same version
return 1;
}

@Override
public TypeSerializerSchemaCompatibility<T> resolveSchemaCompatibility(
TypeSerializer<T> newSerializer) {
// We assume compatibility because we don't have a way of checking schema compatibility
return TypeSerializerSchemaCompatibility.compatibleAsIs();
}
return new UnversionedTypeSerializerSnapshot<>(this);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import java.io.OutputStream;
import java.io.Serializable;
import org.apache.beam.runners.core.construction.SerializablePipelineOptions;
import org.apache.beam.runners.flink.translation.types.compat.UnversionedTypeSerializerSnapshot;
import org.apache.beam.sdk.coders.AtomicCoder;
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.coders.StringUtf8Coder;
Expand Down Expand Up @@ -71,7 +72,7 @@ private void testWriteAndReadConfigSnapshot(Coder<String> coder) throws IOExcept
ComparatorTestBase.TestOutputView outView = new ComparatorTestBase.TestOutputView();
writtenSnapshot.writeSnapshot(outView);

TypeSerializerSnapshot readSnapshot = new CoderTypeSerializer.LegacySnapshot();
TypeSerializerSnapshot readSnapshot = new UnversionedTypeSerializerSnapshot();
readSnapshot.readSnapshot(
writtenSnapshot.getCurrentVersion(), outView.getInputView(), getClass().getClassLoader());

Expand Down
4 changes: 4 additions & 0 deletions settings.gradle.kts
Original file line number Diff line number Diff line change
Expand Up @@ -141,6 +141,10 @@ include(":runners:flink:1.15:job-server-container")
include(":runners:flink:1.16")
include(":runners:flink:1.16:job-server")
include(":runners:flink:1.16:job-server-container")
// Flink 1.17
include(":runners:flink:1.17")
include(":runners:flink:1.17:job-server")
include(":runners:flink:1.17:job-server-container")
/* End Flink Runner related settings */
include(":runners:twister2")
include(":runners:google-cloud-dataflow-java")
Expand Down

0 comments on commit 5b13d19

Please sign in to comment.