You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2022/08/29 23:14:10 UTC

[GitHub] [beam] TheNeuralBit commented on a diff in pull request #22575: Add BatchConverter implementations for pandas types, use Batched DoFns in DataFrame convert utilities

TheNeuralBit commented on code in PR #22575:
URL: https://github.com/apache/beam/pull/22575#discussion_r957857602


##########
sdks/python/apache_beam/typehints/pandas_type_compatibility_test.py:
##########
@@ -0,0 +1,191 @@
+#
+# 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.
+#
+
+"""Unit tests for pandas batched type converters."""
+
+import unittest
+from typing import Optional
+
+import numpy as np
+import pandas as pd
+from parameterized import parameterized
+from parameterized import parameterized_class
+
+from apache_beam.typehints import row_type
+from apache_beam.typehints import typehints
+from apache_beam.typehints.batch import BatchConverter
+
+
+@parameterized_class([
+    {
+        'batch_typehint': pd.DataFrame,
+        'element_typehint': row_type.RowTypeConstraint.from_fields([
+            ('foo', int),
+            ('bar', float),
+            ('baz', str),
+        ]),
+        'match_index': False,
+        'batch': pd.DataFrame({
+            'foo': pd.Series(range(100), dtype='int64'),
+            'bar': pd.Series([i / 100 for i in range(100)], dtype='float64'),
+            'baz': pd.Series([str(i) for i in range(100)],
+                             dtype=pd.StringDtype()),
+        }),
+    },
+    {
+        'batch_typehint': pd.DataFrame,
+        'element_typehint': row_type.RowTypeConstraint.from_fields(
+            [
+                ('an_index', int),
+                ('foo', int),
+                ('bar', float),
+                ('baz', str),
+            ],
+            field_options={'an_index': [('beam:dataframe:index', None)]},
+        ),
+        'match_index': True,
+        'batch': pd.DataFrame({
+            'foo': pd.Series(range(100), dtype='int64'),
+            'bar': pd.Series([i / 100 for i in range(100)], dtype='float64'),
+            'baz': pd.Series([str(i) for i in range(100)],
+                             dtype=pd.StringDtype()),
+        }).set_index(pd.Int64Index(range(123, 223), name='an_index')),
+    },
+    {
+        'batch_typehint': pd.DataFrame,
+        'element_typehint': row_type.RowTypeConstraint.from_fields(
+            [
+                ('an_index', int),
+                ('another_index', int),
+                ('foo', int),
+                ('bar', float),
+                ('baz', str),
+            ],
+            field_options={
+                'an_index': [('beam:dataframe:index', None)],
+                'another_index': [('beam:dataframe:index', None)],
+            }),
+        'match_index': True,
+        'batch': pd.DataFrame({
+            'foo': pd.Series(range(100), dtype='int64'),
+            'bar': pd.Series([i / 100 for i in range(100)], dtype='float64'),
+            'baz': pd.Series([str(i) for i in range(100)],
+                             dtype=pd.StringDtype()),
+        }).set_index([
+            pd.Int64Index(range(123, 223), name='an_index'),
+            pd.Int64Index(range(475, 575), name='another_index'),
+        ]),
+    },
+    {
+        'batch_typehint': pd.Series,
+        'element_typehint': int,
+        'match_index': False,
+        'batch': pd.Series(range(500)),
+    },
+    {
+        'batch_typehint': pd.Series,
+        'element_typehint': str,
+        'batch': pd.Series(['foo', 'bar', 'baz', 'def', 'ghi', 'abc'] * 10,
+                           dtype=pd.StringDtype()),
+    },
+    {
+        'batch_typehint': pd.Series,
+        'element_typehint': Optional[np.int64],
+        'batch': pd.Series((i if i % 11 else None for i in range(500)),
+                           dtype=pd.Int64Dtype()),
+    },
+    {
+        'batch_typehint': pd.Series,
+        'element_typehint': Optional[str],
+        'batch': pd.Series(['foo', None, 'bar', 'baz', None, 'def', 'ghi'] * 10,
+                           dtype=pd.StringDtype()),
+    },
+])
+class DataFrameBatchConverterTest(unittest.TestCase):
+  def create_batch_converter(self):
+    return BatchConverter.from_typehints(
+        element_type=self.element_typehint, batch_type=self.batch_typehint)
+
+  def setUp(self):
+    self.converter = self.create_batch_converter()
+    self.normalized_batch_typehint = typehints.normalize(self.batch_typehint)
+    self.normalized_element_typehint = typehints.normalize(
+        self.element_typehint)
+
+  def equality_check(self, left, right):
+    if isinstance(left, pd.DataFrame):
+      if self.match_index:
+        pd.testing.assert_frame_equal(left.sort_index(), right.sort_index())
+      else:
+        pd.testing.assert_frame_equal(
+            left.sort_values(by=list(left.columns)).reset_index(drop=True),
+            right.sort_values(by=list(right.columns)).reset_index(drop=True))
+    elif isinstance(left, pd.Series):
+      pd.testing.assert_series_equal(
+          left.sort_values().reset_index(drop=True),
+          right.sort_values().reset_index(drop=True))
+    else:
+      raise TypeError(f"Encountered unexpected type, left is a {type(left)!r}")
+
+  def test_typehint_validates(self):
+    typehints.validate_composite_type_param(self.batch_typehint, '')
+    typehints.validate_composite_type_param(self.element_typehint, '')
+
+  def test_type_check(self):

Review Comment:
   Done!



