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

Consider beam_services option when determining the expansion service type #27353

Merged
merged 1 commit into from
Jul 5, 2023
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
8 changes: 8 additions & 0 deletions sdks/python/apache_beam/transforms/external.py
Original file line number Diff line number Diff line change
Expand Up @@ -885,6 +885,9 @@ def __init__(
self._service_count = 0
self._append_args = append_args or []

def is_existing_service(self):
return subprocess_server.is_service_endpoint(self._path_to_jar)

@staticmethod
def _expand_jars(jar):
if glob.glob(jar):
Expand Down Expand Up @@ -987,6 +990,11 @@ def _maybe_use_transform_service(provided_service=None, options=None):
if not isinstance(provided_service, JavaJarExpansionService):
return provided_service

if provided_service.is_existing_service():
# This is an existing service supported through the 'beam_services'
# PipelineOption.
return provided_service

def is_java_available():
cmd = ['java', '--version']

Expand Down
7 changes: 4 additions & 3 deletions sdks/python/apache_beam/utils/subprocess_server.py
Original file line number Diff line number Diff line change
Expand Up @@ -169,7 +169,7 @@ def __init__(self, stub_class, path_to_jar, java_arguments, classpath=None):
path_to_jar = self.make_classpath_jar(path_to_jar, classpath)
super().__init__(
stub_class, ['java', '-jar', path_to_jar] + list(java_arguments))
self._existing_service = path_to_jar if _is_service_endpoint(
self._existing_service = path_to_jar if is_service_endpoint(
path_to_jar) else None

def start_process(self):
Expand Down Expand Up @@ -258,7 +258,7 @@ def local_jar(cls, url, cache_dir=None):
if cache_dir is None:
cache_dir = cls.JAR_CACHE
# TODO: Verify checksum?
if _is_service_endpoint(url):
if is_service_endpoint(url):
return url
elif os.path.exists(url):
return url
Expand Down Expand Up @@ -330,7 +330,8 @@ def make_classpath_jar(cls, main_jar, extra_jars, cache_dir=None):
return composite_jar


def _is_service_endpoint(path):
def is_service_endpoint(path):
"""Checks whether the path conforms to the 'beam_services' PipelineOption."""
return re.match(r'^[a-zA-Z0-9.-]+:\d+$', path)


Expand Down