This is an automated email from the ASF dual-hosted git repository.

tvalentyn pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/beam.git


The following commit(s) were added to refs/heads/master by this push:
     new 2ca9af8ecbc Upgrade Pandas dependency to 2.1 (#31185)
2ca9af8ecbc is described below

commit 2ca9af8ecbcb3d0cf403bb4ce7d90a8a362b124d
Author: tvalentyn <tvalen...@users.noreply.github.com>
AuthorDate: Tue May 7 10:19:11 2024 -0700

    Upgrade Pandas dependency to 2.1 (#31185)
    
    * Upgrade to Pandas 2.1
    
    * Pandas 2.1: Disable interchange protocol tests.
    
    * Exclude attrs tests as it is not supported.
    
    * Exclude new doctests that exercise unsupported order-sensitive ops.
    
    * Iteration over deferred DFs is not supported
    
    * Skip 'mul' op when index is used as an axis
    
    * Exclude new tests that use index.
    
    * Exclude shift test as order-sensitive.
    
    * Exclude known failure modes.
    
    * Exclude failures that existed on Pandas 1.
    
    * Allow bulk-exclusion of an example in all tests.
    
    * Exclude examples that use to_timedelta.
    
    * Exclude the test that evaluates an inferred .tz value.
    
    * Exclude more tz and timedelta tests.
    
    * Exclude a tests exercision PeriodProporties.end_time
    
    * Exclude tests exercising unsupported GroupBy operations.
    
    * Expand the list of elementwise string methods.
    
    * Exclude known WontImpl ops
    
    * Fix test output normalization.
    
    * Exclude remaining new tests that didn't work
    
    * Remove test that uses values, an unsupported non-deferred op.
    
    * lint
---
 sdks/python/apache_beam/dataframe/doctests.py      |  16 ++-
 sdks/python/apache_beam/dataframe/frames.py        |  19 ++-
 sdks/python/apache_beam/dataframe/frames_test.py   |  14 ++
 .../apache_beam/dataframe/pandas_doctests_test.py  | 150 ++++++++++++++++++++-
 sdks/python/setup.py                               |   6 +-
 5 files changed, 190 insertions(+), 15 deletions(-)

diff --git a/sdks/python/apache_beam/dataframe/doctests.py 
b/sdks/python/apache_beam/dataframe/doctests.py
index a7cff5af5ab..61e904c4f9d 100644
--- a/sdks/python/apache_beam/dataframe/doctests.py
+++ b/sdks/python/apache_beam/dataframe/doctests.py
@@ -225,6 +225,8 @@ class 
_DeferrredDataframeOutputChecker(doctest.OutputChecker):
 
   def fix(self, want, got):
     if 'DeferredBase' in got:
+      # When we have a tuple of Dataframes, pandas prints each from a new line.
+      got = re.sub(r'DeferredBase\[(\d+)\],', '\\g<0>\n', got)
       try:
         to_compute = {
             m.group(0): self._env._all_frames[int(m.group(1))]
@@ -381,20 +383,23 @@ class BeamDataframeDoctestRunner(doctest.DocTestRunner):
     self._skipped_set = set()
 
   def _is_wont_implement_ok(self, example, test):
+    always_wont_implement = self._wont_implement_ok.get('*', [])
     return any(
-        wont_implement(example)
-        for wont_implement in self._wont_implement_ok.get(test.name, []))
+        wont_implement(example) for wont_implement in (
+            self._wont_implement_ok.get(test.name, []) + 
always_wont_implement))
 
   def _is_not_implemented_ok(self, example, test):
+    always_not_impl = self._not_implemented_ok.get('*', [])
     return any(
-        not_implemented(example)
-        for not_implemented in self._not_implemented_ok.get(test.name, []))
+        not_implemented(example) for not_implemented in (
+            self._not_implemented_ok.get(test.name, []) + always_not_impl))
 
   def run(self, test, **kwargs):
     self._checker.reset()
+    always_skip = self._skip.get('*', [])
     for example in test.examples:
       if any(should_skip(example)
-             for should_skip in self._skip.get(test.name, [])):
+             for should_skip in self._skip.get(test.name, []) + always_skip):
         self._skipped_set.add(example)
         example.source = 'pass'
         example.want = ''
