chamikaramj commented on a change in pull request #15126:
URL: https://github.com/apache/beam/pull/15126#discussion_r680588403



##########
File path: sdks/python/apache_beam/io/filebasedsource.py
##########
@@ -377,8 +382,13 @@ def process(self, element, *args, **kwargs):
     if not source_list:
       return
     source = source_list[0].source
+
+    data = []
     for record in source.read(range.new_tracker()):
-      yield record
+      data.append(record)

Review comment:
       I think this could result in OOMs. There's not size limit to the number 
of records that can be read here so putting all records in an array is not a 
viable option.
   
   May be just do a yield of file name and data ?

##########
File path: sdks/python/apache_beam/io/parquetio.py
##########
@@ -64,14 +64,21 @@
 class _ArrowTableToRowDictionaries(DoFn):
   """ A DoFn that consumes an Arrow table and yields a python dictionary for
   each row in the table."""
-  def process(self, table):
+  def process(self, table, with_filename=False):
+    if with_filename:
+      file_name = table[0]
+      table = table[1]
     num_rows = table.num_rows
     data_items = table.to_pydict().items()
+    rows = []

Review comment:
       Ditto. Adding all records to an array could result in OOMs.

##########
File path: sdks/python/apache_beam/io/textio_test.py
##########
@@ -582,6 +582,16 @@ def test_read_all_many_file_patterns(self):
           [pattern1, pattern2, pattern3]) | 'ReadAll' >> ReadAllFromText()
       assert_that(pcoll, equal_to(expected_data))
 
+  def test_read_all_with_filename(self):
+    pattern, expected_data = write_pattern([5, 3], return_filenames=True)
+    assert len(expected_data) == 8
+
+    with TestPipeline() as pipeline:
+      pcoll = pipeline \
+              | 'Create' >> Create([pattern]) \

Review comment:
       Nit: I think using paranthtesis for formatting is preferred over "\".




-- 
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.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to