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/workflows/README.md
Original file line number Diff line number Diff line change
Expand Up @@ -344,6 +344,7 @@ PostCommit Jobs run in a schedule against master branch and generally do not get
| [ PostCommit Java Nexmark Direct ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Direct.yml) | N/A |`beam_PostCommit_Java_Nexmark_Direct.json`| [![.github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Direct.yml?query=event%3Aschedule) |
| [ PostCommit Java Nexmark Flink ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Flink.yml) | N/A |`beam_PostCommit_Java_Nexmark_Flink.json`| [![.github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Flink.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Flink.yml?query=event%3Aschedule) |
| [ PostCommit Java Nexmark Spark ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Spark.yml) | N/A |`beam_PostCommit_Java_Nexmark_Spark.json`| [![.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Spark.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_Nexmark_Spark.yml?query=event%3Aschedule) |
| [ PostCommit Java PVR Flink Batch ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Flink_Batch.yml) | N/A |`beam_PostCommit_Java_PVR_Flink_Batch.json`| [![.github/workflows/beam_PostCommit_Java_PVR_Flink_Batch.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Flink_Batch.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Flink_Batch.yml?query=event%3Aschedule) |
| [ PostCommit Java PVR Flink Streaming ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml) | N/A |`beam_PostCommit_Java_PVR_Flink_Streaming.json`| [![.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml?query=event%3Aschedule) |
| [ PostCommit Java PVR Samza ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Samza.yml) | N/A |`beam_PostCommit_Java_PVR_Samza.json`| [![.github/workflows/beam_PostCommit_Java_PVR_Samza.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Samza.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_PVR_Samza.yml?query=event%3Aschedule) |
| [ PostCommit Java SingleStoreIO IT ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml) | N/A |`beam_PostCommit_Java_SingleStoreIO_IT.json`| [![.github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml?query=event%3Aschedule) |
Expand Down
106 changes: 106 additions & 0 deletions .github/workflows/beam_PostCommit_Java_PVR_Flink_Batch.yml
Original file line number Diff line number Diff line change
@@ -0,0 +1,106 @@
# 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.

name: PostCommit Java PVR Flink Batch

on:
push:
tags: ['v*']
branches: ['master', 'release-*']
paths:
- 'runners/flink/**'
- 'runners/java-fn-execution/**'
- 'sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/**'
- '.github/workflows/beam_PostCommit_Java_PVR_Flink_Batch.yml'
pull_request_target:
branches: ['master', 'release-*']
paths:
- 'release/trigger_all_tests.json'
- '.github/trigger_files/beam_PostCommit_Java_PVR_Flink_Batch.json'
schedule:
- cron: '15 2/6 * * *'
workflow_dispatch:

# This allows a subsequently queued workflow run to interrupt previous runs
concurrency:
group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.event.pull_request.head.label || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.id || github.event.sender.login }}'
cancel-in-progress: true

#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event
permissions:
actions: write
pull-requests: write
checks: write
contents: read
deployments: read
id-token: none
issues: write
discussions: read
packages: read
pages: read
repository-projects: read
security-events: read
statuses: read

env:
DEVELOCITY_ACCESS_KEY: ${{ secrets.DEVELOCITY_ACCESS_KEY }}
GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }}
GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }}

jobs:
beam_PostCommit_Java_PVR_Flink_Batch:
name: ${{ matrix.job_name }} (${{ matrix.job_phrase }})
strategy:
matrix:
job_name: ["beam_PostCommit_Java_PVR_Flink_Batch"]
job_phrase: ["Run Java_PVR_Flink_Batch PostCommit"]
timeout-minutes: 240
runs-on: [self-hosted, ubuntu-20.04, highmem]
if: |
github.event_name == 'push' ||
github.event_name == 'pull_request_target' ||
(github.event_name == 'schedule' && github.repository == 'apache/beam') ||
github.event_name == 'workflow_dispatch' ||
github.event.comment.body == 'Run Java_PVR_Flink_Batch PostCommit'
steps:
- uses: actions/checkout@v4
- name: Setup repository
uses: ./.github/actions/setup-action
with:
comment_phrase: ${{ matrix.job_phrase }}
github_token: ${{ secrets.GITHUB_TOKEN }}
github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }})
- name: Setup environment
uses: ./.github/actions/setup-environment-action
- name: run validatesPortableRunnerBatch script
uses: ./.github/actions/gradle-command-self-hosted-action
with:
gradle-command: :runners:flink:1.20:job-server:validatesPortableRunnerBatchDataSet
env:
CLOUDSDK_CONFIG: ${{ env.KUBELET_GCLOUD_CONFIG_PATH }}
- name: Archive JUnit Test Results
uses: actions/upload-artifact@v4
if: ${{ !success() }}
with:
name: JUnit Test Results
path: "**/build/reports/tests/"
- name: Upload test report
uses: actions/upload-artifact@v4
with:
name: java-code-coverage-report
path: "**/build/test-results/**/*.xml"
# TODO: Investigate 'Max retries exceeded' issue with EnricoMi/publish-unit-test-result-action@v2.
Original file line number Diff line number Diff line change
Expand Up @@ -552,7 +552,8 @@ class BeamModulePlugin implements Plugin<Project> {
project.ext.currentJavaVersion = getSupportedJavaVersion()

project.ext.allFlinkVersions = project.flink_versions.split(',')
project.ext.latestFlinkVersion = project.ext.allFlinkVersions.last()
// TODO(https://github.com/apache/beam/issues/36947): Move to use project.ext.allFlinkVersions.last() when Flink 2 support completed
project.ext.latestFlinkVersion = '1.20'

project.ext.nativeArchitecture = {
// Best guess as to this system's normalized native architecture name.
Expand Down
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.17,1.18,1.19,1.20
flink_versions=1.17,1.18,1.19,1.20,2.0
# supported python versions
python_versions=3.10,3.11,3.12,3.13
43 changes: 43 additions & 0 deletions runners/flink/2.0/build.gradle
Original file line number Diff line number Diff line change
@@ -0,0 +1,43 @@
/*
* 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.
*/

project.ext {
flink_major = '2.0'
flink_version = '2.0.1'
excluded_files = [
'main': [
// Used by DataSet API only
"org/apache/beam/runners/flink/adapter/BeamFlinkDataSetAdapter.java",
"org/apache/beam/runners/flink/FlinkBatchPipelineTranslator.java",
"org/apache/beam/runners/flink/FlinkBatchPortablePipelineTranslator.java",
"org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java",
"org/apache/beam/runners/flink/translation/functions/FlinkNonMergingReduceFunction.java",
// Moved to org.apache.flink.runtime.state.StateBackendFactory
"org/apache/beam/runners/flink/FlinkStateBackendFactory.java",
],
'test': [
// Used by DataSet API only
"org/apache/beam/runners/flink/adapter/BeamFlinkDataSetAdapterTest.java",
"org/apache/beam/runners/flink/batch/NonMergingGroupByKeyTest.java",
"org/apache/beam/runners/flink/batch/ReshuffleTest.java",
]
]
}

// Load the main build script which contains all build logic.
apply from: "../flink_runner.gradle"
26 changes: 26 additions & 0 deletions runners/flink/2.0/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/2.0/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-2.0-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,116 @@
/*
* 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;

import java.util.Map;
import org.apache.beam.runners.flink.translation.types.CoderTypeInformation;
import org.apache.beam.runners.flink.translation.utils.CountingPipelineVisitor;
import org.apache.beam.runners.flink.translation.utils.LookupPipelineVisitor;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.options.PipelineOptions;
import org.apache.beam.sdk.runners.AppliedPTransform;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PValue;
import org.apache.beam.sdk.values.TupleTag;
import org.apache.beam.sdk.values.WindowedValue;
import org.apache.beam.sdk.values.WindowedValues;
import org.apache.beam.sdk.values.WindowingStrategy;
import org.apache.flink.api.common.typeinfo.TypeInformation;

/**
* Helper for {@link FlinkBatchPipelineTranslator} and translators in {@link
* FlinkBatchTransformTranslators}.
*/
@SuppressWarnings({
"nullness" // TODO(https://github.com/apache/beam/issues/20497)
})
class FlinkBatchTranslationContext {
private final PipelineOptions options;

private AppliedPTransform<?, ?, ?> currentTransform;

private final CountingPipelineVisitor countingPipelineVisitor = new CountingPipelineVisitor();
private final LookupPipelineVisitor lookupPipelineVisitor = new LookupPipelineVisitor();

// ------------------------------------------------------------------------

FlinkBatchTranslationContext(PipelineOptions options) {
this.options = options;
}

void init(Pipeline pipeline) {
pipeline.traverseTopologically(countingPipelineVisitor);
pipeline.traverseTopologically(lookupPipelineVisitor);
}

public PipelineOptions getPipelineOptions() {
return options;
}

/**
* Sets the AppliedPTransform which carries input/output.
*
* @param currentTransform Current transformation.
*/
void setCurrentTransform(AppliedPTransform<?, ?, ?> currentTransform) {
this.currentTransform = currentTransform;
}

AppliedPTransform<?, ?, ?> getCurrentTransform() {
return currentTransform;
}

Map<TupleTag<?>, Coder<?>> getOutputCoders(PTransform<?, ?> transform) {
return lookupPipelineVisitor.getOutputCoders(transform);
}

<T> TypeInformation<WindowedValue<T>> getTypeInfo(PCollection<T> collection) {
return getTypeInfo(collection.getCoder(), collection.getWindowingStrategy());
}

<T> TypeInformation<WindowedValue<T>> getTypeInfo(
Coder<T> coder, WindowingStrategy<?, ?> windowingStrategy) {
WindowedValues.FullWindowedValueCoder<T> windowedValueCoder =
WindowedValues.getFullCoder(coder, windowingStrategy.getWindowFn().windowCoder());

return new CoderTypeInformation<>(windowedValueCoder, options);
}

Map<TupleTag<?>, PCollection<?>> getInputs(PTransform<?, ?> transform) {
return lookupPipelineVisitor.getInputs(transform);
}

<T extends PValue> T getInput(PTransform<T, ?> transform) {
return lookupPipelineVisitor.getInput(transform);
}

Map<TupleTag<?>, PCollection<?>> getOutputs(PTransform<?, ?> transform) {
return lookupPipelineVisitor.getOutputs(transform);
}

<T extends PValue> T getOutput(PTransform<?, T> transform) {
return lookupPipelineVisitor.getOutput(transform);
}

/** {@link CountingPipelineVisitor#getNumConsumers(PValue)}. */
int getNumConsumers(PValue value) {
return countingPipelineVisitor.getNumConsumers(value);
}
}
Loading
Loading