@@ -726,6 +731,7 @@ def with_run_patched_docstring(target=None):
         verify the examples, else use PartitioningSession to simulate
         distributed execution.
       skip (Dict[str,str]): A set of examples to skip entirely.
+        If a key is '*', an example will be skipped in all test scenarios.
       wont_implement_ok (Dict[str,str]): A set of examples that are allowed to
         raise WontImplementError.
       not_implemented_ok (Dict[str,str]): A set of examples that are allowed to
diff --git a/sdks/python/apache_beam/dataframe/frames.py 
b/sdks/python/apache_beam/dataframe/frames.py
index b7aa130fbbd..421430ec972 100644
--- a/sdks/python/apache_beam/dataframe/frames.py
+++ b/sdks/python/apache_beam/dataframe/frames.py
@@ -1181,8 +1181,11 @@ class 
DeferredDataFrameOrSeries(frame_base.DeferredFrame):
       pd.DataFrame, 'hist', reason="plotting-tools")
 
   attrs = property(
-      frame_base.wont_implement_method(
-          pd.DataFrame, 'attrs', reason='experimental'))
+      fget=frame_base.wont_implement_method(
+          pd.DataFrame, 'attrs', reason='experimental'),
+      fset=frame_base.wont_implement_method(
+          pd.DataFrame, 'attrs', reason='experimental'),
+  )
 
   reorder_levels = frame_base._proxy_method(
       'reorder_levels',
@@ -5124,13 +5127,18 @@ class _DeferredStringMethods(frame_base.DeferredBase):
 ELEMENTWISE_STRING_METHODS = [
             'capitalize',
             'casefold',
+            'center',
             'contains',
             'count',
+            'decode',
+            'encode',
             'endswith',
             'extract',
+            'find',
             'findall',
             'fullmatch',
             'get',
+            'index',
             'isalnum',
             'isalpha',
             'isdecimal',
@@ -5142,15 +5150,21 @@ ELEMENTWISE_STRING_METHODS = [
             'isupper',
             'join',
             'len',
+            'lfind',
+            'ljust',
             'lower',
             'lstrip',
             'match',
+            'normalize',
             'pad',
             'partition',
             'removeprefix',
             'removesuffix',
             'replace',
             'rpartition',
+            'rfind',
+            'rindex',
+            'rjust',
             'rstrip',
             'slice',
             'slice_replace',
@@ -5158,6 +5172,7 @@ ELEMENTWISE_STRING_METHODS = [
             'strip',
             'swapcase',
             'title',
+            'translate',
             'upper',
             'wrap',
             'zfill',
diff --git a/sdks/python/apache_beam/dataframe/frames_test.py 
b/sdks/python/apache_beam/dataframe/frames_test.py
index 6e32acefc61..3ee9adc2bfe 100644
--- a/sdks/python/apache_beam/dataframe/frames_test.py
+++ b/sdks/python/apache_beam/dataframe/frames_test.py
@@ -29,6 +29,7 @@ from apache_beam.dataframe import expressions
 from apache_beam.dataframe import frame_base
 from apache_beam.dataframe import frames
 from apache_beam.dataframe.convert import to_dataframe
+from apache_beam.dataframe.doctests import teststring
 from apache_beam.runners.interactive import interactive_beam as ib
 from apache_beam.runners.interactive import interactive_environment as ie
 from apache_beam.runners.interactive.interactive_runner import 
InteractiveRunner
@@ -363,6 +364,19 @@ class DeferredFrameTest(_AbstractFrameTest):
     })
     self._run_inplace_test(new_column, df)
 
+  def test_tz_with_utc_zone_set_explicitly(self):
+    test = """
+      >>> s = pd.Series(["1/1/2020 10:00:00+00:00", "2/1/2020 11:00:00+03:00"])
+      >>> s = pd.to_datetime(s, utc=True)
+      >>> s
+      0   2020-01-01 10:00:00+00:00
+      1   2020-02-01 08:00:00+00:00
+      dtype: datetime64[ns, UTC]
+      >>> s.dt.tz
+      datetime.timezone.utc
+    """
+    teststring(test)
+
   def test_tz_localize_ambiguous_series(self):
     # This replicates a tz_localize doctest:
     #   s.tz_localize('CET', ambiguous=np.array([True, True, False]))
diff --git a/sdks/python/apache_beam/dataframe/pandas_doctests_test.py 
b/sdks/python/apache_beam/dataframe/pandas_doctests_test.py
index 4fb05780fbe..a4bd0d0a812 100644
--- a/sdks/python/apache_beam/dataframe/pandas_doctests_test.py
+++ b/sdks/python/apache_beam/dataframe/pandas_doctests_test.py
@@ -39,10 +39,16 @@ class DoctestTest(unittest.TestCase):
         use_beam=False,
         report=True,
         wont_implement_ok={
+            'pandas.core.generic.NDFrame.attrs': ['*'],
+            'pandas.core.generic.NDFrame.bfill': ['*'],
+            'pandas.core.generic.NDFrame.ffill': ['*'],
+            'pandas.core.generic.NDFrame.first_valid_index': ['*'],
             'pandas.core.generic.NDFrame.head': ['*'],
+            'pandas.core.generic.NDFrame.last_valid_index': ['*'],
             'pandas.core.generic.NDFrame.shift': [
                 'df.shift(periods=3)',
                 'df.shift(periods=3, fill_value=0)',
+                "df['Col1'].shift(periods=[0, 1, 2])",
             ],
             'pandas.core.generic.NDFrame.tail': ['*'],
             'pandas.core.generic.NDFrame.take': ['*'],
@@ -103,6 +109,7 @@ class DoctestTest(unittest.TestCase):
             'pandas.core.generic.NDFrame.get': ['*'],
         },
         not_implemented_ok={
+            'pandas.core.generic.NDFrame.__iter__': ['*'],
             'pandas.core.generic.NDFrame.asof': ['*'],
             'pandas.core.generic.NDFrame.at_time': ['*'],
             'pandas.core.generic.NDFrame.between_time': ['*'],
@@ -224,6 +231,7 @@ class DoctestTest(unittest.TestCase):
             'pandas.core.frame.DataFrame.shift': [
                 'df.shift(periods=3)',
                 'df.shift(periods=3, fill_value=0)',
+                "df['Col1'].shift(periods=[0, 1, 2])",
             ],
             'pandas.core.frame.DataFrame.unstack': ['*'],
             'pandas.core.frame.DataFrame.memory_usage': ['*'],
@@ -303,6 +311,21 @@ class DoctestTest(unittest.TestCase):
             'pandas.core.frame.DataFrame.to_timestamp': ['*']
         },
         skip={
+            # These examples occur in docstrings for several ops.
+            '*': [
+                # mul doesn't work in Beam with axis='index'.
+                "df.mul({'circle': 0, 'triangle': 2, 'rectangle': 3}, "
+                  "axis='index')",
+                # eq doesn't work with axis='index'.
+                "df.eq([250, 250, 100], axis='index')",
+                # New test in Pandas 2.1 that uses indexes.
+                'df != pd.Series([100, 250], index=["cost", "revenue"])',
+                # New test in Pandas 2.1 that uses indexes.
+                'df.le(df_multindex, level=1)'
+
+            ],
+            # DeferredDataFrame  doesn't implement the DF interchange protocol.
+            'pandas.core.frame.DataFrame.__dataframe__': ['*'],
             # DataFrame construction from a dictionary, Series, or other
             # DataFrame requires using the len() function, which is a
             # non-deferred operation that we do not allow
@@ -349,6 +372,8 @@ class DoctestTest(unittest.TestCase):
                 # and fail with set_axis(axis='index')
                 "df.set_axis(['a', 'b', 'c'], axis='index')"
             ],
+            # Beam's implementation takes a filepath as an argument.
+            'pandas.core.frame.DataFrame.to_html': ['*'],
             'pandas.core.frame.DataFrame.to_markdown': ['*'],
             'pandas.core.frame.DataFrame.to_parquet': ['*'],
 
@@ -388,6 +413,18 @@ class DoctestTest(unittest.TestCase):
                 "          ignore_index=True)"
             ],
             'pandas.core.frame.DataFrame.eval': ['df'],
+            # Fails when result is a singleton:
+            # https://github.com/apache/beam/issues/28559
+            'pandas.core.frame.DataFrame.kurt': [
+                'df.kurt(axis=None).round(6)',
+                's.kurt()'
+            ],
+            # Fails when result is a singleton:
+            # https://github.com/apache/beam/issues/28559
+            'pandas.core.frame.DataFrame.sem': [
+                'df.sem().round(6)',
+                's.sem().round(6)'
+            ],
             'pandas.core.frame.DataFrame.melt': [
                 "df.columns = [list('ABC'), list('DEF')]", "df"
             ],
@@ -418,6 +455,10 @@ class DoctestTest(unittest.TestCase):
                 'df_copy.iloc[0, 0] = pd.NA',
                 "df_copy.applymap(lambda x: len(str(x)), na_action='ignore')",
             ],