##########
sdks/python/apache_beam/typehints/pandas_type_compatibility.py:
##########
@@ -0,0 +1,299 @@
+#
+# 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.
+#
+
+r"""Utilities for converting between Beam schemas and pandas DataFrames.
+
+Imposes a mapping between native Python typings (specifically those compatible
+with :mod:`apache_beam.typehints.schemas`), and common pandas dtypes::
+
+  pandas dtype                    Python typing
+  np.int{8,16,32,64}      <-----> np.int{8,16,32,64}*
+  pd.Int{8,16,32,64}Dtype <-----> Optional[np.int{8,16,32,64}]*
+  np.float{32,64}         <-----> Optional[np.float{32,64}]
+                             \--- np.float{32,64}
+  Not supported           <------ Optional[bytes]
+  np.bool                 <-----> np.bool
+  np.dtype('S')           <-----> bytes
+  pd.BooleanDType()       <-----> Optional[bool]
+  pd.StringDType()        <-----> Optional[str]
+                             \--- str
+  np.object               <-----> Any
+
+  * int, float, bool are treated the same as np.int64, np.float64, np.bool
+
+Note that when converting to pandas dtypes, any types not specified here are
+shunted to ``np.object``.
+
+Similarly when converting from pandas to Python types, types that aren't
+otherwise specified here are shunted to ``Any``. Notably, this includes
+``np.datetime64``.
+
+Pandas does not support hierarchical data natively. Currently, all structured
+types (``Sequence``, ``Mapping``, nested ``NamedTuple`` types), are
+shunted to ``np.object`` like all other unknown types. In the future these
+types may be given special consideration.
+
+Note utilities in this package are for internal use only, we make no backward
+compatibility guarantees, except for the type mapping itself.
+"""
+
+from typing import Any
+from typing import List
+from typing import Optional
+
+import numpy as np
+import pandas as pd
+
+from apache_beam.typehints.batch import BatchConverter
+from apache_beam.typehints.row_type import RowTypeConstraint
+from apache_beam.typehints.typehints import is_nullable
+from apache_beam.typehints.typehints import normalize
+
+# No public API currently, this just exists to register BatchConverter
+# implementations.
+__all__ = []
+
+# Name for a valueless field-level option which, when present, indicates that
+# a field should map to an index in the Beam DataFrame API.
+INDEX_OPTION_NAME = 'beam:dataframe:index'
+
+# Generate type map (presented visually in the docstring)
+_BIDIRECTIONAL = [
+    (bool, bool),
+    (np.int8, np.int8),
+    (np.int16, np.int16),
+    (np.int32, np.int32),
+    (np.int64, np.int64),
+    (pd.Int8Dtype(), Optional[np.int8]),
+    (pd.Int16Dtype(), Optional[np.int16]),
+    (pd.Int32Dtype(), Optional[np.int32]),
+    (pd.Int64Dtype(), Optional[np.int64]),
+    (np.float32, Optional[np.float32]),
+    (np.float64, Optional[np.float64]),
+    (object, Any),
+    (pd.StringDtype(), Optional[str]),
+    (pd.BooleanDtype(), Optional[bool]),
+]
+
+PANDAS_TO_BEAM = {
+    pd.Series([], dtype=dtype).dtype: fieldtype
+    for dtype,
+    fieldtype in _BIDIRECTIONAL
+}
+BEAM_TO_PANDAS = {fieldtype: dtype for dtype, fieldtype in _BIDIRECTIONAL}
+
+# Shunt non-nullable Beam types to the same pandas types as their non-nullable
+# equivalents for FLOATs, DOUBLEs, and STRINGs. pandas has no non-nullable dtype
+# for these.
+OPTIONAL_SHUNTS = [np.float32, np.float64, str]
+
+for typehint in OPTIONAL_SHUNTS:
+  BEAM_TO_PANDAS[typehint] = BEAM_TO_PANDAS[Optional[typehint]]
+
+# int, float -> int64, np.float64
+BEAM_TO_PANDAS[int] = BEAM_TO_PANDAS[np.int64]
+BEAM_TO_PANDAS[Optional[int]] = BEAM_TO_PANDAS[Optional[np.int64]]
+BEAM_TO_PANDAS[float] = BEAM_TO_PANDAS[np.float64]
+BEAM_TO_PANDAS[Optional[float]] = BEAM_TO_PANDAS[Optional[np.float64]]
+
+BEAM_TO_PANDAS[bytes] = 'bytes'
+
+# Add shunts for normalized (Beam) typehints as well
+BEAM_TO_PANDAS.update({
+    normalize(typehint): pandas_dtype
+    for (typehint, pandas_dtype) in BEAM_TO_PANDAS.items()
+})
+
+
+def dtype_from_typehint(typehint):
+  # Default to np.object. This is lossy, we won't be able to recover
+  # the type at the output.
+  return BEAM_TO_PANDAS.get(typehint, object)
+
+
+def dtype_to_fieldtype(dtype):
+  fieldtype = PANDAS_TO_BEAM.get(dtype)
+
+  if fieldtype is not None:
+    return fieldtype
+  elif dtype.kind == 'S':
+    return bytes
+  else:
+    return Any
+
+
+@BatchConverter.register
+def create_pandas_batch_converter(
+    element_type: type, batch_type: type) -> BatchConverter:
+  if batch_type == pd.DataFrame:
+    return DataFrameBatchConverter.from_typehints(
+        element_type=element_type, batch_type=batch_type)
+  elif batch_type == pd.Series:
+    return SeriesBatchConverter.from_typehints(
+        element_type=element_type, batch_type=batch_type)
+
+  return None
+
+
+class DataFrameBatchConverter(BatchConverter):
+  def __init__(
+      self,
+      element_type: RowTypeConstraint,
+  ):
+    super().__init__(pd.DataFrame, element_type)
+    self._columns = [name for name, _ in element_type._fields]
+
+  @staticmethod
+  def from_typehints(element_type,
+                     batch_type) -> Optional['DataFrameBatchConverter']:
+    if not batch_type == pd.DataFrame:
+      return None
+
+    if not isinstance(element_type, RowTypeConstraint):
+      element_type = RowTypeConstraint.from_user_type(element_type)
+      if element_type is None:
+        return None
+
+    index_columns = [
+        field_name
+        for (field_name, field_options) in element_type._field_options.items()
+        if any(key == INDEX_OPTION_NAME for key, value in field_options)
+    ]
+
+    if index_columns:
+      return DataFrameBatchConverterKeepIndex(element_type, index_columns)
+    else:
+      return DataFrameBatchConverterDropIndex(element_type)
+
+  def _get_series(self, batch: pd.DataFrame):
+    raise NotImplementedError
+
+  def explode_batch(self, batch: pd.DataFrame):
+    # TODO: Only do null checks for nullable types
+    def make_null_checking_generator(series):
+      nulls = pd.isnull(series)
+      return (None if isnull else value for isnull, value in zip(nulls, series))
+
+    all_series = self._get_series(batch)
+    iterators = [make_null_checking_generator(series) for series in all_series]
+
+    for values in zip(*iterators):
+      yield self._element_type.user_type(
+          **{column: value
+             for column, value in zip(self._columns, values)})
+
+  def combine_batches(self, batches: List[pd.DataFrame]):
+    return pd.concat(batches)
+
+  def estimate_byte_size(self, batch: pd.DataFrame):
+    return batch.memory_usage().sum()
+
+  def get_length(self, batch: pd.DataFrame):
+    return len(batch)

Review Comment:
   Done, thank you! I also filed #22950 - we should have a standard test suite to test all the `BatchConverter` implementations.



##########
sdks/python/apache_beam/typehints/pandas_type_compatibility.py:
##########
@@ -0,0 +1,299 @@
+#
+# 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.
+#
+
+r"""Utilities for converting between Beam schemas and pandas DataFrames.
+
+Imposes a mapping between native Python typings (specifically those compatible
+with :mod:`apache_beam.typehints.schemas`), and common pandas dtypes::
+
+  pandas dtype                    Python typing
+  np.int{8,16,32,64}      <-----> np.int{8,16,32,64}*
+  pd.Int{8,16,32,64}Dtype <-----> Optional[np.int{8,16,32,64}]*
+  np.float{32,64}         <-----> Optional[np.float{32,64}]
+                             \--- np.float{32,64}
+  Not supported           <------ Optional[bytes]
+  np.bool                 <-----> np.bool
+  np.dtype('S')           <-----> bytes
+  pd.BooleanDType()       <-----> Optional[bool]
+  pd.StringDType()        <-----> Optional[str]
+                             \--- str
+  np.object               <-----> Any
+
+  * int, float, bool are treated the same as np.int64, np.float64, np.bool
+
+Note that when converting to pandas dtypes, any types not specified here are
+shunted to ``np.object``.
+
+Similarly when converting from pandas to Python types, types that aren't
+otherwise specified here are shunted to ``Any``. Notably, this includes
+``np.datetime64``.
+
+Pandas does not support hierarchical data natively. Currently, all structured
+types (``Sequence``, ``Mapping``, nested ``NamedTuple`` types), are
+shunted to ``np.object`` like all other unknown types. In the future these
+types may be given special consideration.
+
+Note utilities in this package are for internal use only, we make no backward
+compatibility guarantees, except for the type mapping itself.
+"""
+
+from typing import Any
+from typing import List
+from typing import Optional
+
+import numpy as np
+import pandas as pd
+
+from apache_beam.typehints.batch import BatchConverter
+from apache_beam.typehints.row_type import RowTypeConstraint
+from apache_beam.typehints.typehints import is_nullable
+from apache_beam.typehints.typehints import normalize
+
+# No public API currently, this just exists to register BatchConverter
+# implementations.
+__all__ = []
+
+# Name for a valueless field-level option which, when present, indicates that
+# a field should map to an index in the Beam DataFrame API.
+INDEX_OPTION_NAME = 'beam:dataframe:index'
+
+# Generate type map (presented visually in the docstring)
+_BIDIRECTIONAL = [
+    (bool, bool),
+    (np.int8, np.int8),
+    (np.int16, np.int16),
+    (np.int32, np.int32),
+    (np.int64, np.int64),
+    (pd.Int8Dtype(), Optional[np.int8]),
+    (pd.Int16Dtype(), Optional[np.int16]),
+    (pd.Int32Dtype(), Optional[np.int32]),
+    (pd.Int64Dtype(), Optional[np.int64]),
+    (np.float32, Optional[np.float32]),
+    (np.float64, Optional[np.float64]),
+    (object, Any),
+    (pd.StringDtype(), Optional[str]),
+    (pd.BooleanDtype(), Optional[bool]),
+]
+
+PANDAS_TO_BEAM = {
+    pd.Series([], dtype=dtype).dtype: fieldtype
+    for dtype,
+    fieldtype in _BIDIRECTIONAL
+}
+BEAM_TO_PANDAS = {fieldtype: dtype for dtype, fieldtype in _BIDIRECTIONAL}
+
+# Shunt non-nullable Beam types to the same pandas types as their non-nullable
+# equivalents for FLOATs, DOUBLEs, and STRINGs. pandas has no non-nullable dtype
+# for these.
+OPTIONAL_SHUNTS = [np.float32, np.float64, str]
+
+for typehint in OPTIONAL_SHUNTS:
+  BEAM_TO_PANDAS[typehint] = BEAM_TO_PANDAS[Optional[typehint]]
+
+# int, float -> int64, np.float64
+BEAM_TO_PANDAS[int] = BEAM_TO_PANDAS[np.int64]
+BEAM_TO_PANDAS[Optional[int]] = BEAM_TO_PANDAS[Optional[np.int64]]
+BEAM_TO_PANDAS[float] = BEAM_TO_PANDAS[np.float64]
+BEAM_TO_PANDAS[Optional[float]] = BEAM_TO_PANDAS[Optional[np.float64]]
+
+BEAM_TO_PANDAS[bytes] = 'bytes'
+
+# Add shunts for normalized (Beam) typehints as well
+BEAM_TO_PANDAS.update({
+    normalize(typehint): pandas_dtype
+    for (typehint, pandas_dtype) in BEAM_TO_PANDAS.items()
+})
+
+
+def dtype_from_typehint(typehint):
+  # Default to np.object. This is lossy, we won't be able to recover
+  # the type at the output.
+  return BEAM_TO_PANDAS.get(typehint, object)
+
+
+def dtype_to_fieldtype(dtype):
+  fieldtype = PANDAS_TO_BEAM.get(dtype)
+
+  if fieldtype is not None:
+    return fieldtype
+  elif dtype.kind == 'S':
+    return bytes
+  else:
+    return Any
+
+
+@BatchConverter.register
+def create_pandas_batch_converter(
+    element_type: type, batch_type: type) -> BatchConverter:
+  if batch_type == pd.DataFrame:
+    return DataFrameBatchConverter.from_typehints(
+        element_type=element_type, batch_type=batch_type)
+  elif batch_type == pd.Series:
+    return SeriesBatchConverter.from_typehints(
+        element_type=element_type, batch_type=batch_type)
+
+  return None
+
+
+class DataFrameBatchConverter(BatchConverter):
+  def __init__(
+      self,
+      element_type: RowTypeConstraint,
+  ):
+    super().__init__(pd.DataFrame, element_type)
+    self._columns = [name for name, _ in element_type._fields]
+
+  @staticmethod
+  def from_typehints(element_type,
+                     batch_type) -> Optional['DataFrameBatchConverter']:
+    if not batch_type == pd.DataFrame:
+      return None
+
+    if not isinstance(element_type, RowTypeConstraint):
+      element_type = RowTypeConstraint.from_user_type(element_type)
+      if element_type is None:
+        return None
+
+    index_columns = [
+        field_name
+        for (field_name, field_options) in element_type._field_options.items()
+        if any(key == INDEX_OPTION_NAME for key, value in field_options)
+    ]
+
+    if index_columns:
+      return DataFrameBatchConverterKeepIndex(element_type, index_columns)
+    else:
+      return DataFrameBatchConverterDropIndex(element_type)
+
+  def _get_series(self, batch: pd.DataFrame):
+    raise NotImplementedError
+
+  def explode_batch(self, batch: pd.DataFrame):
+    # TODO: Only do null checks for nullable types
+    def make_null_checking_generator(series):
+      nulls = pd.isnull(series)
+      return (None if isnull else value for isnull, value in zip(nulls, series))
+
+    all_series = self._get_series(batch)
+    iterators = [make_null_checking_generator(series) for series in all_series]
+
+    for values in zip(*iterators):

