Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[BEAM-10894] Basic CSV reading and writing. #12841

Merged
merged 12 commits into from Sep 18, 2020
6 changes: 6 additions & 0 deletions sdks/python/apache_beam/dataframe/frames.py
Expand Up @@ -20,6 +20,7 @@

from apache_beam.dataframe import expressions
from apache_beam.dataframe import frame_base
from apache_beam.dataframe import io
from apache_beam.dataframe import partitionings


Expand Down Expand Up @@ -764,6 +765,11 @@ def unstack(self, *args, **kwargs):
requires_partition_by=partitionings.Index(),
preserves_partition_by=partitionings.Index())

@frame_base.args_to_kwargs(pd.DataFrame)
@frame_base.populate_defaults(pd.DataFrame)
def to_csv(self, path_or_buf, **kwargs):
return io.write_csv(self, path_or_buf, **kwargs)

for meth in ('filter', ):
setattr(DeferredDataFrame, meth, frame_base._elementwise_method(meth))

Expand Down
180 changes: 180 additions & 0 deletions sdks/python/apache_beam/dataframe/io.py
@@ -0,0 +1,180 @@
#
# 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 io import BytesIO
from io import StringIO
from io import TextIOWrapper

import pandas as pd

import apache_beam as beam
from apache_beam import io
from apache_beam.dataframe import frame_base
from apache_beam.io import fileio


def read_csv(path, *args, **kwargs):
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please add pydocs for public API here (or add a TODO/JIRA for this).

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The idea here is to mirror the Pandas APIs. I suppose I should reference them at least.

"""Emulates `pd.read_csv` from Pandas, but as a Beam PTransform.

Use this as

df = p | beam.dataframe.io.read_csv(...)

to get a deferred Beam dataframe representing the contents of the file.
"""
return _ReadFromPandas(pd.read_csv, path, args, kwargs)

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is there a reason to provide these as methods instead of transforms (similar to other IO connectors) ?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Mirroring the Pandas APIs. The _ReadFromPandas will be widely shared.


def write_csv(df, path, *args, **kwargs):
from apache_beam.dataframe import convert
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Import at top (here and below) ?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I was running into circular import issues. Will add a comment.

# TODO(roberwb): Amortize the computation for multiple writes?
return convert.to_pcollection(df) | _WriteToPandas(
pd.DataFrame.to_csv, path, args, kwargs, incremental=True, binary=False)


def _prefix_range_index_with(prefix, df):
if isinstance(df.index, pd.RangeIndex):
return df.set_index(prefix + df.index.map(str).astype(str))
else:
return df


class _ReadFromPandas(beam.PTransform):
def __init__(self, reader, path, args, kwargs):
if not isinstance(path, str):
raise frame_base.WontImplementError('non-deferred')
self.reader = reader
self.path = path
self.args = args
self.kwargs = kwargs

def expand(self, root):
# TODO(robertwb): Handle streaming (with explicit schema).
paths_pcoll = root | beam.Create([self.path])
first = io.filesystems.FileSystems.match([self.path],
limits=[1
])[0].metadata_list[0].path
with io.filesystems.FileSystems.open(first) as handle:
df = next(self.reader(handle, *self.args, chunksize=100, **self.kwargs))

pcoll = (
paths_pcoll
| fileio.MatchFiles(self.path)
| fileio.ReadMatches()
| beam.ParDo(_ReadFromPandasDoFn(self.reader, self.args, self.kwargs)))
from apache_beam.dataframe import convert
return convert.to_dataframe(
pcoll, proxy=_prefix_range_index_with(':', df[:0]))


# TODO(robertwb): Actually make an SDF.
class _ReadFromPandasDoFn(beam.DoFn):
def __init__(self, reader, args, kwargs):
# avoid pickling issues
self.reader = reader.__name__
self.args = args
self.kwargs = kwargs

def process(self, readable_file):
reader = getattr(pd, self.reader)
with readable_file.open() as handle:
for df in reader(handle, *self.args, chunksize=100, **self.kwargs):
yield _prefix_range_index_with(readable_file.metadata.path + ':', df)


