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

[BEAM-10208] add cross-language KafkaIO integration test #11942

Merged
merged 1 commit into from
Jun 16, 2020
Merged
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
2 changes: 2 additions & 0 deletions build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -244,6 +244,7 @@ task pythonFormatterPreCommit() {
}

task python2PostCommit() {
dependsOn ":sdks:python:test-suites:portable:py2:crossLanguagePythonJavaKafkaIOFlink"
dependsOn ":sdks:python:test-suites:portable:py2:crossLanguageTests"
dependsOn ":sdks:python:test-suites:dataflow:py2:postCommitIT"
dependsOn ":sdks:python:test-suites:direct:py2:directRunnerIT"
Expand All @@ -265,6 +266,7 @@ task python36PostCommit() {
}

task python37PostCommit() {
dependsOn ":sdks:python:test-suites:portable:py37:crossLanguagePythonJavaKafkaIOFlink"
dependsOn ":sdks:python:test-suites:dataflow:py37:postCommitIT"
dependsOn ":sdks:python:test-suites:direct:py37:postCommitIT"
dependsOn ":sdks:python:test-suites:direct:py37:hdfsIntegrationTest"
Expand Down
28 changes: 12 additions & 16 deletions sdks/python/apache_beam/io/external/xlang_kafkaio_it_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import os
import socket
import subprocess
import sys
import time
import typing
import unittest
Expand All @@ -32,7 +33,6 @@
from apache_beam.io.external.kafka import ReadFromKafka
from apache_beam.io.external.kafka import WriteToKafka
from apache_beam.metrics import Metrics
from apache_beam.options.pipeline_options import PipelineOptions
from apache_beam.testing.test_pipeline import TestPipeline


Expand Down Expand Up @@ -86,6 +86,12 @@ def run_xlang_kafkaio(self, pipeline):
os.environ.get('LOCAL_KAFKA_JAR'),
"LOCAL_KAFKA_JAR environment var is not provided.")
class CrossLanguageKafkaIOTest(unittest.TestCase):
def get_platform_localhost(self):
if sys.platform == 'darwin':
return 'host.docker.internal'
else:
return 'localhost'

def get_open_port(self):
s = None
try:
Expand Down Expand Up @@ -113,25 +119,15 @@ def local_kafka_service(self, local_kafka_jar_file):
if kafka_server:
kafka_server.kill()

def get_options(self):
options = PipelineOptions([
'--runner',
'FlinkRunner',
'--parallelism',
'2',
'--experiment',
'beam_fn_api'
])
return options

def test_kafkaio_write(self):
local_kafka_jar = os.environ.get('LOCAL_KAFKA_JAR')
with self.local_kafka_service(local_kafka_jar) as kafka_port:
options = self.get_options()
p = TestPipeline(options=options)
p = TestPipeline()
p.not_use_test_runner_api = True
CrossLanguageKafkaIO('localhost:%s' % kafka_port,
'xlang_kafkaio_test').build_write_pipeline(p)
xlang_kafkaio = CrossLanguageKafkaIO(
'%s:%s' % (self.get_platform_localhost(), kafka_port),
'xlang_kafkaio_test')
xlang_kafkaio.build_write_pipeline(p)
job = p.run()
job.wait_until_finish()

Expand Down
32 changes: 32 additions & 0 deletions sdks/python/test-suites/portable/common.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -99,6 +99,38 @@ task flinkTriggerTranscript() {
}
}

task crossLanguagePythonJavaKafkaIOFlink {
dependsOn 'setupVirtualenv'
dependsOn ':runners:flink:1.10:job-server:shadowJar'
dependsOn ":sdks:python:container:py${pythonVersionSuffix}:docker"
dependsOn ':sdks:java:container:docker'
dependsOn ':sdks:java:io:expansion-service:shadowJar'
dependsOn ':sdks:java:testing:kafka-service:buildTestKafkaServiceJar'

doLast {
def kafkaJar = project(":sdks:java:testing:kafka-service:").buildTestKafkaServiceJar.archivePath
def options = [
"--runner=FlinkRunner",
"--parallelism=2",
"--environment_type=DOCKER",
"--environment_cache_millis=10000",
"--experiment=beam_fn_api",
]
exec {
environment "LOCAL_KAFKA_JAR", kafkaJar
executable 'sh'
args '-c', """
. ${envdir}/bin/activate \\
&& cd ${pythonRootDir} \\
&& pip install -e .[test] \\
&& python setup.py nosetests \\
--tests apache_beam.io.external.xlang_kafkaio_it_test:CrossLanguageKafkaIOTest \\
--test-pipeline-options='${options.join(' ')}'
"""
}
}
}

project.task("preCommitPy${pythonVersionSuffix}") {
dependsOn = [":sdks:python:container:py${pythonVersionSuffix}:docker",
':runners:flink:1.10:job-server:shadowJar',
Expand Down