Review Comment:
   Im not quite sure what this would look like, could you clarify?
   
   Related: It would be good to add a microbenchmark for produce_batch and explode_batch so we can easily evaluate alternative implementations. But I'd prefer to leave that as future work. For now this just preserves the implementation from `apache_beam.dataframe.schemas`.



##########
sdks/python/apache_beam/typehints/pandas_type_compatibility.py:
##########
@@ -0,0 +1,299 @@
+#
+# 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.
+#
+
+r"""Utilities for converting between Beam schemas and pandas DataFrames.
+
+Imposes a mapping between native Python typings (specifically those compatible
+with :mod:`apache_beam.typehints.schemas`), and common pandas dtypes::
+
+  pandas dtype                    Python typing
+  np.int{8,16,32,64}      <-----> np.int{8,16,32,64}*
+  pd.Int{8,16,32,64}Dtype <-----> Optional[np.int{8,16,32,64}]*
+  np.float{32,64}         <-----> Optional[np.float{32,64}]
+                             \--- np.float{32,64}
+  Not supported           <------ Optional[bytes]
+  np.bool                 <-----> np.bool
+  np.dtype('S')           <-----> bytes
+  pd.BooleanDType()       <-----> Optional[bool]
+  pd.StringDType()        <-----> Optional[str]
+                             \--- str
+  np.object               <-----> Any
+
+  * int, float, bool are treated the same as np.int64, np.float64, np.bool
+
+Note that when converting to pandas dtypes, any types not specified here are
+shunted to ``np.object``.
+
+Similarly when converting from pandas to Python types, types that aren't
+otherwise specified here are shunted to ``Any``. Notably, this includes
+``np.datetime64``.
+
+Pandas does not support hierarchical data natively. Currently, all structured
+types (``Sequence``, ``Mapping``, nested ``NamedTuple`` types), are
+shunted to ``np.object`` like all other unknown types. In the future these
+types may be given special consideration.
+
+Note utilities in this package are for internal use only, we make no backward
+compatibility guarantees, except for the type mapping itself.
+"""
+
+from typing import Any
+from typing import List
+from typing import Optional
+
+import numpy as np
+import pandas as pd
+
+from apache_beam.typehints.batch import BatchConverter
+from apache_beam.typehints.row_type import RowTypeConstraint
+from apache_beam.typehints.typehints import is_nullable
+from apache_beam.typehints.typehints import normalize
+
+# No public API currently, this just exists to register BatchConverter
+# implementations.
+__all__ = []
+
+# Name for a valueless field-level option which, when present, indicates that
+# a field should map to an index in the Beam DataFrame API.
+INDEX_OPTION_NAME = 'beam:dataframe:index'
+
+# Generate type map (presented visually in the docstring)
+_BIDIRECTIONAL = [
+    (bool, bool),
+    (np.int8, np.int8),
+    (np.int16, np.int16),
+    (np.int32, np.int32),
+    (np.int64, np.int64),
+    (pd.Int8Dtype(), Optional[np.int8]),
+    (pd.Int16Dtype(), Optional[np.int16]),
+    (pd.Int32Dtype(), Optional[np.int32]),
+    (pd.Int64Dtype(), Optional[np.int64]),
+    (np.float32, Optional[np.float32]),
+    (np.float64, Optional[np.float64]),
+    (object, Any),
+    (pd.StringDtype(), Optional[str]),
+    (pd.BooleanDtype(), Optional[bool]),
+]
+
+PANDAS_TO_BEAM = {
+    pd.Series([], dtype=dtype).dtype: fieldtype
+    for dtype,
+    fieldtype in _BIDIRECTIONAL
+}
+BEAM_TO_PANDAS = {fieldtype: dtype for dtype, fieldtype in _BIDIRECTIONAL}
+
+# Shunt non-nullable Beam types to the same pandas types as their non-nullable
+# equivalents for FLOATs, DOUBLEs, and STRINGs. pandas has no non-nullable dtype
+# for these.
+OPTIONAL_SHUNTS = [np.float32, np.float64, str]
+
+for typehint in OPTIONAL_SHUNTS:
+  BEAM_TO_PANDAS[typehint] = BEAM_TO_PANDAS[Optional[typehint]]
+
+# int, float -> int64, np.float64
+BEAM_TO_PANDAS[int] = BEAM_TO_PANDAS[np.int64]
+BEAM_TO_PANDAS[Optional[int]] = BEAM_TO_PANDAS[Optional[np.int64]]
+BEAM_TO_PANDAS[float] = BEAM_TO_PANDAS[np.float64]
+BEAM_TO_PANDAS[Optional[float]] = BEAM_TO_PANDAS[Optional[np.float64]]
+
+BEAM_TO_PANDAS[bytes] = 'bytes'
+
+# Add shunts for normalized (Beam) typehints as well
+BEAM_TO_PANDAS.update({
+    normalize(typehint): pandas_dtype
+    for (typehint, pandas_dtype) in BEAM_TO_PANDAS.items()
+})
+
+
+def dtype_from_typehint(typehint):
+  # Default to np.object. This is lossy, we won't be able to recover
+  # the type at the output.
+  return BEAM_TO_PANDAS.get(typehint, object)
+
+
+def dtype_to_fieldtype(dtype):
+  fieldtype = PANDAS_TO_BEAM.get(dtype)
+
+  if fieldtype is not None:
+    return fieldtype
+  elif dtype.kind == 'S':
+    return bytes
+  else:
+    return Any
+
+
+@BatchConverter.register
+def create_pandas_batch_converter(
+    element_type: type, batch_type: type) -> BatchConverter:
+  if batch_type == pd.DataFrame:
+    return DataFrameBatchConverter.from_typehints(
+        element_type=element_type, batch_type=batch_type)
+  elif batch_type == pd.Series:
+    return SeriesBatchConverter.from_typehints(
+        element_type=element_type, batch_type=batch_type)
+
+  return None
+
+
+class DataFrameBatchConverter(BatchConverter):
+  def __init__(
+      self,
+      element_type: RowTypeConstraint,
+  ):
+    super().__init__(pd.DataFrame, element_type)
+    self._columns = [name for name, _ in element_type._fields]
+
+  @staticmethod
+  def from_typehints(element_type,
+                     batch_type) -> Optional['DataFrameBatchConverter']:
+    if not batch_type == pd.DataFrame:
+      return None
+
+    if not isinstance(element_type, RowTypeConstraint):
+      element_type = RowTypeConstraint.from_user_type(element_type)
+      if element_type is None:
+        return None
+
+    index_columns = [
+        field_name
+        for (field_name, field_options) in element_type._field_options.items()
+        if any(key == INDEX_OPTION_NAME for key, value in field_options)
+    ]
+
+    if index_columns:
+      return DataFrameBatchConverterKeepIndex(element_type, index_columns)
+    else:
+      return DataFrameBatchConverterDropIndex(element_type)
+
+  def _get_series(self, batch: pd.DataFrame):
+    raise NotImplementedError
+
+  def explode_batch(self, batch: pd.DataFrame):
+    # TODO: Only do null checks for nullable types
+    def make_null_checking_generator(series):
+      nulls = pd.isnull(series)
+      return (None if isnull else value for isnull, value in zip(nulls, series))
+
+    all_series = self._get_series(batch)
+    iterators = [make_null_checking_generator(series) for series in all_series]
+
+    for values in zip(*iterators):
+      yield self._element_type.user_type(
+          **{column: value
+             for column, value in zip(self._columns, values)})
+
+  def combine_batches(self, batches: List[pd.DataFrame]):
+    return pd.concat(batches)
+
+  def estimate_byte_size(self, batch: pd.DataFrame):
+    return batch.memory_usage().sum()
+
+  def get_length(self, batch: pd.DataFrame):
+    return len(batch)
+
+
+class DataFrameBatchConverterDropIndex(DataFrameBatchConverter):
+  """A DataFrameBatchConverter that assumes the DataFrame index has no meaning.
+
+  When producing a DataFrame from Rows, a meaningless index will be generated.
+  When exploding a DataFrame into Rows, the index will be dropped.
+  """
+  def _get_series(self, batch: pd.DataFrame):
+    return [batch[column] for column in batch.columns]
+
+  def produce_batch(self, elements):
+    # Note from_records has an index= parameter
+    batch = pd.DataFrame.from_records(elements, columns=self._columns)
+
+    for column, typehint in self._element_type._fields:
+      batch[column] = batch[column].astype(dtype_from_typehint(typehint))
+
+    return batch
+
+
+class DataFrameBatchConverterKeepIndex(DataFrameBatchConverter):
+  """A DataFrameBatchConverter that preserves the DataFrame index.
+
+  This is tracked via options on the Beam schema. Each field in the schema that
+  should map to the index is tagged in an option with name 'dataframe:index'.
+  """
+  def __init__(self, element_type: RowTypeConstraint, index_columns: List[Any]):
+    super().__init__(element_type)
+    self._index_columns = index_columns
+
+  def _get_series(self, batch: pd.DataFrame):
+    assert list(batch.index.names) == self._index_columns
+    return [
+        batch.index.get_level_values(i) for i in range(len(batch.index.names))
+    ] + [batch[column] for column in batch.columns]
+
+  def produce_batch(self, elements):
+    # Note from_records has an index= parameter
+    batch = pd.DataFrame.from_records(elements, columns=self._columns)

