[
https://issues.apache.org/jira/browse/BEAM-6747?focusedWorklogId=222892&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-222892
]
ASF GitHub Bot logged work on BEAM-6747:
----------------------------------------
Author: ASF GitHub Bot
Created on: 04/Apr/19 09:51
Start Date: 04/Apr/19 09:51
Worklog Time Spent: 10m
Work Description: mxm commented on pull request #7954: [BEAM-6747] Adding
ExternalTransform in JavaSDK
URL: https://github.com/apache/beam/pull/7954#discussion_r272095794
##########
File path:
sdks/python/apache_beam/runners/portability/expansion_service_test.py
##########
@@ -0,0 +1,155 @@
+#
+# 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.
+#
+from __future__ import absolute_import
+
+import argparse
+import concurrent.futures as futures
+import logging
+import sys
+import time
+
+import grpc
+
+import apache_beam as beam
+import apache_beam.transforms.combiners as combine
+from apache_beam.pipeline import PipelineOptions
+from apache_beam.portability.api import beam_expansion_api_pb2_grpc
+from apache_beam.runners.portability import expansion_service
+from apache_beam.transforms import ptransform
+
+# This script provides an expansion service and example ptransforms for running
+# external transform test cases. See external_test.py for details.
+
+
[email protected]_urn('count_per_element_bytes', None)
+class KV2BytesTransform(ptransform.PTransform):
+ def expand(self, pcoll):
+ return (
+ pcoll
+ | combine.Count.PerElement()
+ | beam.Map(
+ lambda x: '{}->{}'.format(x[0], x[1])).with_output_types(bytes)
+ )
+
+ def to_runner_api_parameter(self, unused_context):
+ return 'kv_to_bytes', None
+
+ @staticmethod
+ def from_runner_api_parameter(unused_parameter, unused_context):
+ return KV2BytesTransform()
+
+
[email protected]_urn('simple', None)
+class SimpleTransform(ptransform.PTransform):
+ def expand(self, pcoll):
+ return pcoll | beam.Map(lambda x: 'Simple(%s)' % x)
+
+ def to_runner_api_parameter(self, unused_context):
+ return 'simple', None
+
+ @staticmethod
+ def from_runner_api_parameter(unused_parameter, unused_context):
+ return SimpleTransform()
+
+
[email protected]_urn('multi', None)
+class MutltiTransform(ptransform.PTransform):
+ def expand(self, pcolls):
+ return {
+ 'main':
+ (pcolls['main1'], pcolls['main2'])
+ | beam.Flatten()
+ | beam.Map(lambda x, s: x + s,
+ beam.pvalue.AsSingleton(pcolls['side'])),
+ 'side': pcolls['side'] | beam.Map(lambda x: x + x),
+ }
+
+ def to_runner_api_parameter(self, unused_context):
+ return 'multi', None
+
+ @staticmethod
+ def from_runner_api_parameter(unused_parameter, unused_context):
+ return MutltiTransform()
+
+
[email protected]_urn('payload', bytes)
+class PayloadTransform(ptransform.PTransform):
+ def __init__(self, payload):
+ self._payload = payload
+
+ def expand(self, pcoll):
+ return pcoll | beam.Map(lambda x, s: x + s, self._payload)
+
+ def to_runner_api_parameter(self, unused_context):
+ return b'payload', self._payload.encode('ascii')
+
+ @staticmethod
+ def from_runner_api_parameter(payload, unused_context):
+ return PayloadTransform(payload.decode('ascii'))
+
+
[email protected]_urn('fib', bytes)
+class FibTransform(ptransform.PTransform):
+ def __init__(self, level):
+ self._level = level
+
+ def expand(self, p):
+ if self._level <= 2:
+ return p | beam.Create([1])
+ else:
+ a = p | 'A' >> beam.ExternalTransform(
+ 'fib', str(self._level - 1).encode('ascii'),
+ expansion_service.ExpansionServiceServicer())
+ b = p | 'B' >> beam.ExternalTransform(
+ 'fib', str(self._level - 2).encode('ascii'),
+ expansion_service.ExpansionServiceServicer())
+ return (
+ (a, b)
+ | beam.Flatten()
+ | beam.CombineGlobally(sum).without_defaults())
+
+ def to_runner_api_parameter(self, unused_context):
+ return 'fib', str(self._level).encode('ascii')
+
+ @staticmethod
+ def from_runner_api_parameter(level, unused_context):
+ return FibTransform(int(level.decode('ascii')))
+
+
+def main(unused_argv):
+ parser = argparse.ArgumentParser()
+ parser.add_argument('-p', '--port',
+ type=int,
+ help='port on which to serve the job api')
+ options = parser.parse_args()
+ server = grpc.server(futures.ThreadPoolExecutor(max_workers=2))
+ beam_expansion_api_pb2_grpc.add_ExpansionServiceServicer_to_server(
+ expansion_service.ExpansionServiceServicer(PipelineOptions()), server
+ )
+ server.add_insecure_port('localhost:{}'.format(options.port))
+ server.start()
+ try:
+ while True:
+ logging.info('Listening for expansion requests at %d', options.port)
+ time.sleep(300)
Review comment:
I don't understand why we need this loop. Either we should timeout after 300
seconds or block here forever.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 222892)
Time Spent: 3h 40m (was: 3.5h)
> Adding ExternalTransform in JavaSDK
> -----------------------------------
>
> Key: BEAM-6747
> URL: https://issues.apache.org/jira/browse/BEAM-6747
> Project: Beam
> Issue Type: Improvement
> Components: runner-core
> Reporter: Heejong Lee
> Assignee: Heejong Lee
> Priority: Major
> Time Spent: 3h 40m
> Remaining Estimate: 0h
>
> Adding Java counterpart of Python ExternalTransform for testing Python
> transforms from pipelines in Java SDK.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)