class _WriteToPandas(beam.PTransform):
def __init__(
self, writer, path, args, kwargs, incremental=False, binary=True):
self.writer = writer
self.path = path
self.args = args
self.kwargs = kwargs
self.incremental = incremental
self.binary = binary

def expand(self, pcoll):
dir, name = io.filesystems.FileSystems.split(self.path)
return pcoll | fileio.WriteToFiles(
path=dir,
file_naming=fileio.default_file_naming(name),
sink=_WriteToPandasFileSink(
self.writer, self.args, self.kwargs, self.incremental, self.binary))


class _WriteToPandasFileSink(fileio.FileSink):
def __init__(self, writer, args, kwargs, incremental, binary):
self.writer = writer
self.args = args
self.kwargs = kwargs
self.incremental = incremental
self.binary = binary
self.StringOrBytesIO = BytesIO if binary else StringIO
if incremental:
self.write = self.write_record_incremental
self.flush = self.close_incremental
else:
self.write = self.buffer_record
self.flush = self.flush_buffer

def open(self, file_handle):
self.buffer = []
self.empty = self.header = self.footer = None
if not self.binary:
file_handle = TextIOWrapper(file_handle)
self.file_handle = file_handle

def write_to(self, df, file_handle=None):
non_none_handle = file_handle or self.StringOrBytesIO()
self.writer(df, non_none_handle, *self.args, **self.kwargs)
if file_handle is None:
return non_none_handle.getvalue()

def write_record_incremental(self, value):
if self.empty is None:
self.empty = self.write_to(value[:0])
if self.header is None and len(value):
one_row = self.write_to(value[:1])
for ix, c in enumerate(self.empty):
if one_row[ix] != c:
break
else:
ix = len(self.empty)
self.header = self.empty[:ix]
self.footer = self.empty[ix:]
self.file_handle.write(self.header)

if len(value):
# IDEA(robertwb): Construct a "truncating" stream wrapper to avoid the
# in-memory copy.
rows = self.write_to(value)
self.file_handle.write(rows[len(self.header):-len(self.footer) or None])

def close_incremental(self):
if self.footer is not None:
self.file_handle.write(self.footer)
elif self.empty is not None:
self.file_handle.write(self.empty)
self.file_handle.flush()

def buffer_record(self, value):
self.buffer.append(value)

def flush_buffer(self, file_handle):
if self.buffer:
self.write_to(pd.concat(self.buffer), file_handle)
self.file_handle.flush()
67 changes: 67 additions & 0 deletions sdks/python/apache_beam/dataframe/io_test.py
@@ -0,0 +1,67 @@
#
# 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 glob
import os
import shutil
import sys
import tempfile
import unittest

import apache_beam as beam
from apache_beam.dataframe import io


class IOTest(unittest.TestCase):
def setUp(self):
self._temp_roots = []

def tearDown(self):
for root in self._temp_roots:
shutil.rmtree(root)

def temp_dir(self, files=None):
dir = tempfile.mkdtemp(prefix='beam-test')
self._temp_roots.append(dir)
if files:
for name, contents in files.items():
with open(os.path.join(dir, name), 'w') as fout:
fout.write(contents)
return dir + os.sep

def read_all_lines(self, pattern):
for path in glob.glob(pattern):
with open(path) as fin:
# TODO(Py3): yield from
for line in fin:
yield line.rstrip('\n')

@unittest.skipIf(sys.version_info[0] < 3, 'unicode issues')
def test_write_csv(self):
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

test_read_csv (seems like this is testing read_csv) ?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This tests tests both read and write.

input = self.temp_dir({'1.csv': 'a,b\n1,2\n', '2.csv': 'a,b\n3,4\n'})
output = self.temp_dir()
with beam.Pipeline() as p:
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Add a test for write_cvs as well ?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This tests tests both read and write. Updated name.

df = p | io.read_csv(input + '*.csv')
df['c'] = df.a + df.b
df.to_csv(output + 'out.csv', index=False)
self.assertCountEqual(['a,b,c', '1,2,3', '3,4,7'],
set(self.read_all_lines(output + 'out.csv*')))


if __name__ == '__main__':
unittest.main()