Review Comment:
   Yeah that's right, I think the above comment was just a note to self as I was iterating on this. I dropped the comment.



##########
sdks/python/apache_beam/typehints/pandas_type_compatibility.py:
##########
@@ -0,0 +1,299 @@
+#
+# 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.
+#
+
+r"""Utilities for converting between Beam schemas and pandas DataFrames.
+
+Imposes a mapping between native Python typings (specifically those compatible
+with :mod:`apache_beam.typehints.schemas`), and common pandas dtypes::
+
+  pandas dtype                    Python typing
+  np.int{8,16,32,64}      <-----> np.int{8,16,32,64}*
+  pd.Int{8,16,32,64}Dtype <-----> Optional[np.int{8,16,32,64}]*
+  np.float{32,64}         <-----> Optional[np.float{32,64}]
+                             \--- np.float{32,64}
+  Not supported           <------ Optional[bytes]
+  np.bool                 <-----> np.bool
+  np.dtype('S')           <-----> bytes
+  pd.BooleanDType()       <-----> Optional[bool]
+  pd.StringDType()        <-----> Optional[str]
+                             \--- str
+  np.object               <-----> Any
+
+  * int, float, bool are treated the same as np.int64, np.float64, np.bool
+
+Note that when converting to pandas dtypes, any types not specified here are
+shunted to ``np.object``.
+
+Similarly when converting from pandas to Python types, types that aren't
+otherwise specified here are shunted to ``Any``. Notably, this includes
+``np.datetime64``.
+
+Pandas does not support hierarchical data natively. Currently, all structured
+types (``Sequence``, ``Mapping``, nested ``NamedTuple`` types), are
+shunted to ``np.object`` like all other unknown types. In the future these
+types may be given special consideration.
+
+Note utilities in this package are for internal use only, we make no backward
+compatibility guarantees, except for the type mapping itself.
+"""
+
+from typing import Any
+from typing import List
+from typing import Optional
+
+import numpy as np
+import pandas as pd
+
+from apache_beam.typehints.batch import BatchConverter
+from apache_beam.typehints.row_type import RowTypeConstraint
+from apache_beam.typehints.typehints import is_nullable
+from apache_beam.typehints.typehints import normalize
+
+# No public API currently, this just exists to register BatchConverter
+# implementations.
+__all__ = []
+
+# Name for a valueless field-level option which, when present, indicates that
+# a field should map to an index in the Beam DataFrame API.
+INDEX_OPTION_NAME = 'beam:dataframe:index'
+
+# Generate type map (presented visually in the docstring)
+_BIDIRECTIONAL = [
+    (bool, bool),
+    (np.int8, np.int8),
+    (np.int16, np.int16),
+    (np.int32, np.int32),
+    (np.int64, np.int64),
+    (pd.Int8Dtype(), Optional[np.int8]),
+    (pd.Int16Dtype(), Optional[np.int16]),
+    (pd.Int32Dtype(), Optional[np.int32]),
+    (pd.Int64Dtype(), Optional[np.int64]),
+    (np.float32, Optional[np.float32]),
+    (np.float64, Optional[np.float64]),
+    (object, Any),
+    (pd.StringDtype(), Optional[str]),
+    (pd.BooleanDtype(), Optional[bool]),
+]
+
+PANDAS_TO_BEAM = {
+    pd.Series([], dtype=dtype).dtype: fieldtype
+    for dtype,
+    fieldtype in _BIDIRECTIONAL
+}
+BEAM_TO_PANDAS = {fieldtype: dtype for dtype, fieldtype in _BIDIRECTIONAL}
+
+# Shunt non-nullable Beam types to the same pandas types as their non-nullable
+# equivalents for FLOATs, DOUBLEs, and STRINGs. pandas has no non-nullable dtype
+# for these.
+OPTIONAL_SHUNTS = [np.float32, np.float64, str]
+
+for typehint in OPTIONAL_SHUNTS:
+  BEAM_TO_PANDAS[typehint] = BEAM_TO_PANDAS[Optional[typehint]]
+
+# int, float -> int64, np.float64
+BEAM_TO_PANDAS[int] = BEAM_TO_PANDAS[np.int64]
+BEAM_TO_PANDAS[Optional[int]] = BEAM_TO_PANDAS[Optional[np.int64]]
+BEAM_TO_PANDAS[float] = BEAM_TO_PANDAS[np.float64]
+BEAM_TO_PANDAS[Optional[float]] = BEAM_TO_PANDAS[Optional[np.float64]]
+
+BEAM_TO_PANDAS[bytes] = 'bytes'
+
+# Add shunts for normalized (Beam) typehints as well
+BEAM_TO_PANDAS.update({
+    normalize(typehint): pandas_dtype
+    for (typehint, pandas_dtype) in BEAM_TO_PANDAS.items()
+})
+
+
+def dtype_from_typehint(typehint):
+  # Default to np.object. This is lossy, we won't be able to recover
+  # the type at the output.
+  return BEAM_TO_PANDAS.get(typehint, object)
+
+
+def dtype_to_fieldtype(dtype):
+  fieldtype = PANDAS_TO_BEAM.get(dtype)
+
+  if fieldtype is not None:
+    return fieldtype
+  elif dtype.kind == 'S':
+    return bytes
+  else:
+    return Any
+
+
+@BatchConverter.register
+def create_pandas_batch_converter(
+    element_type: type, batch_type: type) -> BatchConverter:
+  if batch_type == pd.DataFrame:
+    return DataFrameBatchConverter.from_typehints(
+        element_type=element_type, batch_type=batch_type)
+  elif batch_type == pd.Series:
+    return SeriesBatchConverter.from_typehints(
+        element_type=element_type, batch_type=batch_type)
+
+  return None
+
+
+class DataFrameBatchConverter(BatchConverter):
+  def __init__(
+      self,
+      element_type: RowTypeConstraint,
+  ):
+    super().__init__(pd.DataFrame, element_type)
+    self._columns = [name for name, _ in element_type._fields]
+
+  @staticmethod
+  def from_typehints(element_type,
+                     batch_type) -> Optional['DataFrameBatchConverter']:
+    if not batch_type == pd.DataFrame:
+      return None
+
+    if not isinstance(element_type, RowTypeConstraint):
+      element_type = RowTypeConstraint.from_user_type(element_type)
+      if element_type is None:
+        return None
+
+    index_columns = [
+        field_name
+        for (field_name, field_options) in element_type._field_options.items()
+        if any(key == INDEX_OPTION_NAME for key, value in field_options)
+    ]
+
+    if index_columns:
+      return DataFrameBatchConverterKeepIndex(element_type, index_columns)
+    else:
+      return DataFrameBatchConverterDropIndex(element_type)
+
+  def _get_series(self, batch: pd.DataFrame):
+    raise NotImplementedError
+
+  def explode_batch(self, batch: pd.DataFrame):
+    # TODO: Only do null checks for nullable types