+            'pandas.core.frame.DataFrame.map': [
+                'df_copy.iloc[0, 0] = pd.NA',
+                "df_copy.map(lambda x: len(str(x)), na_action='ignore')",
+            ],
             # Skipped so we don't need to install natsort
             'pandas.core.frame.DataFrame.sort_values': [
                 'from natsort import index_natsorted',
@@ -429,7 +470,8 @@ class DoctestTest(unittest.TestCase):
             # Mode that we don't yet support, documentation added in pandas
             # 1.2.0 (https://github.com/pandas-dev/pandas/issues/35912)
             'pandas.core.frame.DataFrame.aggregate': [
-                "df.agg(x=('A', max), y=('B', 'min'), z=('C', np.mean))"
+                "df.agg(x=('A', max), y=('B', 'min'), z=('C', np.mean))",
+                "df.agg(x=('A', 'max'), y=('B', 'min'), z=('C', 'mean'))"
             ],
         })
     self.assertEqual(result.failed, 0)
@@ -441,6 +483,7 @@ class DoctestTest(unittest.TestCase):
         report=True,
         wont_implement_ok={
             'pandas.core.series.Series.__array__': ['*'],
+            'pandas.core.series.Series.argsort': ['*'],
             'pandas.core.series.Series.array': ['*'],
             'pandas.core.series.Series.cummax': ['*'],
             'pandas.core.series.Series.cummin': ['*'],
@@ -471,6 +514,7 @@ class DoctestTest(unittest.TestCase):
                 "s.nsmallest(3, keep='last')",
             ],
             'pandas.core.series.Series.pop': ['*'],
