[ 
https://issues.apache.org/jira/browse/BEAM-7443?focusedWorklogId=251079&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-251079
 ]

ASF GitHub Bot logged work on BEAM-7443:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 30/May/19 16:34
            Start Date: 30/May/19 16:34
    Worklog Time Spent: 10m 
      Work Description: lukecwik commented on pull request #8641: [BEAM-7443] 
Create a BoundedSource -> SDF wrapper in Python SDK
URL: https://github.com/apache/beam/pull/8641#discussion_r289065788
 
 

 ##########
 File path: sdks/python/apache_beam/io/sdf_restriction_provider.py
 ##########
 @@ -0,0 +1,56 @@
+#
+# 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
+
+from apache_beam.io.restriction_trackers import 
SDFBoundedSourceRestrictionTracker
+from apache_beam.transforms.core import RestrictionProvider
+
+__all__ = ['SDFBoundedSourceRestrictionProvider']
+
+
+class SDFBoundedSourceRestrictionProvider(RestrictionProvider):
+  """A `RestrictionProvider` that is used by SDF for `BoundedSource`."""
+
+  def __init__(self, source, desired_chunk_size=None):
+    self._source = source
+    self._desired_chunk_size = desired_chunk_size
+    # The size cannot be calculated directly by end_pos - start_pos since the
+    # position may not be numeric(e.g., key space position).
+    self._restriction_size_map = {}
+
+  def initial_restriction(self, element):
+    # Get initial range_tracker from source
+    range_tracker = self._source.get_range_tracker(None, None)
+    return (range_tracker.start_position(), range_tracker.stop_position())
+
+  def create_tracker(self, restriction):
+    return SDFBoundedSourceRestrictionTracker(
+        self._source.get_range_tracker(restriction[0],
+                                       restriction[1]))
+
+  def split(self, element, restriction):
+    # Invoke source.split to get initial splitting results.
+    source_bundles = self._source.split(self._desired_chunk_size)
+    for source_bundle in source_bundles:
+      self._restriction_size_map[(source_bundle.start_position,
+                                  source_bundle.stop_position)]\
+        = source_bundle.weight
+      yield (source_bundle.start_position, source_bundle.stop_position)
 
 Review comment:
   I don't believe you can assume that the source you have as the element equal 
to the source being returned within SourceBundle.
   
   If you take a look at the file based source, you can see that the 
SourceBundle may return a ConcatSource or a _SingleFileSource.
   
   I believe you'll want to make the "restriction" a SourceBundle and use the 
source from the SourceBundle.
   
   The initial restriction could return the SourceBundle(None, source, None, 
None). Conveniently, this would allow you to return source_bundle.weight and 
fall back to source_bundle.source.estimate_size() if source_bundle.weight is 
None.
 
----------------------------------------------------------------
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: 251079)
            Time Spent: 10m
    Remaining Estimate: 0h

>  BoundedSource->SDF needs a wrapper in Python SDK
> -------------------------------------------------
>
>                 Key: BEAM-7443
>                 URL: https://issues.apache.org/jira/browse/BEAM-7443
>             Project: Beam
>          Issue Type: New Feature
>          Components: sdk-py-core
>            Reporter: Boyuan Zhang
>            Assignee: Boyuan Zhang
>            Priority: Major
>          Time Spent: 10m
>  Remaining Estimate: 0h
>




--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to