Review Comment:
   There is now :) (https://github.com/apache/beam/issues/22948)



##########
sdks/python/apache_beam/typehints/pandas_type_compatibility_test.py:
##########
@@ -0,0 +1,191 @@
+#
+# 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.
+#
+
+"""Unit tests for pandas batched type converters."""
+
+import unittest
+from typing import Optional
+
+import numpy as np
+import pandas as pd
+from parameterized import parameterized
+from parameterized import parameterized_class
+
+from apache_beam.typehints import row_type
+from apache_beam.typehints import typehints
+from apache_beam.typehints.batch import BatchConverter
+
+
+@parameterized_class([
+    {
+        'batch_typehint': pd.DataFrame,
+        'element_typehint': row_type.RowTypeConstraint.from_fields([
+            ('foo', int),
+            ('bar', float),
+            ('baz', str),
+        ]),
+        'match_index': False,
+        'batch': pd.DataFrame({
+            'foo': pd.Series(range(100), dtype='int64'),
+            'bar': pd.Series([i / 100 for i in range(100)], dtype='float64'),
+            'baz': pd.Series([str(i) for i in range(100)],
+                             dtype=pd.StringDtype()),
+        }),
+    },
+    {
+        'batch_typehint': pd.DataFrame,
+        'element_typehint': row_type.RowTypeConstraint.from_fields(
+            [
+                ('an_index', int),
+                ('foo', int),
+                ('bar', float),
+                ('baz', str),
+            ],
+            field_options={'an_index': [('beam:dataframe:index', None)]},
+        ),
+        'match_index': True,
+        'batch': pd.DataFrame({
+            'foo': pd.Series(range(100), dtype='int64'),
+            'bar': pd.Series([i / 100 for i in range(100)], dtype='float64'),
+            'baz': pd.Series([str(i) for i in range(100)],
+                             dtype=pd.StringDtype()),
+        }).set_index(pd.Int64Index(range(123, 223), name='an_index')),
+    },
+    {
+        'batch_typehint': pd.DataFrame,
+        'element_typehint': row_type.RowTypeConstraint.from_fields(
+            [
+                ('an_index', int),
+                ('another_index', int),
+                ('foo', int),
+                ('bar', float),
+                ('baz', str),
+            ],
+            field_options={
+                'an_index': [('beam:dataframe:index', None)],
+                'another_index': [('beam:dataframe:index', None)],
+            }),
+        'match_index': True,
+        'batch': pd.DataFrame({
+            'foo': pd.Series(range(100), dtype='int64'),
+            'bar': pd.Series([i / 100 for i in range(100)], dtype='float64'),
+            'baz': pd.Series([str(i) for i in range(100)],
+                             dtype=pd.StringDtype()),
+        }).set_index([
+            pd.Int64Index(range(123, 223), name='an_index'),
+            pd.Int64Index(range(475, 575), name='another_index'),
+        ]),
+    },
+    {
+        'batch_typehint': pd.Series,
+        'element_typehint': int,
+        'match_index': False,
+        'batch': pd.Series(range(500)),
+    },
+    {
+        'batch_typehint': pd.Series,
+        'element_typehint': str,
+        'batch': pd.Series(['foo', 'bar', 'baz', 'def', 'ghi', 'abc'] * 10,
+                           dtype=pd.StringDtype()),
+    },
+    {
+        'batch_typehint': pd.Series,
+        'element_typehint': Optional[np.int64],
+        'batch': pd.Series((i if i % 11 else None for i in range(500)),
+                           dtype=pd.Int64Dtype()),
+    },
+    {
+        'batch_typehint': pd.Series,
+        'element_typehint': Optional[str],
+        'batch': pd.Series(['foo', None, 'bar', 'baz', None, 'def', 'ghi'] * 10,
+                           dtype=pd.StringDtype()),
+    },
+])
+class DataFrameBatchConverterTest(unittest.TestCase):
+  def create_batch_converter(self):
+    return BatchConverter.from_typehints(
+        element_type=self.element_typehint, batch_type=self.batch_typehint)
+
+  def setUp(self):
+    self.converter = self.create_batch_converter()
+    self.normalized_batch_typehint = typehints.normalize(self.batch_typehint)
+    self.normalized_element_typehint = typehints.normalize(
+        self.element_typehint)
+
+  def equality_check(self, left, right):
+    if isinstance(left, pd.DataFrame):
+      if self.match_index:
+        pd.testing.assert_frame_equal(left.sort_index(), right.sort_index())
+      else:
+        pd.testing.assert_frame_equal(
+            left.sort_values(by=list(left.columns)).reset_index(drop=True),
+            right.sort_values(by=list(right.columns)).reset_index(drop=True))
+    elif isinstance(left, pd.Series):
+      pd.testing.assert_series_equal(
+          left.sort_values().reset_index(drop=True),
+          right.sort_values().reset_index(drop=True))
+    else:
+      raise TypeError(f"Encountered unexpected type, left is a {type(left)!r}")
+
+  def test_typehint_validates(self):
+    typehints.validate_composite_type_param(self.batch_typehint, '')
+    typehints.validate_composite_type_param(self.element_typehint, '')
+
+  def test_type_check(self):
+    typehints.check_constraint(self.normalized_batch_typehint, self.batch)
+
+  def test_type_check_element(self):
+    for element in self.converter.explode_batch(self.batch):
+      typehints.check_constraint(self.normalized_element_typehint, element)
+
+  def test_explode_rebatch(self):
+    exploded = list(self.converter.explode_batch(self.batch))
+    rebatched = self.converter.produce_batch(exploded)
+
+    typehints.check_constraint(self.normalized_batch_typehint, rebatched)
+    self.equality_check(self.batch, rebatched)
+
+  @parameterized.expand([
+      (2, ),
+      (3, ),
+      (10, ),
+  ])
+  def test_combine_batches(self, N):
+    elements = list(self.converter.explode_batch(self.batch))
+
+    # Split elements into N contiguous partitions, create a batch out of each
+    batches = [
+        self.converter.produce_batch(
+            elements[len(elements) * i // N:len(elements) * (i + 1) // N])
+        for i in range(N)
+    ]
+
+    # Combine the batches, output should be equivalent to the original batch
+    combined = self.converter.combine_batches(batches)
+
+    self.equality_check(self.batch, combined)
+
+  def test_equals(self):
+    self.assertTrue(self.converter == self.create_batch_converter())
+    self.assertTrue(self.create_batch_converter() == self.converter)

Review Comment:
   This is just being overly cautious - in theory the instances on either side could be a different type and could have a different `__eq__` implementation.



##########
sdks/python/apache_beam/typehints/pandas_type_compatibility.py:
##########
@@ -0,0 +1,299 @@
+#
+# 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.
+#
+
+r"""Utilities for converting between Beam schemas and pandas DataFrames.
+
+Imposes a mapping between native Python typings (specifically those compatible
+with :mod:`apache_beam.typehints.schemas`), and common pandas dtypes::
+
+  pandas dtype                    Python typing
+  np.int{8,16,32,64}      <-----> np.int{8,16,32,64}*
+  pd.Int{8,16,32,64}Dtype <-----> Optional[np.int{8,16,32,64}]*
+  np.float{32,64}         <-----> Optional[np.float{32,64}]
+                             \--- np.float{32,64}
+  Not supported           <------ Optional[bytes]
+  np.bool                 <-----> np.bool
+  np.dtype('S')           <-----> bytes
+  pd.BooleanDType()       <-----> Optional[bool]
+  pd.StringDType()        <-----> Optional[str]
+                             \--- str
+  np.object               <-----> Any
+
+  * int, float, bool are treated the same as np.int64, np.float64, np.bool
+
+Note that when converting to pandas dtypes, any types not specified here are
+shunted to ``np.object``.
+
+Similarly when converting from pandas to Python types, types that aren't
+otherwise specified here are shunted to ``Any``. Notably, this includes
+``np.datetime64``.
+
+Pandas does not support hierarchical data natively. Currently, all structured
+types (``Sequence``, ``Mapping``, nested ``NamedTuple`` types), are
+shunted to ``np.object`` like all other unknown types. In the future these
+types may be given special consideration.
+
+Note utilities in this package are for internal use only, we make no backward
+compatibility guarantees, except for the type mapping itself.
+"""
+
+from typing import Any
+from typing import List
+from typing import Optional
+
+import numpy as np
+import pandas as pd
+
+from apache_beam.typehints.batch import BatchConverter
+from apache_beam.typehints.row_type import RowTypeConstraint
+from apache_beam.typehints.typehints import is_nullable
+from apache_beam.typehints.typehints import normalize
+
+# No public API currently, this just exists to register BatchConverter
+# implementations.
+__all__ = []
+
+# Name for a valueless field-level option which, when present, indicates that
+# a field should map to an index in the Beam DataFrame API.
+INDEX_OPTION_NAME = 'beam:dataframe:index'
+
+# Generate type map (presented visually in the docstring)
+_BIDIRECTIONAL = [
+    (bool, bool),
+    (np.int8, np.int8),
+    (np.int16, np.int16),
+    (np.int32, np.int32),
+    (np.int64, np.int64),
+    (pd.Int8Dtype(), Optional[np.int8]),
+    (pd.Int16Dtype(), Optional[np.int16]),
+    (pd.Int32Dtype(), Optional[np.int32]),
+    (pd.Int64Dtype(), Optional[np.int64]),
+    (np.float32, Optional[np.float32]),
+    (np.float64, Optional[np.float64]),
+    (object, Any),
+    (pd.StringDtype(), Optional[str]),
+    (pd.BooleanDtype(), Optional[bool]),
+]
+
+PANDAS_TO_BEAM = {
+    pd.Series([], dtype=dtype).dtype: fieldtype
+    for dtype,
+    fieldtype in _BIDIRECTIONAL
+}
+BEAM_TO_PANDAS = {fieldtype: dtype for dtype, fieldtype in _BIDIRECTIONAL}
+
+# Shunt non-nullable Beam types to the same pandas types as their non-nullable
+# equivalents for FLOATs, DOUBLEs, and STRINGs. pandas has no non-nullable dtype
+# for these.
+OPTIONAL_SHUNTS = [np.float32, np.float64, str]
+
+for typehint in OPTIONAL_SHUNTS:
+  BEAM_TO_PANDAS[typehint] = BEAM_TO_PANDAS[Optional[typehint]]
+
+# int, float -> int64, np.float64
+BEAM_TO_PANDAS[int] = BEAM_TO_PANDAS[np.int64]
+BEAM_TO_PANDAS[Optional[int]] = BEAM_TO_PANDAS[Optional[np.int64]]
+BEAM_TO_PANDAS[float] = BEAM_TO_PANDAS[np.float64]
+BEAM_TO_PANDAS[Optional[float]] = BEAM_TO_PANDAS[Optional[np.float64]]
+
+BEAM_TO_PANDAS[bytes] = 'bytes'
+
+# Add shunts for normalized (Beam) typehints as well
+BEAM_TO_PANDAS.update({
+    normalize(typehint): pandas_dtype
+    for (typehint, pandas_dtype) in BEAM_TO_PANDAS.items()
+})
+
+
+def dtype_from_typehint(typehint):
+  # Default to np.object. This is lossy, we won't be able to recover
+  # the type at the output.
+  return BEAM_TO_PANDAS.get(typehint, object)
+
+
+def dtype_to_fieldtype(dtype):
+  fieldtype = PANDAS_TO_BEAM.get(dtype)
+
+  if fieldtype is not None:
+    return fieldtype
+  elif dtype.kind == 'S':
+    return bytes
+  else:
+    return Any
+
+
+@BatchConverter.register
+def create_pandas_batch_converter(
+    element_type: type, batch_type: type) -> BatchConverter:
+  if batch_type == pd.DataFrame:
+    return DataFrameBatchConverter.from_typehints(
+        element_type=element_type, batch_type=batch_type)
+  elif batch_type == pd.Series:
+    return SeriesBatchConverter.from_typehints(
+        element_type=element_type, batch_type=batch_type)
+
+  return None

Review Comment:
   yep, this is handled when we try construction all the registered implementations: https://github.com/apache/beam/blob/7cc48e9dc3f8c44b0cc5dcfd2cd4eca51096ecdc/sdks/python/apache_beam/typehints/batch.py#L77-L90
   
   Note this is very naive right now. In the future this should include helpful debug information to handle cases where one or more implementations _almost_ matches. Tracked in #21654



##########
sdks/python/apache_beam/dataframe/convert.py:
##########
@@ -94,14 +119,58 @@ def _make_unbatched_pcoll(
     label += " with indexes"
 
   if label not in UNBATCHED_CACHE:
-    UNBATCHED_CACHE[label] = pc | label >> schemas.UnbatchPandas(
-        expr.proxy(), include_indexes=include_indexes)
+    proxy = expr.proxy()
+    shim_dofn: beam.DoFn
+    if isinstance(proxy, pd.DataFrame):
+      shim_dofn = DataFrameToRowsFn(proxy, include_indexes)
+    elif isinstance(proxy, pd.Series):
+      if include_indexes:
+        warnings.warn(
+            "Pipeline is converting a DeferredSeries to PCollection "
+            "with include_indexes=True. Note that this parameter is "
+            "_not_ respected for DeferredSeries conversion. To "
+            "include the index with your data, produce a"
+            "DeferredDataFrame instead.")
+
+      shim_dofn = SeriesToElementsFn(proxy)
+    else:
+      raise TypeError(f"Proxy '{proxy}' has unsupported type '{type(proxy)}'")
+
+    UNBATCHED_CACHE[label] = pc | label >> beam.ParDo(shim_dofn)
 
   # Note unbatched cache is keyed by the expression id as well as parameters
   # for the unbatching (i.e. include_indexes)
   return UNBATCHED_CACHE[label]
 
 
+class DataFrameToRowsFn(beam.DoFn):
+  def __init__(self, proxy, include_indexes):
+    self._proxy = proxy
+    self._include_indexes = include_indexes
+
+  @beam.DoFn.yields_batches
+  def process(self, element: pd.DataFrame) -> Iterable[pd.DataFrame]:
+    yield element
+
+  def infer_output_type(self, input_element_type):
+    return element_typehint_from_dataframe_proxy(
+        self._proxy, self._include_indexes)
+
+
+class SeriesToElementsFn(beam.DoFn):
+  def __init__(self, proxy):
+    self._proxy = proxy
+
+  @beam.DoFn.yields_batches
+  def process(self, element: pd.Series) -> Iterable[pd.Series]:
+    yield element
+
+  def infer_output_type(self, input_element_type):
+    # Raise a TypeError if proxy has an unknown type

Review Comment:
   Oops, this comment references behavior that was removed in https://github.com/apache/beam/commit/2b0597e9309711807669f5d22fa29042b944c077
   
   Now we will just shunt to Any in this case. I removed the comment. Thanks for raising this!



##########
sdks/python/apache_beam/typehints/pandas_type_compatibility.py:
##########
@@ -0,0 +1,299 @@
+#
+# 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.
+#
+
+r"""Utilities for converting between Beam schemas and pandas DataFrames.
+
+Imposes a mapping between native Python typings (specifically those compatible
+with :mod:`apache_beam.typehints.schemas`), and common pandas dtypes::
+
+  pandas dtype                    Python typing
+  np.int{8,16,32,64}      <-----> np.int{8,16,32,64}*
+  pd.Int{8,16,32,64}Dtype <-----> Optional[np.int{8,16,32,64}]*
+  np.float{32,64}         <-----> Optional[np.float{32,64}]
+                             \--- np.float{32,64}
+  Not supported           <------ Optional[bytes]
+  np.bool                 <-----> np.bool
+  np.dtype('S')           <-----> bytes
+  pd.BooleanDType()       <-----> Optional[bool]
+  pd.StringDType()        <-----> Optional[str]
+                             \--- str
+  np.object               <-----> Any
+
+  * int, float, bool are treated the same as np.int64, np.float64, np.bool
+
+Note that when converting to pandas dtypes, any types not specified here are
+shunted to ``np.object``.
+
+Similarly when converting from pandas to Python types, types that aren't
+otherwise specified here are shunted to ``Any``. Notably, this includes
+``np.datetime64``.
+
+Pandas does not support hierarchical data natively. Currently, all structured
+types (``Sequence``, ``Mapping``, nested ``NamedTuple`` types), are
+shunted to ``np.object`` like all other unknown types. In the future these
+types may be given special consideration.
+
+Note utilities in this package are for internal use only, we make no backward
+compatibility guarantees, except for the type mapping itself.
+"""
+
+from typing import Any
+from typing import List
+from typing import Optional
+
+import numpy as np
+import pandas as pd
+
+from apache_beam.typehints.batch import BatchConverter
+from apache_beam.typehints.row_type import RowTypeConstraint
+from apache_beam.typehints.typehints import is_nullable
+from apache_beam.typehints.typehints import normalize
+
+# No public API currently, this just exists to register BatchConverter
+# implementations.
+__all__ = []
+
+# Name for a valueless field-level option which, when present, indicates that
+# a field should map to an index in the Beam DataFrame API.
+INDEX_OPTION_NAME = 'beam:dataframe:index'
+
+# Generate type map (presented visually in the docstring)
+_BIDIRECTIONAL = [
+    (bool, bool),
+    (np.int8, np.int8),
+    (np.int16, np.int16),
+    (np.int32, np.int32),
+    (np.int64, np.int64),
+    (pd.Int8Dtype(), Optional[np.int8]),
+    (pd.Int16Dtype(), Optional[np.int16]),
+    (pd.Int32Dtype(), Optional[np.int32]),
+    (pd.Int64Dtype(), Optional[np.int64]),
+    (np.float32, Optional[np.float32]),
+    (np.float64, Optional[np.float64]),
+    (object, Any),
+    (pd.StringDtype(), Optional[str]),
+    (pd.BooleanDtype(), Optional[bool]),
+]
+
+PANDAS_TO_BEAM = {
+    pd.Series([], dtype=dtype).dtype: fieldtype
+    for dtype,
+    fieldtype in _BIDIRECTIONAL
+}
+BEAM_TO_PANDAS = {fieldtype: dtype for dtype, fieldtype in _BIDIRECTIONAL}
+
+# Shunt non-nullable Beam types to the same pandas types as their non-nullable
+# equivalents for FLOATs, DOUBLEs, and STRINGs. pandas has no non-nullable dtype
+# for these.
+OPTIONAL_SHUNTS = [np.float32, np.float64, str]
+
+for typehint in OPTIONAL_SHUNTS:
+  BEAM_TO_PANDAS[typehint] = BEAM_TO_PANDAS[Optional[typehint]]
+
+# int, float -> int64, np.float64
+BEAM_TO_PANDAS[int] = BEAM_TO_PANDAS[np.int64]
+BEAM_TO_PANDAS[Optional[int]] = BEAM_TO_PANDAS[Optional[np.int64]]
+BEAM_TO_PANDAS[float] = BEAM_TO_PANDAS[np.float64]
+BEAM_TO_PANDAS[Optional[float]] = BEAM_TO_PANDAS[Optional[np.float64]]
+
+BEAM_TO_PANDAS[bytes] = 'bytes'
+
+# Add shunts for normalized (Beam) typehints as well
+BEAM_TO_PANDAS.update({
+    normalize(typehint): pandas_dtype
+    for (typehint, pandas_dtype) in BEAM_TO_PANDAS.items()
+})
+
+
+def dtype_from_typehint(typehint):
+  # Default to np.object. This is lossy, we won't be able to recover
+  # the type at the output.
+  return BEAM_TO_PANDAS.get(typehint, object)
+
+
+def dtype_to_fieldtype(dtype):
+  fieldtype = PANDAS_TO_BEAM.get(dtype)
+
+  if fieldtype is not None:
+    return fieldtype
+  elif dtype.kind == 'S':
+    return bytes
+  else:
+    return Any
+
+
+@BatchConverter.register
+def create_pandas_batch_converter(
+    element_type: type, batch_type: type) -> BatchConverter:
+  if batch_type == pd.DataFrame:
+    return DataFrameBatchConverter.from_typehints(
+        element_type=element_type, batch_type=batch_type)
+  elif batch_type == pd.Series:
+    return SeriesBatchConverter.from_typehints(
+        element_type=element_type, batch_type=batch_type)
+
+  return None
+
+
+class DataFrameBatchConverter(BatchConverter):
+  def __init__(
+      self,
+      element_type: RowTypeConstraint,
+  ):
+    super().__init__(pd.DataFrame, element_type)
+    self._columns = [name for name, _ in element_type._fields]
+
+  @staticmethod
+  def from_typehints(element_type,
+                     batch_type) -> Optional['DataFrameBatchConverter']:
+    if not batch_type == pd.DataFrame:
+      return None
+
+    if not isinstance(element_type, RowTypeConstraint):
+      element_type = RowTypeConstraint.from_user_type(element_type)
+      if element_type is None:
+        return None
+
+    index_columns = [
+        field_name
+        for (field_name, field_options) in element_type._field_options.items()
+        if any(key == INDEX_OPTION_NAME for key, value in field_options)
+    ]
+
+    if index_columns:
+      return DataFrameBatchConverterKeepIndex(element_type, index_columns)
+    else:
+      return DataFrameBatchConverterDropIndex(element_type)
+
+  def _get_series(self, batch: pd.DataFrame):
+    raise NotImplementedError
+
+  def explode_batch(self, batch: pd.DataFrame):
+    # TODO: Only do null checks for nullable types
+    def make_null_checking_generator(series):
+      nulls = pd.isnull(series)
+      return (None if isnull else value for isnull, value in zip(nulls, series))
+
+    all_series = self._get_series(batch)
+    iterators = [make_null_checking_generator(series) for series in all_series]
+
+    for values in zip(*iterators):
+      yield self._element_type.user_type(
+          **{column: value
+             for column, value in zip(self._columns, values)})
+
+  def combine_batches(self, batches: List[pd.DataFrame]):
+    return pd.concat(batches)
+
+  def estimate_byte_size(self, batch: pd.DataFrame):
+    return batch.memory_usage().sum()
+
+  def get_length(self, batch: pd.DataFrame):
+    return len(batch)
+
+
+class DataFrameBatchConverterDropIndex(DataFrameBatchConverter):
+  """A DataFrameBatchConverter that assumes the DataFrame index has no meaning.
+
+  When producing a DataFrame from Rows, a meaningless index will be generated.
+  When exploding a DataFrame into Rows, the index will be dropped.
+  """
+  def _get_series(self, batch: pd.DataFrame):
+    return [batch[column] for column in batch.columns]
+
+  def produce_batch(self, elements):
+    # Note from_records has an index= parameter
+    batch = pd.DataFrame.from_records(elements, columns=self._columns)
+
+    for column, typehint in self._element_type._fields:
+      batch[column] = batch[column].astype(dtype_from_typehint(typehint))
+
+    return batch
+
+
+class DataFrameBatchConverterKeepIndex(DataFrameBatchConverter):
+  """A DataFrameBatchConverter that preserves the DataFrame index.
+
+  This is tracked via options on the Beam schema. Each field in the schema that
+  should map to the index is tagged in an option with name 'dataframe:index'.
+  """
+  def __init__(self, element_type: RowTypeConstraint, index_columns: List[Any]):
+    super().__init__(element_type)
+    self._index_columns = index_columns
+
+  def _get_series(self, batch: pd.DataFrame):
+    assert list(batch.index.names) == self._index_columns
+    return [
+        batch.index.get_level_values(i) for i in range(len(batch.index.names))
+    ] + [batch[column] for column in batch.columns]
+
+  def produce_batch(self, elements):
+    # Note from_records has an index= parameter
+    batch = pd.DataFrame.from_records(elements, columns=self._columns)
+
+    for column, typehint in self._element_type._fields:
+      batch[column] = batch[column].astype(dtype_from_typehint(typehint))
+
+    return batch.set_index(self._index_columns)
+
+
+class SeriesBatchConverter(BatchConverter):
+  def __init__(
+      self,
+      element_type: type,
+      dtype,
+  ):
+    super().__init__(pd.DataFrame, element_type)
+    self._dtype = dtype
+
+    if is_nullable(element_type):
+
+      def unbatch(series):
+        for isnull, value in zip(pd.isnull(series), series):
+          yield None if isnull else value
+    else:
+
+      def unbatch(series):
+        yield from series
+
+    self.explode_batch = unbatch
+
+  @staticmethod
+  def from_typehints(element_type,
+                     batch_type) -> Optional['SeriesBatchConverter']:
+    if not batch_type == pd.Series:
+      return None
+
+    dtype = dtype_from_typehint(element_type)
+
+    return SeriesBatchConverter(element_type, dtype)
+
+  def produce_batch(self, elements: List[Any]) -> pd.Series:
+    return pd.Series(elements, dtype=self._dtype)
+
+  def explode_batch(self, batch: pd.Series):
+    raise NotImplementedError(
+        "explode_batch should be generated in SeriesBatchConverter.__init__")

Review Comment:
   We branch on `is_nullable` one time in `__init__` and assign explode_batch with a null-checking or non-null-checking alternative.



##########
sdks/python/apache_beam/typehints/pandas_type_compatibility_test.py:
##########
@@ -0,0 +1,191 @@
+#
+# 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.
+#
+
+"""Unit tests for pandas batched type converters."""
+
+import unittest
+from typing import Optional
+
+import numpy as np
+import pandas as pd
+from parameterized import parameterized
+from parameterized import parameterized_class
+
+from apache_beam.typehints import row_type
+from apache_beam.typehints import typehints
+from apache_beam.typehints.batch import BatchConverter
+
+
+@parameterized_class([
+    {
+        'batch_typehint': pd.DataFrame,
+        'element_typehint': row_type.RowTypeConstraint.from_fields([
+            ('foo', int),
+            ('bar', float),
+            ('baz', str),
+        ]),
+        'match_index': False,
+        'batch': pd.DataFrame({
+            'foo': pd.Series(range(100), dtype='int64'),
+            'bar': pd.Series([i / 100 for i in range(100)], dtype='float64'),
+            'baz': pd.Series([str(i) for i in range(100)],
+                             dtype=pd.StringDtype()),
+        }),
+    },
+    {
+        'batch_typehint': pd.DataFrame,
+        'element_typehint': row_type.RowTypeConstraint.from_fields(
+            [
+                ('an_index', int),
+                ('foo', int),
+                ('bar', float),
+                ('baz', str),
+            ],
+            field_options={'an_index': [('beam:dataframe:index', None)]},
+        ),
+        'match_index': True,
+        'batch': pd.DataFrame({
+            'foo': pd.Series(range(100), dtype='int64'),
+            'bar': pd.Series([i / 100 for i in range(100)], dtype='float64'),
+            'baz': pd.Series([str(i) for i in range(100)],
+                             dtype=pd.StringDtype()),
+        }).set_index(pd.Int64Index(range(123, 223), name='an_index')),
+    },
+    {
+        'batch_typehint': pd.DataFrame,
+        'element_typehint': row_type.RowTypeConstraint.from_fields(
+            [
+                ('an_index', int),
+                ('another_index', int),
+                ('foo', int),
+                ('bar', float),
+                ('baz', str),
+            ],
+            field_options={
+                'an_index': [('beam:dataframe:index', None)],
+                'another_index': [('beam:dataframe:index', None)],
+            }),
+        'match_index': True,
+        'batch': pd.DataFrame({
+            'foo': pd.Series(range(100), dtype='int64'),
+            'bar': pd.Series([i / 100 for i in range(100)], dtype='float64'),
+            'baz': pd.Series([str(i) for i in range(100)],
+                             dtype=pd.StringDtype()),
+        }).set_index([
+            pd.Int64Index(range(123, 223), name='an_index'),
+            pd.Int64Index(range(475, 575), name='another_index'),
+        ]),
+    },
+    {
+        'batch_typehint': pd.Series,
+        'element_typehint': int,
+        'match_index': False,
+        'batch': pd.Series(range(500)),
+    },
+    {
+        'batch_typehint': pd.Series,
+        'element_typehint': str,
+        'batch': pd.Series(['foo', 'bar', 'baz', 'def', 'ghi', 'abc'] * 10,
+                           dtype=pd.StringDtype()),
+    },
+    {
+        'batch_typehint': pd.Series,
+        'element_typehint': Optional[np.int64],
+        'batch': pd.Series((i if i % 11 else None for i in range(500)),
+                           dtype=pd.Int64Dtype()),
+    },
+    {
+        'batch_typehint': pd.Series,
+        'element_typehint': Optional[str],
+        'batch': pd.Series(['foo', None, 'bar', 'baz', None, 'def', 'ghi'] * 10,
+                           dtype=pd.StringDtype()),
+    },
+])
+class DataFrameBatchConverterTest(unittest.TestCase):
+  def create_batch_converter(self):
+    return BatchConverter.from_typehints(
+        element_type=self.element_typehint, batch_type=self.batch_typehint)
+
+  def setUp(self):
+    self.converter = self.create_batch_converter()
+    self.normalized_batch_typehint = typehints.normalize(self.batch_typehint)
+    self.normalized_element_typehint = typehints.normalize(
+        self.element_typehint)
+
+  def equality_check(self, left, right):
+    if isinstance(left, pd.DataFrame):
+      if self.match_index:
+        pd.testing.assert_frame_equal(left.sort_index(), right.sort_index())
+      else:
+        pd.testing.assert_frame_equal(
+            left.sort_values(by=list(left.columns)).reset_index(drop=True),
+            right.sort_values(by=list(right.columns)).reset_index(drop=True))
+    elif isinstance(left, pd.Series):
+      pd.testing.assert_series_equal(
+          left.sort_values().reset_index(drop=True),
+          right.sort_values().reset_index(drop=True))
+    else:
+      raise TypeError(f"Encountered unexpected type, left is a {type(left)!r}")

Review Comment:
   `assert_series_equal` or `assert_frame_equal` will raise if `right` isn't the appropriate type.



-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org