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

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

                Author: ASF GitHub Bot
            Created on: 13/Sep/19 21:52
            Start Date: 13/Sep/19 21:52
    Worklog Time Spent: 10m 
      Work Description: robertwb commented on pull request #9366: [BEAM-8143] 
Build simple LineSource directly on top of SDF
URL: https://github.com/apache/beam/pull/9366#discussion_r324378492
 
 

 ##########
 File path: sdks/python/apache_beam/io/sdf_line_source.py
 ##########
 @@ -0,0 +1,89 @@
+#
+# 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 os
+
+from apache_beam import Create
+from apache_beam import pvalue
+from apache_beam.io.restriction_trackers import OffsetRange
+from apache_beam.io.restriction_trackers import OffsetRestrictionTracker
+from apache_beam.transforms import core
+from apache_beam.transforms import ptransform
+
+
+class SdfReadLineRestrictionProvider(core.RestrictionProvider):
+  """ A `core.RestrictionProvider` implementation which provides
+  `OffsetRestrictionTracker`."""
+  def _get_file_size(self, file_name):
+    with open(file_name, 'rb') as f:
+      f.seek(0, os.SEEK_END)
+      return f.tell()
+
+  def initial_restriction(self, element):
+    return OffsetRange(0, self._get_file_size(element['file_pattern']))
+
+  def create_tracker(self, restriction):
+    return OffsetRestrictionTracker(restriction)
+
+  def split(self, element, restriction):
+    return restriction.split(element['desired_chunk_size'])
+
+  def restriction_size(self, element, restriction):
+    return restriction.size()
+
+
+class SdfReadLine(core.DoFn):
+  """ An SplittableDoFn which reads lines from file and as output."""
+  def process(
+      self,
+      element,
+      restriction_tracker=core.DoFn.RestrictionParam(
+          SdfReadLineRestrictionProvider())):
+    with open(element['file_pattern'], 'rb') as f:
+      start = restriction_tracker.start_position()
+      f.seek(start)
+      if start > 0:
+        f.seek(-1, os.SEEK_CUR)
+        start -= 1
+        start += len(f.readline())
+      current = start
+      line = f.readline()
+      while restriction_tracker.try_claim(current):
+        if not line:
+          return
+        yield line.rstrip(b'\n')
+        current += len(line)
 
 Review comment:
   Won't this get off by one due to stripping the newline before adding the 
value to current? Possibly current = f.tell() would be more consistent. 
 
----------------------------------------------------------------
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:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 312373)
    Time Spent: 20m  (was: 10m)

> Provide a simple LineSource built on top of SDF
> -----------------------------------------------
>
>                 Key: BEAM-8143
>                 URL: https://issues.apache.org/jira/browse/BEAM-8143
>             Project: Beam
>          Issue Type: New Feature
>          Components: sdk-py-core
>            Reporter: Boyuan Zhang
>            Assignee: Boyuan Zhang
>            Priority: Major
>          Time Spent: 20m
>  Remaining Estimate: 0h
>




--
This message was sent by Atlassian Jira
(v8.3.2#803003)

Reply via email to