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 JobServerOption for --jar_cache_dir #32033

Open
wants to merge 27 commits into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from 1 commit
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
5 changes: 4 additions & 1 deletion sdks/python/apache_beam/options/pipeline_options.py
Original file line number Diff line number Diff line change
Expand Up @@ -1627,7 +1627,10 @@ def _add_argparse_args(cls, parser):
action='append',
s21lee marked this conversation as resolved.
Show resolved Hide resolved
default=[],
help='JVM properties to pass to a Java job server.')

parser.add_argument(
'--jar_cache_dir',
default=None,
help='The location to store jar cache for job server.')

class FlinkRunnerOptions(PipelineOptions):

Expand Down
6 changes: 6 additions & 0 deletions sdks/python/apache_beam/options/pipeline_options_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
from apache_beam.options.pipeline_options import DebugOptions
from apache_beam.options.pipeline_options import GoogleCloudOptions
from apache_beam.options.pipeline_options import PipelineOptions
from apache_beam.options.pipeline_options import JobServerOptions
from apache_beam.options.pipeline_options import ProfilingOptions
from apache_beam.options.pipeline_options import TypeOptions
from apache_beam.options.pipeline_options import WorkerOptions
Expand Down Expand Up @@ -639,6 +640,11 @@ def test_transform_name_mapping(self):
mapping = options.view_as(GoogleCloudOptions).transform_name_mapping
self.assertEqual(mapping['from'], 'to')

def test_jar_cache_dir(self):
options = PipelineOptions(['--jar_cache_dir=/path/to/jar_cache_dir'])
jar_cache_dir = options.view_as(JobServerOptions).jar_cache_dir
self.assertEqual(jar_cache_dir, '/path/to/jar_cache_dir')

def test_dataflow_service_options(self):
options = PipelineOptions([
'--dataflow_service_option',
Expand Down
7 changes: 4 additions & 3 deletions sdks/python/apache_beam/runners/portability/job_server.py
Original file line number Diff line number Diff line change
Expand Up @@ -127,6 +127,7 @@ def __init__(self, options):
self._artifacts_dir = options.artifacts_dir
self._java_launcher = options.job_server_java_launcher
self._jvm_properties = options.job_server_jvm_properties
self._jar_cache_dir = options.jar_cache_dir

def java_arguments(
self, job_port, artifact_port, expansion_port, artifacts_dir):
Expand All @@ -141,11 +142,11 @@ def path_to_beam_jar(gradle_target, artifact_id=None):
gradle_target, artifact_id=artifact_id)

@staticmethod
def local_jar(url):
return subprocess_server.JavaJarServer.local_jar(url)
def local_jar(url, jar_cache_dir=None):
return subprocess_server.JavaJarServer.local_jar(url, jar_cache_dir)

def subprocess_cmd_and_endpoint(self):
jar_path = self.local_jar(self.path_to_jar())
jar_path = self.local_jar(self.path_to_jar(), self._jar_cache_dir)
artifacts_dir = (
self._artifacts_dir if self._artifacts_dir else self.local_temp_dir(
prefix='artifacts'))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@

class JavaJarJobServerStub(JavaJarJobServer):
def java_arguments(
self, job_port, artifact_port, expansion_port, artifacts_dir):
self, job_port, artifact_port, expansion_port, artifacts_dir, jar_cache_dir):
return [
'--artifacts-dir',
artifacts_dir,
Expand All @@ -35,13 +35,15 @@ def java_arguments(
artifact_port,
'--expansion-port',
expansion_port
s21lee marked this conversation as resolved.
Show resolved Hide resolved
'--jar_cache_dir',
jar_cache_dir
]

def path_to_jar(self):
return '/path/to/jar'

@staticmethod
def local_jar(url):
def local_jar(url, jar_cache_dir):
return url


Expand All @@ -54,6 +56,7 @@ def test_subprocess_cmd_and_endpoint(self):
'--artifacts_dir=/path/to/artifacts/',
'--job_server_java_launcher=/path/to/java',
'--job_server_jvm_properties=-Dsome.property=value'
'--jar_cache_dir=/path/to/cache_dir'
])
job_server = JavaJarJobServerStub(pipeline_options)
subprocess_cmd, endpoint = job_server.subprocess_cmd_and_endpoint()
Expand All @@ -72,6 +75,8 @@ def test_subprocess_cmd_and_endpoint(self):
8098,
'--expansion-port',
8097
'--jar-cache-dir',
'/path/to/cache_dir'
])
self.assertEqual(endpoint, 'localhost:8099')

Expand Down
Loading