Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 39B3E200D6E for ; Sat, 2 Dec 2017 01:28:23 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id 38648160C18; Sat, 2 Dec 2017 00:28:23 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 591B5160C19 for ; Sat, 2 Dec 2017 01:28:22 +0100 (CET) Received: (qmail 63357 invoked by uid 500); 2 Dec 2017 00:28:21 -0000 Mailing-List: contact commits-help@beam.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@beam.apache.org Delivered-To: mailing list commits@beam.apache.org Received: (qmail 63343 invoked by uid 99); 2 Dec 2017 00:28:21 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd3-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Sat, 02 Dec 2017 00:28:21 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd3-us-west.apache.org (ASF Mail Server at spamd3-us-west.apache.org) with ESMTP id BFAE51807A9 for ; Sat, 2 Dec 2017 00:28:20 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd3-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -99.202 X-Spam-Level: X-Spam-Status: No, score=-99.202 tagged_above=-999 required=6.31 tests=[KAM_ASCII_DIVIDERS=0.8, RP_MATCHES_RCVD=-0.001, SPF_PASS=-0.001, USER_IN_WHITELIST=-100] autolearn=disabled Received: from mx1-lw-us.apache.org ([10.40.0.8]) by localhost (spamd3-us-west.apache.org [10.40.0.10]) (amavisd-new, port 10024) with ESMTP id T5QMD6tfisdK for ; Sat, 2 Dec 2017 00:28:17 +0000 (UTC) Received: from mailrelay1-us-west.apache.org (mailrelay1-us-west.apache.org [209.188.14.139]) by mx1-lw-us.apache.org (ASF Mail Server at mx1-lw-us.apache.org) with ESMTP id 01FDB5FB30 for ; Sat, 2 Dec 2017 00:28:13 +0000 (UTC) Received: from jira-lw-us.apache.org (unknown [207.244.88.139]) by mailrelay1-us-west.apache.org (ASF Mail Server at mailrelay1-us-west.apache.org) with ESMTP id 38974E25BC for ; Sat, 2 Dec 2017 00:28:12 +0000 (UTC) Received: from jira-lw-us.apache.org (localhost [127.0.0.1]) by jira-lw-us.apache.org (ASF Mail Server at jira-lw-us.apache.org) with ESMTP id 99952255CD for ; Sat, 2 Dec 2017 00:28:11 +0000 (UTC) Date: Sat, 2 Dec 2017 00:28:11 +0000 (UTC) From: "ASF GitHub Bot (JIRA)" To: commits@beam.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (BEAM-1630) Add Splittable DoFn to Python SDK MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Sat, 02 Dec 2017 00:28:23 -0000 [ https://issues.apache.org/jira/browse/BEAM-1630?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16275270#comment-16275270 ] ASF GitHub Bot commented on BEAM-1630: -------------------------------------- chamikaramj commented on a change in pull request #4064: [BEAM-1630] Adds support for processing Splittable DoFns using DirectRunner. URL: https://github.com/apache/beam/pull/4064#discussion_r153970950 ########## File path: sdks/python/apache_beam/runners/direct/sdf_direct_runner.py ########## @@ -0,0 +1,264 @@ +# +# 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. +# + +"""This module contains Splittable DoFn logic that is specific to DirectRunner. +""" + +from threading import Lock +from threading import Timer + +import apache_beam as beam +from apache_beam import TimeDomain +from apache_beam import pvalue +from apache_beam.io.iobase import RestrictionTracker +from apache_beam.pipeline import PTransformOverride +from apache_beam.runners.common import DoFnContext +from apache_beam.runners.common import DoFnInvoker +from apache_beam.runners.common import DoFnSignature +from apache_beam.runners.common import OutputProcessor +from apache_beam.runners.direct.evaluation_context import DirectStepContext +from apache_beam.runners.direct.util import KeyedWorkItem +from apache_beam.runners.direct.watermark_manager import WatermarkManager +from apache_beam.runners.sdf_common import ElementAndRestriction +from apache_beam.runners.sdf_common import ProcessKeyedElements +from apache_beam.transforms.core import ProcessContinuation +from apache_beam.transforms.ptransform import PTransform +from apache_beam.transforms.trigger import _ValueStateTag +from apache_beam.utils.windowed_value import WindowedValue + + +class ProcessKeyedElementsViaKeyedWorkItemsOverride(PTransformOverride): + """A transform override for ProcessElements transform.""" + + def get_matcher(self): + def _matcher(applied_ptransform): + return isinstance( + applied_ptransform.transform, ProcessKeyedElements) + + return _matcher + + def get_replacement_transform(self, ptransform): + return ProcessKeyedElementsViaKeyedWorkItems(ptransform) + + +class ProcessKeyedElementsViaKeyedWorkItems(PTransform): + """A transform that processes Splittable DoFn input via KeyedWorkItems. + """ + + def __init__(self, process_keyed_elements_transform): + self._process_keyed_elements_transform = process_keyed_elements_transform + + def expand(self, pcoll): + return pcoll | beam.core.GroupByKey() | ProcessElements( + self._process_keyed_elements_transform) + + +class ProcessElements(PTransform): + """Processes keyed input via Splittable DoFn objects.""" + + def __init__(self, process_keyed_elements_transform): + self._process_keyed_elements_transform = process_keyed_elements_transform + self.sdf = self._process_keyed_elements_transform.sdf + + def expand(self, pcoll): + return pvalue.PCollection(pcoll.pipeline) + + def new_process_fn(self, sdf): + return ProcessFn( + sdf, + self._process_keyed_elements_transform.ptransform_args, + self._process_keyed_elements_transform.ptransform_kwargs) + + +class ProcessFn(beam.DoFn): + + def __init__( + self, sdf, args_for_invoker, kwargs_for_invoker): + self.sdf = sdf + self._element_tag = _ValueStateTag('element') + self._restriction_tag = _ValueStateTag('restriction') + self.watermark_hold_tag = _ValueStateTag('watermark_hold') + self._process_element_invoker = None + + self.sdf_invoker = DoFnInvoker.create_invoker( + DoFnSignature(self.sdf), context=DoFnContext('unused_context'), + input_args=args_for_invoker, input_kwargs=kwargs_for_invoker) + + def set_step_context(self, step_context): + assert isinstance(step_context, DirectStepContext) + self._step_context = step_context + + def set_process_element_invoker(self, process_element_invoker): + assert isinstance(process_element_invoker, SDFProcessElementInvoker) + self._process_element_invoker = process_element_invoker + + def start_bundle(self): + # TODO: support start_bundle() method for SDFs. + pass + + def finish_bundle(self): + # TODO: support finish_bundle() method for SDFs. + pass + + def process(self, element, timestamp=beam.DoFn.TimestampParam, + window=beam.DoFn.WindowParam, * args, **kwargs): + if isinstance(element, KeyedWorkItem): + # Must be a timer firing. + key = element.encoded_key + else: + key, values = element + values = list(values) + assert len(values) == 1 + value = values[0] + + state = self._step_context.get_keyed_state(key) + element_state = state.get_state(window, self._element_tag) + is_seed_call = not element_state # Initially element_state is an empty list. + + if not is_seed_call: + element = state.get_state(window, self._element_tag) + restriction = state.get_state(window, self._restriction_tag) + windowed_element = WindowedValue(element, timestamp, [window]) + else: + element_and_restriction = ( + value.value if isinstance(value, WindowedValue) else value) Review comment: Looks like Python SDK does not always wrap elements with WindowdValue objects so this can be an ElementAndRestriction object. ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: users@infra.apache.org > Add Splittable DoFn to Python SDK > --------------------------------- > > Key: BEAM-1630 > URL: https://issues.apache.org/jira/browse/BEAM-1630 > Project: Beam > Issue Type: Improvement > Components: sdk-py-core > Reporter: Chamikara Jayalath > Assignee: Chamikara Jayalath > > Splittable DoFn [1] is currently being implemented for Java SDK [2]. We should add this to Python SDK as well. > Following document proposes an API for this. > https://docs.google.com/document/d/1h_zprJrOilivK2xfvl4L42vaX4DMYGfH1YDmi-s_ozM/edit?usp=sharing > [1] https://s.apache.org/splittable-do-fn > [2] https://lists.apache.org/thread.html/0ce61ac162460a149d5c93cdface37cc383f8030fe86ca09e5699b18@%3Cdev.beam.apache.org%3E -- This message was sent by Atlassian JIRA (v6.4.14#64029)