[GitHub] [beam] robertwb commented on a change in pull request #10760: [BEAM-9545] Dataframe transforms

2020-03-20 Thread GitBox
robertwb commented on a change in pull request #10760: [BEAM-9545] Dataframe 
transforms
URL: https://github.com/apache/beam/pull/10760#discussion_r395900936
 
 

 ##
 File path: sdks/python/apache_beam/dataframe/transforms.py
 ##
 @@ -0,0 +1,246 @@
+#
+# 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 pandas as pd
+
+import apache_beam as beam
+from apache_beam import transforms
+from apache_beam.dataframe import expressions
+from apache_beam.dataframe import frame_base
+from apache_beam.dataframe import frames  # pylint: disable=unused-import
+
+
+class DataframeTransform(transforms.PTransform):
+  """A PTransform for applying function that takes and returns dataframes
+  to one or more PCollections.
+
+  For example, if pcoll is a PCollection of dataframes, one could write::
+
+  pcoll | DataframeTransform(lambda df: df.group_by('key').sum(), 
proxy=...)
+
+  To pass multiple PCollections, pass a tuple of PCollections wich will be
+  passed to the callable as positional arguments, or a dictionary of
+  PCollections, in which case they will be passed as keyword arguments.
+  """
+  def __init__(self, func, proxy):
+self._func = func
+self._proxy = proxy
+
+  def expand(self, input_pcolls):
+def wrap_as_dict(values):
+  if isinstance(values, dict):
+return values
+  elif isinstance(values, tuple):
+return dict(enumerate(values))
+  else:
+return {None: values}
+
+# TODO: Infer the proxy from the input schema.
+def proxy(key):
+  if key is None:
+return self._proxy
+  else:
+return self._proxy[key]
+
+# The input can be a dictionary, tuple, or plain PCollection.
+# Wrap as a dict for homogeneity.
+# TODO: Possibly inject batching here.
+input_dict = wrap_as_dict(input_pcolls)
+placeholders = {
+key: frame_base.DeferredFrame.wrap(
+expressions.PlaceholderExpression(proxy(key)))
+for key in input_dict.keys()
+}
+
+# The calling convention of the user-supplied func varies according to the
+# type of the input.
+if isinstance(input_pcolls, dict):
+  result_frames = self._func(**placeholders)
+elif isinstance(input_pcolls, tuple):
+  result_frames = self._func(
+  *(value for _, value in sorted(placeholders.items(
+else:
+  result_frames = self._func(placeholders[None])
+
+# Likewise the output may be a dict, tuple, or raw (deferred) Dataframe.
+result_dict = wrap_as_dict(result_frames)
+
+result_pcolls = self._apply_deferred_ops(
+{placeholders[key]._expr: pcoll
+ for key, pcoll in input_dict.items()},
+{key: df._expr
+ for key, df in result_dict.items()})
+
+# Convert the result back into a set of PCollections.
+if isinstance(result_frames, dict):
+  return result_pcolls
+elif isinstance(result_frames, tuple):
+  return tuple((value for _, value in sorted(result_pcolls.items(
+else:
+  return result_pcolls[None]
+
+  def _apply_deferred_ops(
+  self,
+  inputs,  # type: Dict[PlaceholderExpr, PCollection]
+  outputs,  # type: Dict[Any, Expression]
+  ):  # -> Dict[Any, PCollection]
+"""Construct a Beam graph that evaluates a set of expressions on a set of
+input PCollections.
+
+:param inputs: A mapping of placeholder expressions to PCollections.
+:param outputs: A mapping of keys to expressions defined in terms of the
+placeholders of inputs.
+
+Returns a dictionary whose keys are those of outputs, and whose values are
+PCollections corresponding to the values of outputs evaluated at the
+values of inputs.
+
+Logically, `_apply_deferred_ops({x: a, y: b}, {f: F(x, y), g: G(x, y)})`
+returns `{f: F(a, b), g: G(a, b)}`.
+"""
+class ComputeStage(beam.PTransform):
+  """A helper transform that computes a single stage of operations.
+  """
+  def __init__(self, stage):
+self.stage = stage
+
+  def default_label(self):
+return '%s:%s' % (self.stage.ops, id(self))
+
+  def expand(self, pcolls):
+if self.stage.is_grouping:
+  # Arrange such that 

[GitHub] [beam] robertwb commented on a change in pull request #10760: [BEAM-9545] Dataframe transforms

2020-03-19 Thread GitBox
robertwb commented on a change in pull request #10760: [BEAM-9545] Dataframe 
transforms
URL: https://github.com/apache/beam/pull/10760#discussion_r395253978
 
 

 ##
 File path: sdks/python/apache_beam/dataframe/transforms.py
 ##
 @@ -0,0 +1,244 @@
+#
+# 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 pandas as pd
+
+import apache_beam as beam
+from apache_beam import transforms
+from apache_beam.dataframe import expressions
+from apache_beam.dataframe import frame_base
+
+
+class DataframeTransform(transforms.PTransform):
+  """A PTransform for applying function that takes and returns dataframes
+  to one or more PCollections.
+
+  For example, if pcoll is a PCollection of dataframes, one could write::
+
+  pcoll | DataframeTransform(lambda df: df.group_by('key').sum())
+
+  To pass multiple PCollections, pass a tuple of PCollections wich will be
+  passed to the callable as positional arguments, or a dictionary of
+  PCollections, in which case they will be passed as keyword arguments.
+  """
+  def __init__(self, func, proxy):
+self._func = func
+self._proxy = proxy
+
+  def expand(self, input_pcolls):
+def wrap_as_dict(values):
+  if isinstance(values, dict):
+return dict
+  elif isinstance(values, tuple):
+return dict(enumerate(values))
+  else:
+return {None: values}
+
+# TODO: Infer the proxy from the input schema.
+def proxy(key):
+  if key is None:
+return self._proxy
+  else:
+return self._proxy[key]
+
+# The input can be a dictionary, tuple, or plain PCollection.
+# Wrap as a dict for homogeneity.
+# TODO: Possibly inject batching here.
+input_dict = wrap_as_dict(input_pcolls)
+placeholders = {
+key: frame_base.DeferredFrame.wrap(
+expressions.PlaceholderExpression(proxy(key)))
+for key in input_dict.keys()
+}
+
+# The calling convention of the user-supplied func varies according to the
+# type of the input.
+if isinstance(input_pcolls, dict):
+  result_frames = self._func(**placeholders)
+elif isinstance(input_pcolls, tuple):
+  self._func(*(value for _, value in sorted(placeholders.items(
+else:
+  result_frames = self._func(placeholders[None])
+
+# Likewise the output may be a dict, tuple, or raw (deferred) Dataframe.
+result_dict = wrap_as_dict(result_frames)
+
+result_pcolls = self._apply_deferred_ops(
+{placeholders[key]._expr: pcoll
+ for key, pcoll in input_dict.items()},
+{key: df._expr
+ for key, df in result_dict.items()})
+
+# Convert the result back into a set of PCollections.
+if isinstance(result_frames, dict):
+  return result_pcolls
+elif isinstance(result_frames, tuple):
+  return tuple(*(value for _, value in sorted(result_pcolls.items(
+else:
+  return result_pcolls[None]
+
+  def _apply_deferred_ops(
+  self,
+  inputs,  # type: Dict[PlaceholderExpr, PCollection]
+  outputs,  # type: Dict[Any, Expression]
+  ):  # -> Dict[Any, PCollection]
+"""Construct a Beam graph that evaluates a set of expressions on a set of
+input PCollections.
+
+:param inputs: A mapping of placeholder expressions to PCollections.
+:param outputs: A mapping of keys to expressions defined in terms of the
+placeholders of inputs.
+
+Returns a dictionary whose keys are those of outputs, and whose values are
+PCollections corresponding to the values of outputs evaluated at the
+values of inputs.
+
+Logically, `_apply_deferred_ops({x: a, y: b}, {f: F(x, y), g: G(x, y)})`
+returns `{f: F(a, b), g: G(a, b)}`.
+"""
+class ComputeStage(beam.PTransform):
+  """A helper transform that computes a single stage of operations.
+  """
+  def __init__(self, stage):
+self.stage = stage
+
+  def default_label(self):
+return '%s:%s' % (self.stage.ops, id(self))
+
+  def expand(self, pcolls):
+if self.stage.is_grouping:
+  # Arrange such that partitioned_pcoll is properly partitioned.
+  input_pcolls = {
+  k: pcoll | 'Flat%s' % k >> 

[GitHub] [beam] robertwb commented on a change in pull request #10760: [BEAM-9545] Dataframe transforms

2020-03-19 Thread GitBox
robertwb commented on a change in pull request #10760: [BEAM-9545] Dataframe 
transforms
URL: https://github.com/apache/beam/pull/10760#discussion_r395254391
 
 

 ##
 File path: sdks/python/apache_beam/dataframe/transforms.py
 ##
 @@ -0,0 +1,244 @@
+#
+# 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 pandas as pd
+
+import apache_beam as beam
+from apache_beam import transforms
+from apache_beam.dataframe import expressions
+from apache_beam.dataframe import frame_base
+
+
+class DataframeTransform(transforms.PTransform):
+  """A PTransform for applying function that takes and returns dataframes
+  to one or more PCollections.
+
+  For example, if pcoll is a PCollection of dataframes, one could write::
+
+  pcoll | DataframeTransform(lambda df: df.group_by('key').sum())
+
+  To pass multiple PCollections, pass a tuple of PCollections wich will be
+  passed to the callable as positional arguments, or a dictionary of
+  PCollections, in which case they will be passed as keyword arguments.
+  """
+  def __init__(self, func, proxy):
+self._func = func
+self._proxy = proxy
+
+  def expand(self, input_pcolls):
+def wrap_as_dict(values):
+  if isinstance(values, dict):
+return dict
+  elif isinstance(values, tuple):
+return dict(enumerate(values))
+  else:
+return {None: values}
+
+# TODO: Infer the proxy from the input schema.
+def proxy(key):
+  if key is None:
+return self._proxy
+  else:
+return self._proxy[key]
+
+# The input can be a dictionary, tuple, or plain PCollection.
+# Wrap as a dict for homogeneity.
+# TODO: Possibly inject batching here.
+input_dict = wrap_as_dict(input_pcolls)
+placeholders = {
+key: frame_base.DeferredFrame.wrap(
+expressions.PlaceholderExpression(proxy(key)))
+for key in input_dict.keys()
+}
+
+# The calling convention of the user-supplied func varies according to the
+# type of the input.
+if isinstance(input_pcolls, dict):
+  result_frames = self._func(**placeholders)
+elif isinstance(input_pcolls, tuple):
+  self._func(*(value for _, value in sorted(placeholders.items(
+else:
+  result_frames = self._func(placeholders[None])
+
+# Likewise the output may be a dict, tuple, or raw (deferred) Dataframe.
+result_dict = wrap_as_dict(result_frames)
+
+result_pcolls = self._apply_deferred_ops(
+{placeholders[key]._expr: pcoll
+ for key, pcoll in input_dict.items()},
+{key: df._expr
+ for key, df in result_dict.items()})
+
+# Convert the result back into a set of PCollections.
+if isinstance(result_frames, dict):
+  return result_pcolls
+elif isinstance(result_frames, tuple):
+  return tuple(*(value for _, value in sorted(result_pcolls.items(
+else:
+  return result_pcolls[None]
+
+  def _apply_deferred_ops(
+  self,
+  inputs,  # type: Dict[PlaceholderExpr, PCollection]
+  outputs,  # type: Dict[Any, Expression]
+  ):  # -> Dict[Any, PCollection]
+"""Construct a Beam graph that evaluates a set of expressions on a set of
+input PCollections.
+
+:param inputs: A mapping of placeholder expressions to PCollections.
+:param outputs: A mapping of keys to expressions defined in terms of the
+placeholders of inputs.
+
+Returns a dictionary whose keys are those of outputs, and whose values are
+PCollections corresponding to the values of outputs evaluated at the
+values of inputs.
+
+Logically, `_apply_deferred_ops({x: a, y: b}, {f: F(x, y), g: G(x, y)})`
+returns `{f: F(a, b), g: G(a, b)}`.
+"""
+class ComputeStage(beam.PTransform):
+  """A helper transform that computes a single stage of operations.
+  """
+  def __init__(self, stage):
+self.stage = stage
+
+  def default_label(self):
+return '%s:%s' % (self.stage.ops, id(self))
+
+  def expand(self, pcolls):
+if self.stage.is_grouping:
+  # Arrange such that partitioned_pcoll is properly partitioned.
+  input_pcolls = {
+  k: pcoll | 'Flat%s' % k >> 

[GitHub] [beam] robertwb commented on a change in pull request #10760: [BEAM-9545] Dataframe transforms

2020-03-19 Thread GitBox
robertwb commented on a change in pull request #10760: [BEAM-9545] Dataframe 
transforms
URL: https://github.com/apache/beam/pull/10760#discussion_r395226313
 
 

 ##
 File path: sdks/python/apache_beam/dataframe/transforms.py
 ##
 @@ -0,0 +1,244 @@
+#
+# 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 pandas as pd
+
+import apache_beam as beam
+from apache_beam import transforms
+from apache_beam.dataframe import expressions
+from apache_beam.dataframe import frame_base
+
+
+class DataframeTransform(transforms.PTransform):
+  """A PTransform for applying function that takes and returns dataframes
+  to one or more PCollections.
+
+  For example, if pcoll is a PCollection of dataframes, one could write::
+
+  pcoll | DataframeTransform(lambda df: df.group_by('key').sum())
+
+  To pass multiple PCollections, pass a tuple of PCollections wich will be
+  passed to the callable as positional arguments, or a dictionary of
+  PCollections, in which case they will be passed as keyword arguments.
+  """
+  def __init__(self, func, proxy):
+self._func = func
+self._proxy = proxy
+
+  def expand(self, input_pcolls):
+def wrap_as_dict(values):
+  if isinstance(values, dict):
+return dict
+  elif isinstance(values, tuple):
+return dict(enumerate(values))
+  else:
+return {None: values}
+
+# TODO: Infer the proxy from the input schema.
+def proxy(key):
+  if key is None:
+return self._proxy
+  else:
+return self._proxy[key]
+
+# The input can be a dictionary, tuple, or plain PCollection.
+# Wrap as a dict for homogeneity.
+# TODO: Possibly inject batching here.
+input_dict = wrap_as_dict(input_pcolls)
+placeholders = {
+key: frame_base.DeferredFrame.wrap(
+expressions.PlaceholderExpression(proxy(key)))
+for key in input_dict.keys()
+}
+
+# The calling convention of the user-supplied func varies according to the
+# type of the input.
+if isinstance(input_pcolls, dict):
+  result_frames = self._func(**placeholders)
+elif isinstance(input_pcolls, tuple):
+  self._func(*(value for _, value in sorted(placeholders.items(
 
 Review comment:
   Yeah. And now it's tested.


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


With regards,
Apache Git Services


[GitHub] [beam] robertwb commented on a change in pull request #10760: [BEAM-9545] Dataframe transforms

2020-03-19 Thread GitBox
robertwb commented on a change in pull request #10760: [BEAM-9545] Dataframe 
transforms
URL: https://github.com/apache/beam/pull/10760#discussion_r395255358
 
 

 ##
 File path: sdks/python/apache_beam/dataframe/transforms_test.py
 ##
 @@ -0,0 +1,87 @@
+#
+# 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 unittest
+
+import pandas as pd
+
+import apache_beam as beam
+from apache_beam.dataframe import expressions
+from apache_beam.dataframe import frame_base
+from apache_beam.dataframe import transforms
+from apache_beam.testing.util import assert_that
+
+
+class TransformTest(unittest.TestCase):
+  def run_test(self, input, func):
+expected = func(input)
+
+empty = input[0:0]
+input_placeholder = expressions.PlaceholderExpression(empty)
+input_deferred = frame_base.DeferredFrame.wrap(input_placeholder)
+actual_deferred = func(input_deferred)._expr.evaluate_at(
+expressions.Session({input_placeholder: input}))
+
+def check_correct(actual):
+  if actual is None:
+raise AssertionError('Empty frame but expected: \n\n%s' % (expected))
+  sorted_actual = actual.sort_index()
+  sorted_expected = expected.sort_index()
+  if not sorted_actual.equals(sorted_expected):
+raise AssertionError(
+'Dataframes not equal: \n\n%s\n\n%s' %
+(sorted_actual, sorted_expected))
+
+check_correct(actual_deferred)
+
+with beam.Pipeline() as p:
+  input_pcoll = p | beam.Create([input[::2], input[1::2]])
+  output_pcoll = input_pcoll | transforms.DataframeTransform(
+  func, proxy=empty)
+  assert_that(
+  output_pcoll,
+  lambda actual: check_correct(pd.concat(actual) if actual else None))
+
+  def test_identity(self):
+df = pd.DataFrame({
+'Animal': ['Falcon', 'Falcon', 'Parrot', 'Parrot'],
+'Speed': [380., 370., 24., 26.]
+})
+self.run_test(df, lambda x: x)
+
+  def test_sum_mean(self):
+df = pd.DataFrame({
+'Animal': ['Falcon', 'Falcon', 'Parrot', 'Parrot'],
+'Speed': [380., 370., 24., 26.]
+})
+self.run_test(df, lambda df: df.groupby('Animal').sum())
+self.run_test(df, lambda df: df.groupby('Animal').mean())
+
+  def test_filter(self):
+df = pd.DataFrame({
+'Animal': ['Aardvark', 'Ant', 'Elephant', 'Zebra'],
+'Speed': [5, 2, 35, 40]
+})
+self.run_test(df, lambda df: df.filter(items=['Animal']))
+self.run_test(df, lambda df: df.filter(regex='A.*'))
+self.run_test(
+df, lambda df: df.set_index('Animal').filter(regex='F.*', 
axis='index'))
+
+
+if __name__ == '__main__':
+  unittest.main()
 
 Review comment:
   I actually have some code that does just that: it books into the doctest 
framework to run vanilla pandas tests through these functions as beam 
pipelines. I've been meaning to get it into a PR, but didn't want to overwhelm 
you. 


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


With regards,
Apache Git Services


[GitHub] [beam] robertwb commented on a change in pull request #10760: [BEAM-9545] Dataframe transforms

2020-03-19 Thread GitBox
robertwb commented on a change in pull request #10760: [BEAM-9545] Dataframe 
transforms
URL: https://github.com/apache/beam/pull/10760#discussion_r395225830
 
 

 ##
 File path: sdks/python/apache_beam/dataframe/transforms.py
 ##
 @@ -0,0 +1,244 @@
+#
+# 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 pandas as pd
+
+import apache_beam as beam
+from apache_beam import transforms
+from apache_beam.dataframe import expressions
+from apache_beam.dataframe import frame_base
+
+
+class DataframeTransform(transforms.PTransform):
+  """A PTransform for applying function that takes and returns dataframes
+  to one or more PCollections.
+
+  For example, if pcoll is a PCollection of dataframes, one could write::
+
+  pcoll | DataframeTransform(lambda df: df.group_by('key').sum())
 
 Review comment:
   This is aspirational, hoping we can infer the proxy from the pcollection 
schema. Want to take that up :)?
   
   Updated the doc for now. 


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


With regards,
Apache Git Services