+            'pandas.core.series.Series.ravel': ['*'],
             'pandas.core.series.Series.searchsorted': ['*'],
             'pandas.core.series.Series.shift': [
                 'df.shift(periods=3)',
@@ -478,6 +522,7 @@ class DoctestTest(unittest.TestCase):
             ],
             'pandas.core.series.Series.take': ['*'],
             'pandas.core.series.Series.to_dict': ['*'],
+            'pandas.core.series.Series.to_string': ['*'],
             'pandas.core.series.Series.unique': ['*'],
             'pandas.core.series.Series.unstack': ['*'],
             'pandas.core.series.Series.values': ['*'],
@@ -544,6 +589,8 @@ class DoctestTest(unittest.TestCase):
                 # Differs in LSB on jenkins.
                 "s1.cov(s2)",
             ],
+            # Test framework doesn't materialze DeferredIndex.
+            'pandas.core.series.Series.keys': ['s.keys()'],
             # Skipped idxmax/idxmin due an issue with the test framework
             'pandas.core.series.Series.idxmin': ['s.idxmin()'],
             'pandas.core.series.Series.idxmax': ['s.idxmax()'],
@@ -569,6 +616,18 @@ class DoctestTest(unittest.TestCase):
                 's'
             ],
             'pandas.core.series.Series.resample': ['df'],
