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/03 18:51:00 UTC

[GitHub] [beam] TheNeuralBit opened a new pull request, #22575: WIP: Use Batched DoFns in DataFrame convert utilities

TheNeuralBit opened a new pull request, #22575:
URL: https://github.com/apache/beam/pull/22575

   
   
   GitHub Actions Tests Status (on master branch)
   ------------------------------------------------------------------------------------------------
   [![Build python source distribution and wheels](https://github.com/apache/beam/workflows/Build%20python%20source%20distribution%20and%20wheels/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Build+python+source+distribution+and+wheels%22+branch%3Amaster+event%3Aschedule)
   [![Python tests](https://github.com/apache/beam/workflows/Python%20tests/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Python+Tests%22+branch%3Amaster+event%3Aschedule)
   [![Java tests](https://github.com/apache/beam/workflows/Java%20Tests/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Java+Tests%22+branch%3Amaster+event%3Aschedule)
   [![Go tests](https://github.com/apache/beam/workflows/Go%20tests/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Go+tests%22+branch%3Amaster+event%3Aschedule)
   
   See [CI.md](https://github.com/apache/beam/blob/master/CI.md) for more information about GitHub Actions CI.
   


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


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

Posted by GitBox <gi...@apache.org>.
yeandy commented on code in PR #22575:
URL: https://github.com/apache/beam/pull/22575#discussion_r958426976


##########
sdks/python/apache_beam/typehints/pandas_type_compatibility_test.py:
##########
@@ -159,26 +159,46 @@ def test_explode_rebatch(self):
     typehints.check_constraint(self.normalized_batch_typehint, rebatched)
     self.equality_check(self.batch, rebatched)
 
+  def _split_batch_into_n_partitions(self, N):
+    elements = list(self.converter.explode_batch(self.batch))
+
+    # Split elements into N contiguous partitions
+    element_batches = [
+        elements[len(elements) * i // N:len(elements) * (i + 1) // N]
+        for i in range(N)
+    ]
+
+    lengths = [len(element_batch) for element_batch in element_batches]
+    batches = [self.converter.produce_batch(element_batch)
+               for element_batch in element_batches]
+
+    return batches, lengths
+
   @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)
-    ]
+    batches, _ = self._split_batch_into_n_partitions(N)
 
     # Combine the batches, output should be equivalent to the original batch
     combined = self.converter.combine_batches(batches)
 
     self.equality_check(self.batch, combined)
 
+  @parameterized.expand([
+      (2, ),
+      (3, ),
+      (10, ),
+  ])
+  def test_get_lenth(self, N):

Review Comment:
   ```suggestion
     def test_get_length(self, N):
   ```



##########
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:
   Thanks!



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

Review Comment:
   Nit. I actually don't mind the extra lines, especially since we're defining functions here, so it's easier to read. I'll leave it up to you.
   ```suggestion
       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
   ```



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


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

Posted by GitBox <gi...@apache.org>.
TheNeuralBit commented on PR #22575:
URL: https://github.com/apache/beam/pull/22575#issuecomment-1213572473

   CC: @robertwb 


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


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

Posted by GitBox <gi...@apache.org>.
TheNeuralBit commented on PR #22575:
URL: https://github.com/apache/beam/pull/22575#issuecomment-1233005122

   retest this please


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


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

Posted by GitBox <gi...@apache.org>.
yeandy commented on code in PR #22575:
URL: https://github.com/apache/beam/pull/22575#discussion_r956090750


##########
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:
   ```suggestion
     def test_type_check_batch(self):
   ```



##########
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:
   Should we also be checking against the type of `right`?



##########
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:
   Can we add tests for these? And also for `SeriesBatchConverter`?



##########
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:
   Why is should this generated in `__init__`?



##########
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:
   I may have missed this, but where does the error get raised?



##########
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:
   Is there an issue for 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)

Review Comment:
   Why don't we use `index=` parameter here? Is it so it's easier to set the data type in the next 2 lines?



##########
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:
   Can you explain the purpose of checking the equality both ways?



##########
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:
   Could we zip the `self._columns` along with the `iterators`? Might make it harder to read though



##########
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:
   what happens if we return `None`? Do we have checks in other places to detect for a `None` BatchConvertor?



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


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

Posted by GitBox <gi...@apache.org>.
TheNeuralBit commented on PR #22575:
URL: https://github.com/apache/beam/pull/22575#issuecomment-1233420470

   PythonDocs PreCommit has passed (https://ci-beam.apache.org/job/beam_PreCommit_PythonDocs_Commit/9575/), merging


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


[GitHub] [beam] TheNeuralBit commented on pull request #22575: WIP: Use Batched DoFns in DataFrame convert utilities

Posted by GitBox <gi...@apache.org>.
TheNeuralBit commented on PR #22575:
URL: https://github.com/apache/beam/pull/22575#issuecomment-1205833846

   Run Python 3.7 PostCommit


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


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

Posted by GitBox <gi...@apache.org>.
TheNeuralBit commented on PR #22575:
URL: https://github.com/apache/beam/pull/22575#issuecomment-1233193924

   Run Python Examples_Dataflow


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


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

Posted by GitBox <gi...@apache.org>.
TheNeuralBit commented on PR #22575:
URL: https://github.com/apache/beam/pull/22575#issuecomment-1233194092

   Run Python 3.8 PostCommit


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


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

Posted by GitBox <gi...@apache.org>.
yeandy commented on code in PR #22575:
URL: https://github.com/apache/beam/pull/22575#discussion_r958421050


##########
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:
   I was originally thinking of 
   ```
       for values, columns in zip(*iterators, self._columns):
          ...
   ```
   But I had to take a look again to wrap my head around it. Looks like you're zipping to create the rows first, and then in the second zip, you line them up with the column names. The length of an individual iterator in `iterators` isn't necessarily the same as the length of `self._columns`. Plus, we'd probably get `too many values to unpack` error if we had `values, columns`.



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


[GitHub] [beam] TheNeuralBit commented on pull request #22575: WIP: Use Batched DoFns in DataFrame convert utilities

Posted by GitBox <gi...@apache.org>.
TheNeuralBit commented on PR #22575:
URL: https://github.com/apache/beam/pull/22575#issuecomment-1205881429

   > Clarify separation of concerns between pandas_type_compatibility and dataframe.schemas
   
   `dataframe.schemas`:
   - Maintain its current public API (possibly with deprecation notices)
   - Responsible for making proxies for the DataFrame API
   
   `typehints.pandas_type_compatibility`:
   - pandas-Beam type mapping
   - BatchConverter implementations
   


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


[GitHub] [beam] github-actions[bot] commented on pull request #22575: Add BatchConverter implementations for pandas types, use Batched DoFns in DataFrame convert utilities

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #22575:
URL: https://github.com/apache/beam/pull/22575#issuecomment-1226608156

   Stopping reviewer notifications for this pull request: review requested by someone other than the bot, ceding control


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


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

Posted by GitBox <gi...@apache.org>.
TheNeuralBit commented on PR #22575:
URL: https://github.com/apache/beam/pull/22575#issuecomment-1232341573

   Run Python 3.8 PostCommit


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


[GitHub] [beam] github-actions[bot] commented on pull request #22575: Add BatchConverter implementations for pandas types, use Batched DoFns in DataFrame convert utilities

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #22575:
URL: https://github.com/apache/beam/pull/22575#issuecomment-1221303930

   Reminder, please take a look at this pr: @y1chi 


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


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

Posted by GitBox <gi...@apache.org>.
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


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

Posted by GitBox <gi...@apache.org>.
TheNeuralBit commented on PR #22575:
URL: https://github.com/apache/beam/pull/22575#issuecomment-1233193615

   retest this please


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


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

Posted by GitBox <gi...@apache.org>.
TheNeuralBit commented on PR #22575:
URL: https://github.com/apache/beam/pull/22575#issuecomment-1232344461

   Run Python Examples_Direct


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


[GitHub] [beam] codecov[bot] commented on pull request #22575: WIP: Use Batched DoFns in DataFrame convert utilities

Posted by GitBox <gi...@apache.org>.
codecov[bot] commented on PR #22575:
URL: https://github.com/apache/beam/pull/22575#issuecomment-1205848143

   # [Codecov](https://codecov.io/gh/apache/beam/pull/22575?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#22575](https://codecov.io/gh/apache/beam/pull/22575?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (f3ae667) into [master](https://codecov.io/gh/apache/beam/commit/bf39489b2a1fd45e6798483d083e4ad240f66891?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (bf39489) will **increase** coverage by `0.07%`.
   > The diff coverage is `92.98%`.
   
   ```diff
   @@            Coverage Diff             @@
   ##           master   #22575      +/-   ##
   ==========================================
   + Coverage   74.17%   74.25%   +0.07%     
   ==========================================
     Files         706      708       +2     
     Lines       93237    93458     +221     
   ==========================================
   + Hits        69162    69395     +233     
   + Misses      22807    22795      -12     
     Partials     1268     1268              
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | python | `83.62% <92.98%> (+0.07%)` | :arrow_up: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/beam/pull/22575?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...apache\_beam/typehints/pandas\_type\_compatibility.py](https://codecov.io/gh/apache/beam/pull/22575/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vdHlwZWhpbnRzL3BhbmRhc190eXBlX2NvbXBhdGliaWxpdHkucHk=) | `90.62% <90.62%> (ø)` | |
   | [sdks/python/apache\_beam/dataframe/convert.py](https://codecov.io/gh/apache/beam/pull/22575/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vZGF0YWZyYW1lL2NvbnZlcnQucHk=) | `91.20% <93.18%> (+0.83%)` | :arrow_up: |
   | [sdks/python/apache\_beam/coders/row\_coder.py](https://codecov.io/gh/apache/beam/pull/22575/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vY29kZXJzL3Jvd19jb2Rlci5weQ==) | `94.54% <100.00%> (+0.05%)` | :arrow_up: |
   | [sdks/python/apache\_beam/dataframe/schemas.py](https://codecov.io/gh/apache/beam/pull/22575/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vZGF0YWZyYW1lL3NjaGVtYXMucHk=) | `95.62% <100.00%> (-2.02%)` | :arrow_down: |
   | [sdks/python/apache\_beam/transforms/core.py](https://codecov.io/gh/apache/beam/pull/22575/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vdHJhbnNmb3Jtcy9jb3JlLnB5) | `92.94% <100.00%> (ø)` | |
   | [sdks/python/apache\_beam/typehints/row\_type.py](https://codecov.io/gh/apache/beam/pull/22575/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vdHlwZWhpbnRzL3Jvd190eXBlLnB5) | `100.00% <100.00%> (+3.44%)` | :arrow_up: |
   | [sdks/python/apache\_beam/typehints/schemas.py](https://codecov.io/gh/apache/beam/pull/22575/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vdHlwZWhpbnRzL3NjaGVtYXMucHk=) | `94.35% <100.00%> (+0.03%)` | :arrow_up: |
   | [...python/apache\_beam/runners/worker/worker\_status.py](https://codecov.io/gh/apache/beam/pull/22575/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy93b3JrZXIvd29ya2VyX3N0YXR1cy5weQ==) | `77.53% <0.00%> (-2.18%)` | :arrow_down: |
   | [sdks/python/apache\_beam/io/source\_test\_utils.py](https://codecov.io/gh/apache/beam/pull/22575/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vaW8vc291cmNlX3Rlc3RfdXRpbHMucHk=) | `88.01% <0.00%> (-1.39%)` | :arrow_down: |
   | ... and [13 more](https://codecov.io/gh/apache/beam/pull/22575/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   :mega: Codecov can now indicate which changes are the most critical in Pull Requests. [Learn more](https://about.codecov.io/product/feature/runtime-insights/?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   


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


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

Posted by GitBox <gi...@apache.org>.
TheNeuralBit commented on PR #22575:
URL: https://github.com/apache/beam/pull/22575#issuecomment-1226607628

   R: @yeandy 


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


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

Posted by GitBox <gi...@apache.org>.
yeandy commented on code in PR #22575:
URL: https://github.com/apache/beam/pull/22575#discussion_r958421336


##########
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:
   Got it, thanks!



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


[GitHub] [beam] github-actions[bot] commented on pull request #22575: Add BatchConverter implementations for pandas types, use Batched DoFns in DataFrame convert utilities

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #22575:
URL: https://github.com/apache/beam/pull/22575#issuecomment-1213606792

   Assigning reviewers. If you would like to opt out of this review, comment `assign to next reviewer`:
   
   R: @y1chi for label python.
   
   Available commands:
   - `stop reviewer notifications` - opt out of the automated review tooling
   - `remind me after tests pass` - tag the comment author after tests pass
   - `waiting on author` - shift the attention set back to the author (any comment or push by the author will return the attention set to the reviewers)
   
   The PR bot will only process comments in the main thread (not review comments).


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


[GitHub] [beam] TheNeuralBit commented on pull request #22575: WIP: Use Batched DoFns in DataFrame convert utilities

Posted by GitBox <gi...@apache.org>.
TheNeuralBit commented on PR #22575:
URL: https://github.com/apache/beam/pull/22575#issuecomment-1204350066

   Run Python 3.8 PostCommit


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


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

Posted by GitBox <gi...@apache.org>.
TheNeuralBit commented on PR #22575:
URL: https://github.com/apache/beam/pull/22575#issuecomment-1222614865

   @y1chi do you have time to review this?


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


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

Posted by GitBox <gi...@apache.org>.
TheNeuralBit commented on PR #22575:
URL: https://github.com/apache/beam/pull/22575#issuecomment-1232344574

   Run Python Examples_Dataflow


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


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

Posted by GitBox <gi...@apache.org>.
TheNeuralBit merged PR #22575:
URL: https://github.com/apache/beam/pull/22575


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


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

Posted by GitBox <gi...@apache.org>.
TheNeuralBit commented on PR #22575:
URL: https://github.com/apache/beam/pull/22575#issuecomment-1233193801

   Run Python Examples_Direct


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


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

Posted by GitBox <gi...@apache.org>.
TheNeuralBit commented on PR #22575:
URL: https://github.com/apache/beam/pull/22575#issuecomment-1213590723

   Run Python 3.8 PostCommit


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