Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Add flink-conf-dir to flink on portable runner, fix some flink test suites #24254

Closed
wants to merge 20 commits into from
Closed
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
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,6 @@ PostcommitJobBuilder.postCommitJob('beam_PostCommit_Python_Examples_Flink',
gradle {
rootBuildScriptDir(commonJobProperties.checkoutDir)
tasks(":sdks:python:test-suites:portable:flinkExamplesPostCommit")
switches("-PflinkConfDir=$WORKSPACE/src/runners/flink/src/test/resources")
commonJobProperties.setGradleSwitches(delegate)
}
}
Expand Down
2 changes: 0 additions & 2 deletions sdks/go/test/integration/integration.go
Original file line number Diff line number Diff line change
Expand Up @@ -137,8 +137,6 @@ var portableFilters = []string{
var flinkFilters = []string{
// TODO(https://github.com/apache/beam/issues/20723): Flink tests timing out on reads.
"TestXLang_Combine.*",
// TODO(https://github.com/apache/beam/issues/21094): Test fails on post commits: "Insufficient number of network buffers".
"TestXLang_Multi",
"TestDebeziumIO_BasicRead",
// TODO(BEAM-13215): GCP IOs currently do not work in non-Dataflow portable runners.
"TestBigQueryIO.*",
Expand Down
1 change: 1 addition & 0 deletions sdks/python/apache_beam/examples/complete/tfidf_it_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@

class TfIdfIT(unittest.TestCase):
@pytest.mark.examples_postcommit
@pytest.mark.sickbay_flink
def test_basics(self):
test_pipeline = TestPipeline(is_integration_test=True)

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,7 @@ def setUp(self):

@pytest.mark.no_xdist
@pytest.mark.examples_postcommit
@pytest.mark.sickbay_flink
def test_bigquery_side_input_it(self):
state_verifier = PipelineStateMatcher(PipelineState.DONE)
NUM_GROUPS = 3
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -136,6 +136,7 @@ def normalize_tsv_results(self, tsv_data):
return '\n'.join(sorted(lines_out)) + '\n'

@pytest.mark.examples_postcommit
@pytest.mark.sickbay_flink
def test_mergecontacts(self):
test_pipeline = TestPipeline(is_integration_test=True)

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,7 @@ def get_wordcount_results(self, result_path):
return results

@pytest.mark.examples_postcommit
@pytest.mark.sickbay_flink
def test_multiple_output_pardo(self):
test_pipeline = TestPipeline(is_integration_test=True)

Expand Down
1 change: 1 addition & 0 deletions sdks/python/apache_beam/options/pipeline_options.py
Original file line number Diff line number Diff line change
Expand Up @@ -1464,6 +1464,7 @@ def _add_argparse_args(cls, parser):
' directly, rather than starting up a job server.'
' Only applies when flink_master is set to a'
' cluster address. Requires Python 3.6+.')
parser.add_argument('--flink_conf_dir', help='Path to flink-conf-dir')
parser.add_argument(
'--parallelism',
type=int,
Expand Down
6 changes: 5 additions & 1 deletion sdks/python/apache_beam/runners/portability/flink_runner.py
Original file line number Diff line number Diff line change
Expand Up @@ -87,6 +87,7 @@ def __init__(self, options):
self._jar = options.flink_job_server_jar
self._master_url = options.flink_master
self._flink_version = options.flink_version
self.flink_conf_dir = options.flink_conf_dir

def path_to_jar(self):
if self._jar:
Expand All @@ -106,7 +107,7 @@ def path_to_jar(self):

def java_arguments(
self, job_port, artifact_port, expansion_port, artifacts_dir):
return [
args = [
'--flink-master',
self._master_url,
'--artifacts-dir',
Expand All @@ -118,3 +119,6 @@ def java_arguments(
'--expansion-port',
expansion_port
]
if self.flink_conf_dir is not None:
args += ['--flink-conf-dir', self.flink_conf_dir]
return args
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

import argparse
import logging
import pathlib
import shlex
import typing
import unittest
Expand Down Expand Up @@ -55,6 +56,8 @@

Row = typing.NamedTuple("Row", [("col1", int), ("col2", str)])
beam.coders.registry.register_coder(Row, beam.coders.RowCoder)
default_flink_conf_path = pathlib.Path(__file__).parent.joinpath(
'../flink/src/test/resources/').resolve()


class FlinkRunnerTest(portable_runner_test.PortableRunnerTest):
Expand Down Expand Up @@ -144,6 +147,9 @@ def _create_conf_dir(cls):
'metrics.reporter.file.class: %s' % file_reporter,
'metrics.reporter.file.path: %s' % cls.test_metrics_path,
'metrics.scope.operator: <operator_name>',
'parallelism.default: 23',
'taskmanager.memory.network.fraction: 0.2',
'taskmanager.memory.network.max: 2gb',
]))

@classmethod
Expand All @@ -164,7 +170,7 @@ def _subprocess_command(cls, job_port, expansion_port):
'--flink-master',
'[local]',
'--flink-conf-dir',
cls.conf_dir,
cls.conf_dir if cls.conf_dir is not None else default_flink_conf_path,
'--artifacts-dir',
tmp_dir,
'--job-port',
Expand Down
6 changes: 3 additions & 3 deletions sdks/python/test-suites/portable/common.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -230,6 +230,7 @@ project.tasks.register("flinkExamples") {
"--environment_type=LOOPBACK",
"--temp_location=gs://temp-storage-for-end-to-end-tests/temp-it",
"--flink_job_server_jar=${project(":runners:flink:${latestFlinkVersion}:job-server").shadowJar.archivePath}",
"--flink_conf_dir=${rootDir}/runners/flink/src/test/resources",
'--sdk_harness_log_level_overrides=' +
// suppress info level flink.runtime log flood
'{\\"org.apache.flink.runtime\\":\\"WARN\\",' +
Expand Down Expand Up @@ -309,6 +310,7 @@ project.tasks.register("postCommitPy${pythonVersionSuffix}IT") {
"--environment_type=LOOPBACK",
"--temp_location=gs://temp-storage-for-end-to-end-tests/temp-it",
"--flink_job_server_jar=${project(":runners:flink:${latestFlinkVersion}:job-server").shadowJar.archivePath}",
"--flink_conf_dir=${rootDir}/runners/flink/src/test/resources",
'--sdk_harness_log_level_overrides=' +
// suppress info level flink.runtime log flood
'{\\"org.apache.flink.runtime\\":\\"WARN\\",' +
Expand All @@ -317,9 +319,6 @@ project.tasks.register("postCommitPy${pythonVersionSuffix}IT") {
// suppress metric name collision warning logs
'\\"org.apache.flink.runtime.metrics.groups\\":\\"ERROR\\"}'
]
if (project.hasProperty('flinkConfDir')) {
pipelineOpts += ["--flink-conf-dir=${project.property('flinkConfDir')}"]
}
def cmdArgs = mapToArgString([
"test_opts": testOpts,
"suite": "postCommitIT-flink-py${pythonVersionSuffix}",
Expand Down Expand Up @@ -358,6 +357,7 @@ project.tasks.register("xlangSpannerIOIT") {
"--environment_type=LOOPBACK",
"--temp_location=gs://temp-storage-for-end-to-end-tests/temp-it",
"--flink_job_server_jar=${project(":runners:flink:${latestFlinkVersion}:job-server").shadowJar.archivePath}",
"--flink_conf_dir=${rootDir}/runners/flink/src/test/resources",
'--sdk_harness_log_level_overrides=' +
// suppress info level flink.runtime log flood
'{\\"org.apache.flink.runtime\\":\\"WARN\\",' +
Expand Down