+            # Fails when result is a singleton:
+            # https://github.com/apache/beam/issues/28559
+            'pandas.core.series.Series.kurt': [
+                'df.kurt(axis=None).round(6)',
+                's.kurt()'
+            ],
+            # Fails when result is a singleton:
+            # https://github.com/apache/beam/issues/28559
+            'pandas.core.series.Series.sem': [
+                'df.sem().round(6)',
+                's.sem().round(6)'
+            ],
         })
     self.assertEqual(result.failed, 0)
 
@@ -651,6 +710,7 @@ class DoctestTest(unittest.TestCase):
         pd.core.indexes.accessors,
         use_beam=False,
         skip={
+            '*': ["ser = pd.Series(pd.to_timedelta([1, 2, 3], unit='d'))"],
             'pandas.core.indexes.accessors.TimedeltaProperties': [
                 # Seems like an upstream bug. The property is 'second'
                 'seconds_series.dt.seconds'
@@ -663,16 +723,47 @@ class DoctestTest(unittest.TestCase):
             'pandas.core.indexes.accessors.DatetimeProperties.to_pydatetime': [
                 '*'
             ],
+
+            # Beam doesn't have a deferred version of PeriodIndex.
+            # PeriodIndex instance is created in the test scenario.
+            'pandas.core.indexes.accessors.PeriodProperties.end_time': ['*'],
             'pandas.core.indexes.accessors.TimedeltaProperties.components': [
                 '*'
             ],
+            'pandas.core.indexes.accessors.TimedeltaProperties.days': ['*'],
+            'pandas.core.indexes.accessors.TimedeltaProperties.seconds': ['*'],
+            'pandas.core.indexes.accessors.TimedeltaProperties.microseconds': [
+                '*'
+            ],
+            'pandas.core.indexes.accessors.TimedeltaProperties.nanoseconds': [
+                '*'
+            ],
             
'pandas.core.indexes.accessors.TimedeltaProperties.to_pytimedelta': [
                 '*'
             ],
             # pylint: enable=line-too-long
+            # Test uses to_datetime. Beam calls to_datetime element-wise, and
+            # therefore the .tz attribute is not evaluated on entire Series.
+            # Hence, .tz becomes None, unless explicitly set.
+            # See: see test_tz_with_utc_zone_set_explicitly
+            'pandas.core.indexes.accessors.DatetimeProperties.tz': ['*'],
         })
     datetimelike_result = doctests.testmod(
-        pd.core.arrays.datetimelike, use_beam=False)
+        pd.core.arrays.datetimelike, use_beam=False,
+        not_implemented_ok={
+            # Beam Dataframes don't implement a deferred to_timedelta 
operation.
+            # Top-level issue: https://github.com/apache/beam/issues/20318
+            '*': [
+                "ser = pd.Series(pd.to_timedelta([1, 2, 3], unit='d'))",
+                "tdelta_idx = pd.to_timedelta([1, 2, 3], unit='D')",
+                'tdelta_idx = pd.to_timedelta(["0 days", "10 days", "20 
days"])',  # pylint: disable=line-too-long
+
+                "tdelta_idx",
+                "tdelta_idx.inferred_freq",
+                "tdelta_idx.mean()",
+            ],
+        })
+
 
     datetime_result = doctests.testmod(
         pd.core.arrays.datetimes,
@@ -685,10 +776,23 @@ class DoctestTest(unittest.TestCase):
             'pandas.core.arrays.datetimes.DatetimeArray.tz_localize': ['*'],
         },
         not_implemented_ok={
+            # Beam Dataframes don't implement a deferred to_timedelta 
operation.
+            # Top-level issue: https://github.com/apache/beam/issues/20318
+            '*': [
+                "ser = pd.Series(pd.to_timedelta([1, 2, 3], unit='d'))",
+                "tdelta_idx = pd.to_timedelta([1, 2, 3], unit='D')",
+                'tdelta_idx = pd.to_timedelta(["0 days", "10 days", "20 
days"])'],  # pylint: disable=line-too-long
             # Verifies index version of this method
             'pandas.core.arrays.datetimes.DatetimeArray.to_period': [
                 'df.index.to_period("M")'
             ],
+        },
+        skip={
+            # Test uses to_datetime. Beam calls to_datetime element-wise, and
+            # therefore the .tz attribute is not evaluated on entire Series.
+            # Hence, .tz becomes None, unless explicitly set.
+            # See: see test_tz_with_utc_zone_set_explicitly
+            'pandas.core.arrays.datetimes.DatetimeArray.tz': ['*'],
         })
 
     self.assertEqual(indexes_accessors_result.failed, 0)
