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

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

                Author: ASF GitHub Bot
            Created on: 10/Dec/18 13:46
            Start Date: 10/Dec/18 13:46
    Worklog Time Spent: 10m 
      Work Description: robertwb closed pull request #6763: [BEAM-4444] Parquet 
IO for Python SDK
URL: https://github.com/apache/beam/pull/6763
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/sdks/python/apache_beam/io/__init__.py 
b/sdks/python/apache_beam/io/__init__.py
index 4cbb4458864b..6ce26451246b 100644
--- a/sdks/python/apache_beam/io/__init__.py
+++ b/sdks/python/apache_beam/io/__init__.py
@@ -26,6 +26,7 @@
 from apache_beam.io.iobase import Sink
 from apache_beam.io.iobase import Write
 from apache_beam.io.iobase import Writer
+from apache_beam.io.parquetio import *
 from apache_beam.io.textio import *
 from apache_beam.io.tfrecordio import *
 from apache_beam.io.range_trackers import *
diff --git a/sdks/python/apache_beam/io/parquetio.py 
b/sdks/python/apache_beam/io/parquetio.py
new file mode 100644
index 000000000000..2df72029dfdd
--- /dev/null
+++ b/sdks/python/apache_beam/io/parquetio.py
@@ -0,0 +1,469 @@
+#
+# 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.
+#
+"""``PTransforms`` for reading from and writing to Parquet files.
+
+Provides two read ``PTransform``\\s, ``ReadFromParquet`` and
+``ReadAllFromParquet``, that produces a ``PCollection`` of records.
+Each record of this ``PCollection`` will contain a single record read from
+a Parquet file. Records that are of simple types will be mapped into
+corresponding Python types. The actual parquet file operations are done by
+pyarrow. Source splitting is supported at row group granularity.
+
+Additionally, this module provides a write ``PTransform`` ``WriteToParquet``
+that can be used to write a given ``PCollection`` of Python objects to a
+Parquet file.
+"""
+from __future__ import absolute_import
+
+from functools import partial
+
+import pyarrow as pa
+import pyarrow.parquet as pq
+
+from apache_beam.io import filebasedsink
+from apache_beam.io import filebasedsource
+from apache_beam.io.filesystem import CompressionTypes
+from apache_beam.io.iobase import RangeTracker
+from apache_beam.io.iobase import Read
+from apache_beam.io.iobase import Write
+from apache_beam.transforms import PTransform
+
+__all__ = ['ReadFromParquet', 'ReadAllFromParquet', 'WriteToParquet']
+
+
+class ReadFromParquet(PTransform):
+  """A :class:`~apache_beam.transforms.ptransform.PTransform` for reading
+     Parquet files. This `PTransform` is currently experimental. No
+     backward-compatibility guarantees."""
+
+  def __init__(self, file_pattern=None, min_bundle_size=0,
+               validate=True, columns=None):
+    """Initializes :class:`ReadFromParquet`.
+
+    Uses source ``_ParquetSource`` to read a set of Parquet files defined by
+    a given file pattern.
+
+    If ``/mypath/myparquetfiles*`` is a file-pattern that points to a set of
+    Parquet files, a :class:`~apache_beam.pvalue.PCollection` for the records 
in
+    these Parquet files can be created in the following manner.
+
+    .. testcode::
+
+      with beam.Pipeline() as p:
+        records = p | 'Read' >> beam.io.ReadFromParquet('/mypath/mypqfiles*')
+
+    .. NOTE: We're not actually interested in this error; but if we get here,
+       it means that the way of calling this transform hasn't changed.
+
+    .. testoutput::
+      :hide:
+
+      Traceback (most recent call last):
+       ...
+      IOError: No files found based on the file pattern
+
+    Each element of this :class:`~apache_beam.pvalue.PCollection` will contain
+    a Python dictionary representing a single record. The keys will be of type
+    :class:`str` and named after their corresponding column names. The values
+    will be of the type defined in the corresponding Parquet schema. Records
+    that are of simple types will be mapped into corresponding Python types.
+    Records that are of complex types like list and struct will be mapped to
+    Python list and dictionary respectively. For more information on supported
+    types and schema, please see the pyarrow document.
+
+
+    Args:
+      file_pattern (str): the file glob to read
+      min_bundle_size (int): the minimum size in bytes, to be considered when
+        splitting the input into bundles.
+      validate (bool): flag to verify that the files exist during the pipeline
+        creation time.
+      columns (List[str]): list of columns that will be read from files.
+        A column name may be a prefix of a nested field, e.g. 'a' will select
+        'a.b', 'a.c', and 'a.d.e'
+"""
+    super(ReadFromParquet, self).__init__()
+    self._source = _create_parquet_source(
+        file_pattern,
+        min_bundle_size,
+        validate=validate,
+        columns=columns
+    )
+
+  def expand(self, pvalue):
+    return pvalue.pipeline | Read(self._source)
+
+  def display_data(self):
+    return {'source_dd': self._source}
+
+
+class ReadAllFromParquet(PTransform):
+  """A ``PTransform`` for reading ``PCollection`` of Parquet files.
+
+   Uses source ``_ParquetSource`` to read a ``PCollection`` of Parquet files or
+   file patterns and produce a ``PCollection`` of Parquet records. This
+   ``PTransform`` is currently experimental. No backward-compatibility
+   guarantees.
+  """
+
+  DEFAULT_DESIRED_BUNDLE_SIZE = 64 * 1024 * 1024  # 64MB
+
+  def __init__(self, min_bundle_size=0,
+               desired_bundle_size=DEFAULT_DESIRED_BUNDLE_SIZE,
+               columns=None,
+               label='ReadAllFiles'):
+    """Initializes ``ReadAllFromParquet``.
+
+    Args:
+      min_bundle_size: the minimum size in bytes, to be considered when
+                       splitting the input into bundles.
+      desired_bundle_size: the desired size in bytes, to be considered when
+                       splitting the input into bundles.
+      columns: list of columns that will be read from files. A column name
+                       may be a prefix of a nested field, e.g. 'a' will select
+                       'a.b', 'a.c', and 'a.d.e'
+    """
+    super(ReadAllFromParquet, self).__init__()
+    source_from_file = partial(
+        _create_parquet_source,
+        min_bundle_size=min_bundle_size,
+        columns=columns
+    )
+    self._read_all_files = filebasedsource.ReadAllFiles(
+        True, CompressionTypes.UNCOMPRESSED, desired_bundle_size,
+        min_bundle_size, source_from_file)
+
+    self.label = label
+
+  def expand(self, pvalue):
+    return pvalue | self.label >> self._read_all_files
+
+
+def _create_parquet_source(file_pattern=None,
+                           min_bundle_size=0,
+                           validate=False,
+                           columns=None):
+  return \
+    _ParquetSource(
+        file_pattern=file_pattern,
+        min_bundle_size=min_bundle_size,
+        validate=validate,
+        columns=columns
+    )
+
+
+class _ParquetUtils(object):
+  @staticmethod
+  def find_first_row_group_index(pf, start_offset):
+    for i in range(_ParquetUtils.get_number_of_row_groups(pf)):
+      row_group_start_offset = _ParquetUtils.get_offset(pf, i)
+      if row_group_start_offset >= start_offset:
+        return i
+    return -1
+
+  @staticmethod
+  def get_offset(pf, row_group_index):
+    first_column_metadata =\
+      pf.metadata.row_group(row_group_index).column(0)
+    if first_column_metadata.has_dictionary_page:
+      return first_column_metadata.dictionary_page_offset
+    else:
+      return first_column_metadata.data_page_offset
+
+  @staticmethod
+  def get_number_of_row_groups(pf):
+    return pf.metadata.num_row_groups
+
+
+class _ParquetSource(filebasedsource.FileBasedSource):
+  """A source for reading Parquet files.
+  """
+  def __init__(self, file_pattern, min_bundle_size, validate, columns):
+    super(_ParquetSource, self).__init__(
+        file_pattern=file_pattern,
+        min_bundle_size=min_bundle_size,
+        validate=validate
+    )
+    self._columns = columns
+
+  def read_records(self, file_name, range_tracker):
+    next_block_start = -1
+
+    def split_points_unclaimed(stop_position):
+      if next_block_start >= stop_position:
+        # Next block starts at or after the suggested stop position. Hence
+        # there will not be split points to be claimed for the range ending at
+        # suggested stop position.
+        return 0
+      return RangeTracker.SPLIT_POINTS_UNKNOWN
+
+    range_tracker.set_split_points_unclaimed_callback(split_points_unclaimed)
+
+    start_offset = range_tracker.start_position()
+    if start_offset is None:
+      start_offset = 0
+
+    with self.open_file(file_name) as f:
+      pf = pq.ParquetFile(f)
+
+      # find the first dictionary page (or data page if there's no dictionary
+      # page available) offset after the given start_offset. This offset is 
also
+      # the starting offset of any row group since the Parquet specification
+      # describes that the data pages always come first before the meta data in
+      # each row group.
+      index = _ParquetUtils.find_first_row_group_index(pf, start_offset)
+      if index != -1:
+        next_block_start = _ParquetUtils.get_offset(pf, index)
+      else:
+        next_block_start = range_tracker.stop_position()
+      number_of_row_groups = _ParquetUtils.get_number_of_row_groups(pf)
+
+      while range_tracker.try_claim(next_block_start):
+        table = pf.read_row_group(index, self._columns)
+
+        if index + 1 < number_of_row_groups:
+          index = index + 1
+          next_block_start = _ParquetUtils.get_offset(pf, index)
+        else:
+          next_block_start = range_tracker.stop_position()
+
+        num_rows = table.num_rows
+        data_items = table.to_pydict().items()
+        for n in range(num_rows):
+          row = {}
+          for column, values in data_items:
+            row[column] = values[n]
+          yield row
+
+
+class WriteToParquet(PTransform):
+  """A ``PTransform`` for writing parquet files.
+
+    This ``PTransform`` is currently experimental. No backward-compatibility
+    guarantees.
+  """
+
+  def __init__(self,
+               file_path_prefix,
+               schema,
+               row_group_buffer_size=64*1024*1024,
+               record_batch_size=1000,
+               codec='none',
+               use_deprecated_int96_timestamps=False,
+               file_name_suffix='',
+               num_shards=0,
+               shard_name_template=None,
+               mime_type='application/x-parquet'):
+    """Initialize a WriteToParquet transform.
+
+    Writes parquet files from a :class:`~apache_beam.pvalue.PCollection` of
+    records. Each record is a dictionary with keys of a string type that
+    represent column names. Schema must be specified like the example below.
+
+    .. testsetup::
+
+      import pyarrow
+
+    .. testcode::
+
+      with beam.Pipeline() as p:
+        records = p | 'Read' >> beam.Create(
+            [{'name': 'foo', 'age': 10}, {'name': 'bar', 'age': 20}]
+        )
+        _ = records | 'Write' >> beam.io.WriteToParquet('myoutput',
+            pyarrow.schema(
+                [('name', pyarrow.binary()), ('age', pyarrow.int64())]
+            )
+        )
+
+    For more information on supported types and schema, please see the pyarrow
+    document.
+
+    Args:
+      file_path_prefix: The file path to write to. The files written will begin
+        with this prefix, followed by a shard identifier (see num_shards), and
+        end in a common extension, if given by file_name_suffix. In most cases,
+        only this argument is specified and num_shards, shard_name_template, 
and
+        file_name_suffix use default values.
+      schema: The schema to use, as type of ``pyarrow.Schema``.
+      row_group_buffer_size: The byte size of the row group buffer. Note that
+        this size is for uncompressed data on the memory and normally much
+        bigger than the actual row group size written to a file.
+      record_batch_size: The number of records in each record batch. Record
+        batch is a basic unit used for storing data in the row group buffer.
+        A higher record batch size implies low granularity on a row group 
buffer
+        size. For configuring a row group size based on the number of records,
+        set ``row_group_buffer_size`` to 1 and use ``record_batch_size`` to
+        adjust the value.
+      codec: The codec to use for block-level compression. Any string supported
+        by the pyarrow specification is accepted.
+      use_deprecated_int96_timestamps: Write nanosecond resolution timestamps 
to
+        INT96 Parquet format. Defaults to False.
+      file_name_suffix: Suffix for the files written.
+      num_shards: The number of files (shards) used for output. If not set, the
+        service will decide on the optimal number of shards.
+        Constraining the number of shards is likely to reduce
+        the performance of a pipeline.  Setting this value is not recommended
+        unless you require a specific number of output files.
+      shard_name_template: A template string containing placeholders for
+        the shard number and shard count. When constructing a filename for a
+        particular shard number, the upper-case letters 'S' and 'N' are
+        replaced with the 0-padded shard number and shard count respectively.
+        This argument can be '' in which case it behaves as if num_shards was
+        set to 1 and only one file will be generated. The default pattern used
+        is '-SSSSS-of-NNNNN' if None is passed as the shard_name_template.
+      mime_type: The MIME type to use for the produced files, if the filesystem
+        supports specifying MIME types.
+
+    Returns:
+      A WriteToParquet transform usable for writing.
+    """
+    super(WriteToParquet, self).__init__()
+    self._sink = \
+      _create_parquet_sink(
+          file_path_prefix,
+          schema,
+          codec,
+          row_group_buffer_size,
+          record_batch_size,
+          use_deprecated_int96_timestamps,
+          file_name_suffix,
+          num_shards,
+          shard_name_template,
+          mime_type
+      )
+
+  def expand(self, pcoll):
+    return pcoll | Write(self._sink)
+
+  def display_data(self):
+    return {'sink_dd': self._sink}
+
+
+def _create_parquet_sink(file_path_prefix,
+                         schema,
+                         codec,
+                         row_group_buffer_size,
+                         record_batch_size,
+                         use_deprecated_int96_timestamps,
+                         file_name_suffix,
+                         num_shards,
+                         shard_name_template,
+                         mime_type):
+  return \
+    _ParquetSink(
+        file_path_prefix,
+        schema,
+        codec,
+        row_group_buffer_size,
+        record_batch_size,
+        use_deprecated_int96_timestamps,
+        file_name_suffix,
+        num_shards,
+        shard_name_template,
+        mime_type
+    )
+
+
+class _ParquetSink(filebasedsink.FileBasedSink):
+  """A sink for parquet files."""
+
+  def __init__(self,
+               file_path_prefix,
+               schema,
+               codec,
+               row_group_buffer_size,
+               record_batch_size,
+               use_deprecated_int96_timestamps,
+               file_name_suffix,
+               num_shards,
+               shard_name_template,
+               mime_type):
+    super(_ParquetSink, self).__init__(
+        file_path_prefix,
+        file_name_suffix=file_name_suffix,
+        num_shards=num_shards,
+        shard_name_template=shard_name_template,
+        coder=None,
+        mime_type=mime_type,
+        # Compression happens at the block level using the supplied codec, and
+        # not at the file level.
+        compression_type=CompressionTypes.UNCOMPRESSED)
+    self._schema = schema
+    self._codec = codec
+    self._row_group_buffer_size = row_group_buffer_size
+    self._use_deprecated_int96_timestamps = use_deprecated_int96_timestamps
+    self._buffer = [[] for _ in range(len(schema.names))]
+    self._buffer_size = record_batch_size
+    self._record_batches = []
+    self._record_batches_byte_size = 0
+    self._file_handle = None
+
+  def open(self, temp_path):
+    self._file_handle = super(_ParquetSink, self).open(temp_path)
+    return pq.ParquetWriter(
+        self._file_handle, self._schema, compression=self._codec,
+        use_deprecated_int96_timestamps=self._use_deprecated_int96_timestamps
+    )
+
+  def write_record(self, writer, value):
+    if len(self._buffer[0]) >= self._buffer_size:
+      self._flush_buffer()
+
+    if self._record_batches_byte_size >= self._row_group_buffer_size:
+      self._write_batches(writer)
+
+    # reorder the data in columnar format.
+    for i, n in enumerate(self._schema.names):
+      self._buffer[i].append(value[n])
+
+  def close(self, writer):
+    if len(self._buffer[0]) > 0:
+      self._flush_buffer()
+    if self._record_batches_byte_size > 0:
+      self._write_batches(writer)
+
+    writer.close()
+    if self._file_handle:
+      self._file_handle.close()
+      self._file_handle = None
+
+  def display_data(self):
+    res = super(_ParquetSink, self).display_data()
+    res['codec'] = str(self._codec)
+    res['schema'] = str(self._schema)
+    res['row_group_buffer_size'] = str(self._row_group_buffer_size)
+    return res
+
+  def _write_batches(self, writer):
+    table = pa.Table.from_batches(self._record_batches)
+    self._record_batches = []
+    self._record_batches_byte_size = 0
+    writer.write_table(table)
+
+  def _flush_buffer(self):
+    arrays = [[] for _ in range(len(self._schema.names))]
+    for x, y in enumerate(self._buffer):
+      arrays[x] = pa.array(y, type=self._schema.types[x])
+      self._buffer[x] = []
+    rb = pa.RecordBatch.from_arrays(arrays, self._schema.names)
+    self._record_batches.append(rb)
+    size = 0
+    for x in arrays:
+      for b in x.buffers():
+        size = size + b.size
+    self._record_batches_byte_size = self._record_batches_byte_size + size
diff --git a/sdks/python/apache_beam/io/parquetio_it_test.py 
b/sdks/python/apache_beam/io/parquetio_it_test.py
new file mode 100644
index 000000000000..19344e8881fa
--- /dev/null
+++ b/sdks/python/apache_beam/io/parquetio_it_test.py
@@ -0,0 +1,175 @@
+#
+# 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 __future__ import division
+
+import logging
+import string
+import sys
+import unittest
+from collections import Counter
+
+import pyarrow as pa
+from nose.plugins.attrib import attr
+
+from apache_beam import Create
+from apache_beam import DoFn
+from apache_beam import FlatMap
+from apache_beam import Flatten
+from apache_beam import Map
+from apache_beam import ParDo
+from apache_beam import Reshuffle
+from apache_beam.io.filesystems import FileSystems
+from apache_beam.io.parquetio import ReadAllFromParquet
+from apache_beam.io.parquetio import WriteToParquet
+from apache_beam.testing.test_pipeline import TestPipeline
+from apache_beam.testing.util import BeamAssertException
+from apache_beam.transforms import CombineGlobally
+from apache_beam.transforms.combiners import Count
+
+
+class TestParquetIT(unittest.TestCase):
+
+  @classmethod
+  def setUpClass(cls):
+    # Method has been renamed in Python 3
+    if sys.version_info[0] < 3:
+      cls.assertCountEqual = cls.assertItemsEqual
+
+  def setUp(self):
+    pass
+
+  def tearDown(self):
+    pass
+
+  SCHEMA = pa.schema([
+      ('name', pa.binary()),
+      ('favorite_number', pa.int64()),
+      ('favorite_color', pa.binary())
+  ])
+
+  @attr('IT')
+  def test_parquetio_it(self):
+    file_prefix = "parquet_it_test"
+    init_size = 10
+    data_size = 20000
+    p = TestPipeline(is_integration_test=True)
+    pcol = self._generate_data(
+        p, file_prefix, init_size, data_size)
+    self._verify_data(pcol, init_size, data_size)
+    result = p.run()
+    result.wait_until_finish()
+
+  @staticmethod
+  def _sum_verifier(init_size, data_size, x):
+    expected = sum(range(data_size)) * init_size
+    if x != expected:
+      raise BeamAssertException(
+          "incorrect sum: expected(%d) actual(%d)" % (expected, x)
+      )
+    return []
+
+  @staticmethod
+  def _count_verifier(init_size, data_size, x):
+    name, count = x[0], x[1]
+    counter = Counter(
+        [string.ascii_uppercase[x%26] for x in range(0, data_size*4, 4)]
+    )
+    expected_count = counter[name[0]] * init_size
+    if count != expected_count:
+      raise BeamAssertException(
+          "incorrect count(%s): expected(%d) actual(%d)" % (
+              name, expected_count, count
+          )
+      )
+    return []
+
+  def _verify_data(self, pcol, init_size, data_size):
+    read = pcol | 'read' >> ReadAllFromParquet()
+    v1 = (read
+          | 'get_number' >> Map(lambda x: x['number'])
+          | 'sum_globally' >> CombineGlobally(sum)
+          | 'validate_number' >> FlatMap(
+              lambda x: TestParquetIT._sum_verifier(init_size, data_size, x)
+          )
+         )
+    v2 = (read
+          | 'make_pair' >> Map(lambda x: (x['name'], x['number']))
+          | 'count_per_key' >> Count.PerKey()
+          | 'validate_name' >> FlatMap(
+              lambda x: TestParquetIT._count_verifier(init_size, data_size, x)
+          )
+         )
+    _ = ((v1, v2, pcol)
+         | 'flatten' >> Flatten()
+         | 'reshuffle' >> Reshuffle()
+         | 'cleanup' >> Map(lambda x: FileSystems.delete([x]))
+        )
+
+  def _generate_data(
+      self, p, output_prefix, init_size, data_size):
+    init_data = [x for x in range(init_size)]
+
+    lines = (p
+             | 'create' >> Create(init_data)
+             | 'produce' >> ParDo(ProducerFn(data_size))
+            )
+
+    schema = pa.schema([
+        ('name', pa.binary()),
+        ('number', pa.int64())
+    ])
+
+    files = lines | 'write' >> WriteToParquet(
+        output_prefix,
+        schema,
+        codec='snappy',
+        file_name_suffix='.parquet'
+    )
+
+    return files
+
+
+class ProducerFn(DoFn):
+  def __init__(self, number):
+    super(ProducerFn, self).__init__()
+    self._number = number
+    self._string_index = 0
+    self._number_index = 0
+
+  def process(self, element):
+    self._string_index = 0
+    self._number_index = 0
+    for _ in range(self._number):
+      yield {'name': self.get_string(4), 'number': self.get_int()}
+
+  def get_string(self, length):
+    s = []
+    for _ in range(length):
+      s.append(string.ascii_uppercase[self._string_index])
+      self._string_index = (self._string_index + 1) % 26
+    return ''.join(s)
+
+  def get_int(self):
+    i = self._number_index
+    self._number_index = self._number_index + 1
+    return i
+
+
+if __name__ == '__main__':
+  logging.getLogger().setLevel(logging.INFO)
+  unittest.main()
diff --git a/sdks/python/apache_beam/io/parquetio_test.py 
b/sdks/python/apache_beam/io/parquetio_test.py
new file mode 100644
index 000000000000..e42e6a73344e
--- /dev/null
+++ b/sdks/python/apache_beam/io/parquetio_test.py
@@ -0,0 +1,453 @@
+#
+# 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 json
+import logging
+import os
+import shutil
+import sys
+import tempfile
+import unittest
+
+import hamcrest as hc
+import pandas
+import pyarrow as pa
+import pyarrow.lib as pl
+import pyarrow.parquet as pq
+from parameterized import param
+from parameterized import parameterized
+
+from apache_beam import Create
+from apache_beam import Map
+from apache_beam.io import filebasedsource
+from apache_beam.io import source_test_utils
+from apache_beam.io.iobase import RangeTracker
+from apache_beam.io.parquetio import ReadAllFromParquet
+from apache_beam.io.parquetio import ReadFromParquet
+from apache_beam.io.parquetio import WriteToParquet
+from apache_beam.io.parquetio import _create_parquet_sink
+from apache_beam.io.parquetio import _create_parquet_source
+from apache_beam.testing.test_pipeline import TestPipeline
+from apache_beam.testing.util import assert_that
+from apache_beam.testing.util import equal_to
+from apache_beam.transforms.display import DisplayData
+from apache_beam.transforms.display_test import DisplayDataItemMatcher
+
+
+class TestParquet(unittest.TestCase):
+
+  @classmethod
+  def setUpClass(cls):
+    # Method has been renamed in Python 3
+    if sys.version_info[0] < 3:
+      cls.assertCountEqual = cls.assertItemsEqual
+
+  def setUp(self):
+    # Reducing the size of thread pools. Without this test execution may fail 
in
+    # environments with limited amount of resources.
+    filebasedsource.MAX_NUM_THREADS_FOR_SIZE_ESTIMATION = 2
+    self.temp_dir = tempfile.mkdtemp()
+
+  def tearDown(self):
+    shutil.rmtree(self.temp_dir)
+
+  RECORDS = [{'name': 'Thomas',
+              'favorite_number': 1,
+              'favorite_color': 'blue'}, {'name': 'Henry',
+                                          'favorite_number': 3,
+                                          'favorite_color': 'green'},
+             {'name': 'Toby',
+              'favorite_number': 7,
+              'favorite_color': 'brown'}, {'name': 'Gordon',
+                                           'favorite_number': 4,
+                                           'favorite_color': 'blue'},
+             {'name': 'Emily',
+              'favorite_number': -1,
+              'favorite_color': 'Red'}, {'name': 'Percy',
+                                         'favorite_number': 6,
+                                         'favorite_color': 'Green'}]
+
+  SCHEMA = pa.schema([
+      ('name', pa.binary()),
+      ('favorite_number', pa.int64()),
+      ('favorite_color', pa.binary())
+  ])
+
+  SCHEMA96 = pa.schema([
+      ('name', pa.binary()),
+      ('favorite_number', pa.timestamp('ns')),
+      ('favorite_color', pa.binary())
+  ])
+
+  def _record_to_columns(self, records, schema):
+    col_list = []
+    for n in schema.names:
+      column = []
+      for r in records:
+        column.append(r[n])
+      col_list.append(column)
+    return col_list
+
+  def _write_data(self,
+                  directory=None,
+                  schema=None,
+                  prefix=tempfile.template,
+                  row_group_size=1000,
+                  codec='none',
+                  count=len(RECORDS)):
+    if schema is None:
+      schema = self.SCHEMA
+
+    if directory is None:
+      directory = self.temp_dir
+
+    with tempfile.NamedTemporaryFile(
+        delete=False, dir=directory, prefix=prefix) as f:
+      len_records = len(self.RECORDS)
+      data = []
+      for i in range(count):
+        data.append(self.RECORDS[i % len_records])
+      col_data = self._record_to_columns(data, schema)
+      col_array = [
+          pa.array(c, schema.types[cn]) for cn, c in enumerate(col_data)
+      ]
+      table = pa.Table.from_arrays(col_array, schema.names)
+      pq.write_table(
+          table, f, row_group_size=row_group_size, compression=codec,
+          use_deprecated_int96_timestamps=True
+      )
+
+      return f.name
+
+  def _write_pattern(self, num_files):
+    assert num_files > 0
+    temp_dir = tempfile.mkdtemp(dir=self.temp_dir)
+
+    for _ in range(num_files):
+      self._write_data(directory=temp_dir, prefix='mytemp')
+
+    return temp_dir + os.path.sep + 'mytemp*'
+
+  def _run_parquet_test(self, pattern, columns, desired_bundle_size,
+                        perform_splitting, expected_result):
+    source = _create_parquet_source(pattern, columns=columns)
+    if perform_splitting:
+      assert desired_bundle_size
+      sources_info = [
+          (split.source, split.start_position, split.stop_position)
+          for split in source.split(desired_bundle_size=desired_bundle_size)
+      ]
+      if len(sources_info) < 2:
+        raise ValueError('Test is trivial. Please adjust it so that at least '
+                         'two splits get generated')
+
+      source_test_utils.assert_sources_equal_reference_source(
+          (source, None, None), sources_info)
+    else:
+      read_records = source_test_utils.read_from_source(source, None, None)
+      self.assertCountEqual(expected_result, read_records)
+
+  def test_read_without_splitting(self):
+    file_name = self._write_data()
+    expected_result = self.RECORDS
+    self._run_parquet_test(file_name, None, None, False, expected_result)
+
+  def test_read_with_splitting(self):
+    file_name = self._write_data()
+    expected_result = self.RECORDS
+    self._run_parquet_test(file_name, None, 100, True, expected_result)
+
+  def test_source_display_data(self):
+    file_name = 'some_parquet_source'
+    source = \
+        _create_parquet_source(
+            file_name,
+            validate=False
+        )
+    dd = DisplayData.create_from(source)
+
+    expected_items = [
+        DisplayDataItemMatcher('compression', 'auto'),
+        DisplayDataItemMatcher('file_pattern', file_name)]
+    hc.assert_that(dd.items, hc.contains_inanyorder(*expected_items))
+
+  def test_read_display_data(self):
+    file_name = 'some_parquet_source'
+    read = \
+      ReadFromParquet(
+          file_name,
+          validate=False)
+    dd = DisplayData.create_from(read)
+
+    expected_items = [
+        DisplayDataItemMatcher('compression', 'auto'),
+        DisplayDataItemMatcher('file_pattern', file_name)]
+    hc.assert_that(dd.items, hc.contains_inanyorder(*expected_items))
+
+  def test_sink_display_data(self):
+    file_name = 'some_parquet_sink'
+    sink = _create_parquet_sink(
+        file_name,
+        self.SCHEMA,
+        'none',
+        1024*1024,
+        1000,
+        False,
+        '.end',
+        0,
+        None,
+        'application/x-parquet')
+    dd = DisplayData.create_from(sink)
+    expected_items = [
+        DisplayDataItemMatcher(
+            'schema',
+            str(self.SCHEMA)),
+        DisplayDataItemMatcher(
+            'file_pattern',
+            'some_parquet_sink-%(shard_num)05d-of-%(num_shards)05d.end'),
+        DisplayDataItemMatcher(
+            'codec',
+            'none'),
+        DisplayDataItemMatcher(
+            'row_group_buffer_size',
+            str(1024*1024)),
+        DisplayDataItemMatcher(
+            'compression',
+            'uncompressed')]
+    hc.assert_that(dd.items, hc.contains_inanyorder(*expected_items))
+
+  def test_write_display_data(self):
+    file_name = 'some_parquet_sink'
+    write = WriteToParquet(file_name, self.SCHEMA)
+    dd = DisplayData.create_from(write)
+    expected_items = [
+        DisplayDataItemMatcher(
+            'codec',
+            'none'),
+        DisplayDataItemMatcher(
+            'schema',
+            str(self.SCHEMA)),
+        DisplayDataItemMatcher(
+            'row_group_buffer_size',
+            str(64*1024*1024)),
+        DisplayDataItemMatcher(
+            'file_pattern',
+            'some_parquet_sink-%(shard_num)05d-of-%(num_shards)05d'),
+        DisplayDataItemMatcher(
+            'compression',
+            'uncompressed')]
+    hc.assert_that(dd.items, hc.contains_inanyorder(*expected_items))
+
+  def test_sink_transform_int96(self):
+    with tempfile.NamedTemporaryFile() as dst:
+      path = dst.name
+      with self.assertRaises(pl.ArrowInvalid):
+        with TestPipeline() as p:
+          _ = p \
+          | Create(self.RECORDS) \
+          | WriteToParquet(
+              path, self.SCHEMA96, num_shards=1, shard_name_template='')
+
+  def test_sink_transform(self):
+    with tempfile.NamedTemporaryFile() as dst:
+      path = dst.name
+      with TestPipeline() as p:
+        _ = p \
+        | Create(self.RECORDS) \
+        | WriteToParquet(
+            path, self.SCHEMA, num_shards=1, shard_name_template='')
+      with TestPipeline() as p:
+        # json used for stable sortability
+        readback = \
+            p \
+            | ReadFromParquet(path) \
+            | Map(json.dumps)
+        assert_that(readback, equal_to([json.dumps(r) for r in self.RECORDS]))
+
+  @parameterized.expand([
+      param(compression_type='snappy'),
+      param(compression_type='gzip'),
+      param(compression_type='brotli'),
+      param(compression_type='lz4'),
+      param(compression_type='zstd')
+  ])
+  def test_sink_transform_compressed(self, compression_type):
+    with tempfile.NamedTemporaryFile() as dst:
+      path = dst.name
+      with TestPipeline() as p:
+        _ = p \
+        | Create(self.RECORDS) \
+        | WriteToParquet(
+            path, self.SCHEMA, codec=compression_type,
+            num_shards=1, shard_name_template='')
+      with TestPipeline() as p:
+        # json used for stable sortability
+        readback = \
+            p \
+            | ReadFromParquet(path + '*') \
+            | Map(json.dumps)
+        assert_that(readback, equal_to([json.dumps(r) for r in self.RECORDS]))
+
+  def test_read_reentrant(self):
+    file_name = self._write_data()
+    source = _create_parquet_source(file_name)
+    source_test_utils.assert_reentrant_reads_succeed((source, None, None))
+
+  def test_read_without_splitting_multiple_row_group(self):
+    file_name = self._write_data(count=12000)
+    expected_result = self.RECORDS * 2000
+    self._run_parquet_test(file_name, None, None, False, expected_result)
+
+  def test_read_with_splitting_multiple_row_group(self):
+    file_name = self._write_data(count=12000)
+    expected_result = self.RECORDS * 2000
+    self._run_parquet_test(file_name, None, 10000, True, expected_result)
+
+  def test_dynamic_work_rebalancing(self):
+    file_name = self._write_data(count=120, row_group_size=20)
+    source = _create_parquet_source(file_name)
+
+    splits = [
+        split
+        for split in source.split(desired_bundle_size=float('inf'))
+    ]
+    assert len(splits) == 1
+
+    source_test_utils.assert_split_at_fraction_exhaustive(
+        splits[0].source, splits[0].start_position, splits[0].stop_position
+    )
+
+  def test_min_bundle_size(self):
+    file_name = self._write_data(count=120, row_group_size=20)
+
+    source = _create_parquet_source(file_name, min_bundle_size=100*1024*1024)
+    splits = [
+        split for split in source.split(desired_bundle_size=1)
+    ]
+    self.assertEquals(len(splits), 1)
+
+    source = _create_parquet_source(file_name, min_bundle_size=0)
+    splits = [
+        split for split in source.split(desired_bundle_size=1)
+    ]
+    self.assertNotEquals(len(splits), 1)
+
+  def _convert_to_timestamped_record(self, record):
+    timestamped_record = record.copy()
+    timestamped_record['favorite_number'] =\
+      pandas.Timestamp(timestamped_record['favorite_number'])
+    return timestamped_record
+
+  def test_int96_type_conversion(self):
+    file_name = self._write_data(
+        count=120, row_group_size=20, schema=self.SCHEMA96)
+    expected_result = [
+        self._convert_to_timestamped_record(x) for x in self.RECORDS
+    ] * 20
+    self._run_parquet_test(file_name, None, None, False, expected_result)
+
+  def test_split_points(self):
+    file_name = self._write_data(count=12000, row_group_size=3000)
+    source = _create_parquet_source(file_name)
+
+    splits = [
+        split for split in source.split(desired_bundle_size=float('inf'))
+    ]
+    assert len(splits) == 1
+
+    range_tracker = splits[0].source.get_range_tracker(
+        splits[0].start_position, splits[0].stop_position)
+
+    split_points_report = []
+
+    for _ in splits[0].source.read(range_tracker):
+      split_points_report.append(range_tracker.split_points())
+
+    # There are a total of four row groups. Each row group has 3000 records.
+
+    # When reading records of the first group, range_tracker.split_points()
+    # should return (0, iobase.RangeTracker.SPLIT_POINTS_UNKNOWN)
+    self.assertEquals(
+        split_points_report[:10],
+        [(0, RangeTracker.SPLIT_POINTS_UNKNOWN)] * 10)
+
+    # When reading records of last group, range_tracker.split_points() should
+    # return (3, 1)
+    self.assertEquals(split_points_report[-10:], [(3, 1)] * 10)
+
+  def test_selective_columns(self):
+    file_name = self._write_data()
+    expected_result = [{'name': r['name']} for r in self.RECORDS]
+    self._run_parquet_test(file_name, ['name'], None, False, expected_result)
+
+  def test_sink_transform_multiple_row_group(self):
+    with tempfile.NamedTemporaryFile() as dst:
+      path = dst.name
+      with TestPipeline() as p:
+        # writing 623200 bytes of data
+        _ = p \
+        | Create(self.RECORDS * 4000) \
+        | WriteToParquet(
+            path, self.SCHEMA, num_shards=1, codec='none',
+            shard_name_template='', row_group_buffer_size=250000)
+      self.assertEqual(pq.read_metadata(path).num_row_groups, 3)
+
+  def test_read_all_from_parquet_single_file(self):
+    path = self._write_data()
+    with TestPipeline() as p:
+      assert_that(
+          p \
+          | Create([path]) \
+          | ReadAllFromParquet(),
+          equal_to(self.RECORDS))
+
+  def test_read_all_from_parquet_many_single_files(self):
+    path1 = self._write_data()
+    path2 = self._write_data()
+    path3 = self._write_data()
+    with TestPipeline() as p:
+      assert_that(
+          p \
+          | Create([path1, path2, path3]) \
+          | ReadAllFromParquet(),
+          equal_to(self.RECORDS * 3))
+
+  def test_read_all_from_parquet_file_pattern(self):
+    file_pattern = self._write_pattern(5)
+    with TestPipeline() as p:
+      assert_that(
+          p \
+          | Create([file_pattern]) \
+          | ReadAllFromParquet(),
+          equal_to(self.RECORDS * 5))
+
+  def test_read_all_from_parquet_many_file_patterns(self):
+    file_pattern1 = self._write_pattern(5)
+    file_pattern2 = self._write_pattern(2)
+    file_pattern3 = self._write_pattern(3)
+    with TestPipeline() as p:
+      assert_that(
+          p \
+          | Create([file_pattern1, file_pattern2, file_pattern3]) \
+          | ReadAllFromParquet(),
+          equal_to(self.RECORDS * 10))
+
+
+if __name__ == '__main__':
+  logging.getLogger().setLevel(logging.INFO)
+  unittest.main()
diff --git a/sdks/python/setup.py b/sdks/python/setup.py
index 7ebecd32664d..bd6888d0eb84 100644
--- a/sdks/python/setup.py
+++ b/sdks/python/setup.py
@@ -113,6 +113,7 @@ def get_version():
     'crcmod>=1.7,<2.0',
     'dill>=0.2.6,<=0.2.8.2',
     'fastavro>=0.21.4,<0.22',
+    'pyarrow>=0.11.1,<0.12.0',
     'grpcio>=1.8,<2',
     'hdfs>=2.1.0,<3.0.0',
     'httplib2>=0.8,<=0.11.3',
@@ -131,6 +132,7 @@ def get_version():
 
 REQUIRED_TEST_PACKAGES = [
     'nose>=1.3.7',
+    'pandas>=0.23.4,<0.24',
     'parameterized>=0.6.0,<0.7.0',
     'numpy>=1.14.3,<2',
     'pyhamcrest>=1.9,<2.0',


 

----------------------------------------------------------------
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:
[email protected]


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

    Worklog Id:     (was: 173580)
    Time Spent: 12h  (was: 11h 50m)

> Parquet IO for Python SDK
> -------------------------
>
>                 Key: BEAM-4444
>                 URL: https://issues.apache.org/jira/browse/BEAM-4444
>             Project: Beam
>          Issue Type: New Feature
>          Components: sdk-py-core
>            Reporter: Bruce Arctor
>            Assignee: Heejong Lee
>            Priority: Major
>          Time Spent: 12h
>  Remaining Estimate: 0h
>
> Add Parquet Support for the Python SDK.



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

Reply via email to