@@ -718,14 +822,33 @@ class DoctestTest(unittest.TestCase):
         use_beam=False,
         verbose=True,
         wont_implement_ok={
+            '*': [
+                # resample is WontImpl.
+                "ser.resample('MS').count()",
+                "ser.resample('MS').median()",
+                "ser.resample('MS').sem()",
+                "ser.resample('MS').size()",
+            ],
+            'pandas.core.groupby.groupby.BaseGroupBy.indices': ['*'],
+            'pandas.core.groupby.groupby.GroupBy.bfill': ['*'],
+            'pandas.core.groupby.groupby.GroupBy.ffill': ['*'],
+            'pandas.core.groupby.groupby.GroupBy.diff': ['*'],
             'pandas.core.groupby.groupby.GroupBy.first': ['*'],
             'pandas.core.groupby.groupby.GroupBy.head': ['*'],
             'pandas.core.groupby.groupby.GroupBy.last': ['*'],
+            'pandas.core.groupby.groupby.GroupBy.ohlc': ['*'],
+            'pandas.core.groupby.groupby.GroupBy.pct_change': ['*'],
             'pandas.core.groupby.groupby.GroupBy.tail': ['*'],
             'pandas.core.groupby.groupby.GroupBy.nth': ['*'],
             'pandas.core.groupby.groupby.GroupBy.cumcount': ['*'],
+            'pandas.core.groupby.groupby.GroupBy.cummax': ['*'],
+            'pandas.core.groupby.groupby.GroupBy.cummin': ['*'],
+            'pandas.core.groupby.groupby.GroupBy.cumprod': ['*'],
+            'pandas.core.groupby.groupby.GroupBy.cumsum': ['*'],
+            'pandas.core.groupby.groupby.GroupBy.plot': ['*'],
             'pandas.core.groupby.groupby.GroupBy.resample': ['*'],
             'pandas.core.groupby.groupby.GroupBy.rolling': ['*'],
+            'pandas.core.groupby.groupby.GroupBy.shift': ['*'],
         },
         not_implemented_ok={
             'pandas.core.groupby.groupby.GroupBy.first': ['*'],
@@ -738,6 +861,12 @@ class DoctestTest(unittest.TestCase):
             ],
         },
         skip={
+            # New test that didn't pass on Pandas 1.5.x.
+            'pandas.core.groupby.groupby.BaseGroupBy.__iter__': ['*'],
+            # Not implemented; some tests also use resample (won't implement)
+            'pandas.core.groupby.groupby.BaseGroupBy.get_group': ['*'],
+            'pandas.core.groupby.groupby.BaseGroupBy.groups': ['*'],
+            # uses resample, which is WontImplement atm.
             # Uses iloc to mutate a DataFrame
             'pandas.core.groupby.groupby.GroupBy.resample': [
                 'df.iloc[2, 0] = 5',
@@ -757,9 +886,10 @@ class DoctestTest(unittest.TestCase):
         pd.core.groupby.generic,
         use_beam=False,
         wont_implement_ok={
-            # Returns an array by default, not a Series. WontImplement
-            # (non-deferred)
-            'pandas.core.groupby.generic.SeriesGroupBy.unique': ['*'],
+            '*' : [
+                # resample is WontImpl.
+                "ser.resample('MS').nunique()",
+            ],
             # TODO: Is take actually deprecated?
             'pandas.core.groupby.generic.DataFrameGroupBy.take': ['*'],
             'pandas.core.groupby.generic.SeriesGroupBy.take': ['*'],
@@ -776,6 +906,9 @@ class DoctestTest(unittest.TestCase):
             'pandas.core.groupby.generic.DataFrameGroupBy.diff': ['*'],
             'pandas.core.groupby.generic.SeriesGroupBy.diff': ['*'],
             'pandas.core.groupby.generic.DataFrameGroupBy.hist': ['*'],
+            'pandas.core.groupby.generic.SeriesGroupBy.hist': ['*'],
+            'pandas.core.groupby.generic.DataFrameGroupBy.plot': ['*'],
+            'pandas.core.groupby.generic.SeriesGroupBy.plot': ['*'],
             'pandas.core.groupby.generic.DataFrameGroupBy.fillna': [
                 'df.fillna(method=\'ffill\')',
                 'df.fillna(method="ffill")',
@@ -818,6 +951,9 @@ class DoctestTest(unittest.TestCase):
             # Skipped idxmax/idxmin due an issue with the test framework
             'pandas.core.groupby.generic.SeriesGroupBy.idxmin': ['s.idxmin()'],
             'pandas.core.groupby.generic.SeriesGroupBy.idxmax': ['s.idxmax()'],
+            # Order-sensitive operations. TODO: Return a better error message.
+            
'pandas.core.groupby.generic.SeriesGroupBy.is_monotonic_increasing': ['*'],  # 
pylint: disable=line-too-long
+            
'pandas.core.groupby.generic.SeriesGroupBy.is_monotonic_decreasing': ['*'],  # 
pylint: disable=line-too-long
             # Uses as_index, which is currently not_implemented
             'pandas.core.groupby.generic.DataFrameGroupBy.value_counts': [
                 "df.groupby('gender', as_index=False).value_counts()",
@@ -840,6 +976,9 @@ class DoctestTest(unittest.TestCase):
             'pandas.core.groupby.generic.DataFrameGroupBy.transform': ['*'],
             # These examples rely on grouping by a list
             'pandas.core.groupby.generic.SeriesGroupBy.transform': ['*'],
+            # Returns an array by default, not a Series. WontImplement
+            # (non-deferred)
+            'pandas.core.groupby.generic.SeriesGroupBy.unique': ['*'],
         },
     )
     self.assertEqual(result.failed, 0)
@@ -884,6 +1023,7 @@ class DoctestTest(unittest.TestCase):
             'pivot': ['*'],
             'to_datetime': ['s.head()'],
             'to_pickle': ['*'],
+            'unique': ['pd.unique(pd.Series([("a", "b"), ("b", "a"), ("a", 
"c"), ("b", "a")]).values)'],  # pylint: disable=line-too-long
             'melt': [
                 "pd.melt(df, id_vars=['A'], value_vars=['B'])",
                 "pd.melt(df, id_vars=['A'], value_vars=['B', 'C'])",
diff --git a/sdks/python/setup.py b/sdks/python/setup.py
index 13799dca942..ec3933f46c6 100644
--- a/sdks/python/setup.py
+++ b/sdks/python/setup.py
@@ -167,7 +167,7 @@ else:
 # Exclude 1.5.0 and 1.5.1 because of
 # https://github.com/pandas-dev/pandas/issues/45725
 dataframe_dependency = [
-    'pandas>=1.4.3,!=1.5.0,!=1.5.1,<2.1;python_version>="3.8"',
+    'pandas>=1.4.3,!=1.5.0,!=1.5.1,<2.2;python_version>="3.8"',
 ]
 
 
@@ -411,14 +411,14 @@ if __name__ == '__main__':
               # Pinning docutils as a workaround for Sphinx issue:
               # https://github.com/sphinx-doc/sphinx/issues/9727
               'docutils==0.17.1',
-              'pandas<2.1.0',
+              'pandas<2.2.0',
           ],
           'test': [
               'docstring-parser>=0.15,<1.0',
               'freezegun>=0.3.12',
               'joblib>=1.0.1',
               'mock>=1.0.1,<6.0.0',
-              'pandas<2.1.0',
+              'pandas<2.2.0',
               'parameterized>=0.7.1,<0.10.0',
               'pyhamcrest>=1.9,!=1.10.0,<3.0.0',
               'pyyaml>=3.12,<7.0.0',

Reply via email to