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/03/08 19:25:53 UTC

[GitHub] [beam] yeandy opened a new pull request #17043: [BEAM-13966] Add pivot(), a non-deferred column operation on categorical columns

yeandy opened a new pull request #17043:
URL: https://github.com/apache/beam/pull/17043


   Adding `pivot`, a non-deferred operation. i.e. operations that produce a variable set of columns based on the intrinsic nature of the data.
   
   ------------------------
   
   Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:
   
    - [ ] [**Choose reviewer(s)**](https://beam.apache.org/contribute/#make-your-change) and mention them in a comment (`R: @username`).
    - [ ] Format the pull request title like `[BEAM-XXX] Fixes bug in ApproximateQuantiles`, where you replace `BEAM-XXX` with the appropriate JIRA issue, if applicable. This will automatically link the pull request to the issue.
    - [ ] Update `CHANGES.md` with noteworthy changes.
    - [ ] If this contribution is large, please file an Apache [Individual Contributor License Agreement](https://www.apache.org/licenses/icla.pdf).
   
   See the [Contributor Guide](https://beam.apache.org/contribute) for more tips on [how to make review process smoother](https://beam.apache.org/contribute/#make-reviewers-job-easier).
   
   To check the build health, please visit [https://github.com/apache/beam/blob/master/.test-infra/BUILD_STATUS.md](https://github.com/apache/beam/blob/master/.test-infra/BUILD_STATUS.md)
   
   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)
   
   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 change in pull request #17043: [BEAM-13966] Add pivot(), a non-deferred column operation on categorical columns

Posted by GitBox <gi...@apache.org>.
yeandy commented on a change in pull request #17043:
URL: https://github.com/apache/beam/pull/17043#discussion_r837878826



##########
File path: sdks/python/apache_beam/dataframe/frames_test.py
##########
@@ -1295,6 +1295,114 @@ def s_times_shuffled(times, s):
     self._run_test(lambda s: s.pipe(s_times, 2), s)
     self._run_test(lambda s: s.pipe((s_times_shuffled, 's'), 2), s)
 
+  def test_pivot_non_categorical(self):
+    df = pd.DataFrame({
+        'foo': ['one', 'one', 'one', 'two', 'two', 'two'],
+        'bar': ['A', 'B', 'C', 'A', 'B', 'C'],
+        'baz': [1, 2, 3, 4, 5, 6],
+        'zoo': ['x', 'y', 'z', 'q', 'w', 't']
+    })
+    with self.assertRaisesRegex(
+        frame_base.WontImplementError,
+        r"pivot\(\) of non-categorical type is not supported"):
+      self._run_test(
+          lambda df: df.pivot(index='foo', columns='bar', values='baz'), df)
+
+  def test_pivot_pandas_example1(self):
+    # Simple test 1
+    df = pd.DataFrame({
+        'foo': ['one', 'one', 'one', 'two', 'two', 'two'],
+        'bar': ['A', 'B', 'C', 'A', 'B', 'C'],
+        'baz': [1, 2, 3, 4, 5, 6],
+        'zoo': ['x', 'y', 'z', 'q', 'w', 't']
+    })
+    df['bar'] = df['bar'].astype(
+        pd.CategoricalDtype(categories=['A', 'B', 'C']))
+    self._run_test(
+        lambda df: df.pivot(index='foo', columns='bar', values='baz'), df)
+
+  def test_pivot_pandas_example3(self):
+    # Multiple values
+    df = pd.DataFrame({
+        'foo': ['one', 'one', 'one', 'two', 'two', 'two'],
+        'bar': ['A', 'B', 'C', 'A', 'B', 'C'],
+        'baz': [1, 2, 3, 4, 5, 6],
+        'zoo': ['x', 'y', 'z', 'q', 'w', 't']
+    })
+    df['bar'] = df['bar'].astype(
+        pd.CategoricalDtype(categories=['A', 'B', 'C']))
+    self._run_test(
+        lambda df: df.pivot(index='foo', columns='bar', values=['baz', 'zoo']),
+        df)
+
+  def test_pivot_pandas_example4(self):
+    # Multiple columns
+    df = pd.DataFrame({
+        "lev1": [1, 1, 1, 2, 2, 2],
+        "lev2": [1, 1, 2, 1, 1, 2],
+        "lev3": [1, 2, 1, 2, 1, 2],
+        "lev4": [1, 2, 3, 4, 5, 6],
+        "values": [0, 1, 2, 3, 4, 5]
+    })
+    df['lev2'] = df['lev2'].astype(pd.CategoricalDtype(categories=[1, 2]))
+    df['lev3'] = df['lev3'].astype(pd.CategoricalDtype(categories=[1, 2]))
+    df['values'] = df['values'].astype('Int64')
+    self._run_test(
+        lambda df: df.pivot(
+            index="lev1", columns=["lev2", "lev3"], values="values"),
+        df)
+
+  @unittest.skipIf(
+      PD_VERSION < (1, 4), "Bug in DF.pivot with MultiIndex for pandas < 1.4")

Review comment:
       `NotImplementedError: initializing a Series from a MultiIndex is not supported`.
   Fixed [here](https://github.com/pandas-dev/pandas/pull/45141/files).




-- 
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 change in pull request #17043: [BEAM-13966] Add pivot(), a non-deferred column operation on categorical columns

Posted by GitBox <gi...@apache.org>.
TheNeuralBit commented on a change in pull request #17043:
URL: https://github.com/apache/beam/pull/17043#discussion_r835617914



##########
File path: sdks/python/apache_beam/dataframe/frames_test.py
##########
@@ -1295,6 +1295,114 @@ def s_times_shuffled(times, s):
     self._run_test(lambda s: s.pipe(s_times, 2), s)
     self._run_test(lambda s: s.pipe((s_times_shuffled, 's'), 2), s)
 
+  def test_pivot_non_categorical(self):
+    df = pd.DataFrame({
+        'foo': ['one', 'one', 'one', 'two', 'two', 'two'],
+        'bar': ['A', 'B', 'C', 'A', 'B', 'C'],
+        'baz': [1, 2, 3, 4, 5, 6],
+        'zoo': ['x', 'y', 'z', 'q', 'w', 't']
+    })
+    with self.assertRaisesRegex(
+        frame_base.WontImplementError,
+        r"pivot\(\) of non-categorical type is not supported"):
+      self._run_test(
+          lambda df: df.pivot(index='foo', columns='bar', values='baz'), df)
+
+  def test_pivot_pandas_example1(self):
+    # Simple test 1
+    df = pd.DataFrame({
+        'foo': ['one', 'one', 'one', 'two', 'two', 'two'],
+        'bar': ['A', 'B', 'C', 'A', 'B', 'C'],
+        'baz': [1, 2, 3, 4, 5, 6],
+        'zoo': ['x', 'y', 'z', 'q', 'w', 't']
+    })
+    df['bar'] = df['bar'].astype(
+        pd.CategoricalDtype(categories=['A', 'B', 'C']))
+    self._run_test(
+        lambda df: df.pivot(index='foo', columns='bar', values='baz'), df)
+
+  def test_pivot_pandas_example3(self):
+    # Multiple values
+    df = pd.DataFrame({
+        'foo': ['one', 'one', 'one', 'two', 'two', 'two'],
+        'bar': ['A', 'B', 'C', 'A', 'B', 'C'],
+        'baz': [1, 2, 3, 4, 5, 6],
+        'zoo': ['x', 'y', 'z', 'q', 'w', 't']
+    })
+    df['bar'] = df['bar'].astype(
+        pd.CategoricalDtype(categories=['A', 'B', 'C']))
+    self._run_test(
+        lambda df: df.pivot(index='foo', columns='bar', values=['baz', 'zoo']),
+        df)
+
+  def test_pivot_pandas_example4(self):
+    # Multiple columns
+    df = pd.DataFrame({
+        "lev1": [1, 1, 1, 2, 2, 2],
+        "lev2": [1, 1, 2, 1, 1, 2],
+        "lev3": [1, 2, 1, 2, 1, 2],
+        "lev4": [1, 2, 3, 4, 5, 6],
+        "values": [0, 1, 2, 3, 4, 5]
+    })
+    df['lev2'] = df['lev2'].astype(pd.CategoricalDtype(categories=[1, 2]))
+    df['lev3'] = df['lev3'].astype(pd.CategoricalDtype(categories=[1, 2]))
+    df['values'] = df['values'].astype('Int64')
+    self._run_test(
+        lambda df: df.pivot(
+            index="lev1", columns=["lev2", "lev3"], values="values"),
+        df)
+
+  @unittest.skipIf(
+      PD_VERSION < (1, 4), "Bug in DF.pivot with MultiIndex for pandas < 1.4")

Review comment:
       What is the bug?

##########
File path: sdks/python/apache_beam/dataframe/frames.py
##########
@@ -3655,6 +3656,121 @@ def shift(self, axis, freq, **kwargs):
   describe = _agg_method(pd.DataFrame, 'describe')
   max = _agg_method(pd.DataFrame, 'max')
   min = _agg_method(pd.DataFrame, 'min')
+
+  @frame_base.with_docs_from(pd.DataFrame)
+  @frame_base.args_to_kwargs(pd.DataFrame)
+  @frame_base.populate_defaults(pd.DataFrame)
+  def pivot(self, index=None, columns=None, values=None, **kwargs):
+    def verify_all_categorical(all_cols_are_categorical):
+      if not all_cols_are_categorical:
+        raise frame_base.WontImplementError(
+            "pivot() of non-categorical type is not supported because "
+            "the type of the output column depends on the data. Please use "
+            "pd.CategoricalDtype with explicit categories.",
+            reason="non-deferred-columns")

Review comment:
       nit: you might inline this method and instead break out the message as a string constant

##########
File path: sdks/python/apache_beam/dataframe/frames.py
##########
@@ -3655,6 +3656,121 @@ def shift(self, axis, freq, **kwargs):
   describe = _agg_method(pd.DataFrame, 'describe')
   max = _agg_method(pd.DataFrame, 'max')
   min = _agg_method(pd.DataFrame, 'min')
+
+  @frame_base.with_docs_from(pd.DataFrame)
+  @frame_base.args_to_kwargs(pd.DataFrame)
+  @frame_base.populate_defaults(pd.DataFrame)
+  def pivot(self, index=None, columns=None, values=None, **kwargs):
+    def verify_all_categorical(all_cols_are_categorical):
+      if not all_cols_are_categorical:
+        raise frame_base.WontImplementError(
+            "pivot() of non-categorical type is not supported because "
+            "the type of the output column depends on the data. Please use "
+            "pd.CategoricalDtype with explicit categories.",
+            reason="non-deferred-columns")
+
+    # Construct column index
+    if is_list_like(columns) and len(columns) <= 1:
+      columns = columns[0]
+    selected_cols = self._expr.proxy()[columns]
+    if isinstance(selected_cols, pd.Series):
+      all_cols_are_categorical = isinstance(
+        selected_cols.dtype, pd.CategoricalDtype
+      )
+      verify_all_categorical(all_cols_are_categorical)
+
+      # If values not provided, take all remaining columns of dataframe
+      if not values:
+        values = self._expr.proxy() \
+          .drop(index, axis=1).drop(columns, axis=1).columns.values
+
+      # Take the provided values
+      if is_list_like(values) and len(values) > 1:
+        values_in_col_index = values
+        names = [None, columns]
+        col_index = pd.MultiIndex.from_product(
+          [values_in_col_index,
+          selected_cols.dtypes.categories.astype('category')],
+          names=names
+        )
+      else:
+        col_index = pd.CategoricalIndex(
+          selected_cols.dtype.categories,
+          name=columns
+        )
+    else:
+      all_cols_are_categorical = all(
+        isinstance(c, pd.CategoricalDtype) for c in selected_cols.dtypes
+      )
+      verify_all_categorical(all_cols_are_categorical)
+
+      categories = [
+        c.categories.astype('category') for c in selected_cols.dtypes
+      ]
+      if is_list_like(columns) and len(columns) > 1:
+        col_index = pd.MultiIndex.from_product(categories, names=columns)
+      else:
+        col_index = pd.CategoricalIndex(
+            selected_cols.dtype.categories,
+            name=columns
+        )
+
+    # Construct row index
+    if index:
+      per_partition = expressions.ComputedExpression(
+          'pivot-per-partition',
+          lambda df: df.set_index(keys=index), [self._expr],
+          preserves_partition_by=partitionings.Singleton(),
+          requires_partition_by=partitionings.Arbitrary()
+      )
+      if is_list_like(index):

Review comment:
       What if index is a single-element list?

##########
File path: sdks/python/apache_beam/dataframe/frames_test.py
##########
@@ -1295,6 +1295,114 @@ def s_times_shuffled(times, s):
     self._run_test(lambda s: s.pipe(s_times, 2), s)
     self._run_test(lambda s: s.pipe((s_times_shuffled, 's'), 2), s)
 
+  def test_pivot_non_categorical(self):
+    df = pd.DataFrame({
+        'foo': ['one', 'one', 'one', 'two', 'two', 'two'],
+        'bar': ['A', 'B', 'C', 'A', 'B', 'C'],
+        'baz': [1, 2, 3, 4, 5, 6],
+        'zoo': ['x', 'y', 'z', 'q', 'w', 't']
+    })
+    with self.assertRaisesRegex(
+        frame_base.WontImplementError,
+        r"pivot\(\) of non-categorical type is not supported"):
+      self._run_test(
+          lambda df: df.pivot(index='foo', columns='bar', values='baz'), df)
+
+  def test_pivot_pandas_example1(self):
+    # Simple test 1
+    df = pd.DataFrame({
+        'foo': ['one', 'one', 'one', 'two', 'two', 'two'],
+        'bar': ['A', 'B', 'C', 'A', 'B', 'C'],
+        'baz': [1, 2, 3, 4, 5, 6],
+        'zoo': ['x', 'y', 'z', 'q', 'w', 't']
+    })
+    df['bar'] = df['bar'].astype(
+        pd.CategoricalDtype(categories=['A', 'B', 'C']))
+    self._run_test(
+        lambda df: df.pivot(index='foo', columns='bar', values='baz'), df)
+
+  def test_pivot_pandas_example3(self):
+    # Multiple values
+    df = pd.DataFrame({
+        'foo': ['one', 'one', 'one', 'two', 'two', 'two'],
+        'bar': ['A', 'B', 'C', 'A', 'B', 'C'],
+        'baz': [1, 2, 3, 4, 5, 6],
+        'zoo': ['x', 'y', 'z', 'q', 'w', 't']
+    })
+    df['bar'] = df['bar'].astype(
+        pd.CategoricalDtype(categories=['A', 'B', 'C']))
+    self._run_test(
+        lambda df: df.pivot(index='foo', columns='bar', values=['baz', 'zoo']),

Review comment:
       You might also test this with `columns=["bar"]`, sometimes pandas does different things in this case (e.g. df['bar'] yields a Series and `df[['bar']]` yields a DataFrame).

##########
File path: sdks/python/apache_beam/dataframe/frames.py
##########
@@ -3655,6 +3656,121 @@ def shift(self, axis, freq, **kwargs):
   describe = _agg_method(pd.DataFrame, 'describe')
   max = _agg_method(pd.DataFrame, 'max')
   min = _agg_method(pd.DataFrame, 'min')
+
+  @frame_base.with_docs_from(pd.DataFrame)
+  @frame_base.args_to_kwargs(pd.DataFrame)
+  @frame_base.populate_defaults(pd.DataFrame)
+  def pivot(self, index=None, columns=None, values=None, **kwargs):
+    def verify_all_categorical(all_cols_are_categorical):
+      if not all_cols_are_categorical:
+        raise frame_base.WontImplementError(
+            "pivot() of non-categorical type is not supported because "
+            "the type of the output column depends on the data. Please use "
+            "pd.CategoricalDtype with explicit categories.",
+            reason="non-deferred-columns")
+
+    # Construct column index
+    if is_list_like(columns) and len(columns) <= 1:
+      columns = columns[0]
+    selected_cols = self._expr.proxy()[columns]
+    if isinstance(selected_cols, pd.Series):
+      all_cols_are_categorical = isinstance(
+        selected_cols.dtype, pd.CategoricalDtype
+      )
+      verify_all_categorical(all_cols_are_categorical)
+
+      # If values not provided, take all remaining columns of dataframe
+      if not values:
+        values = self._expr.proxy() \
+          .drop(index, axis=1).drop(columns, axis=1).columns.values
+
+      # Take the provided values
+      if is_list_like(values) and len(values) > 1:
+        values_in_col_index = values
+        names = [None, columns]
+        col_index = pd.MultiIndex.from_product(
+          [values_in_col_index,
+          selected_cols.dtypes.categories.astype('category')],
+          names=names
+        )
+      else:
+        col_index = pd.CategoricalIndex(
+          selected_cols.dtype.categories,
+          name=columns
+        )
+    else:
+      all_cols_are_categorical = all(
+        isinstance(c, pd.CategoricalDtype) for c in selected_cols.dtypes
+      )
+      verify_all_categorical(all_cols_are_categorical)
+
+      categories = [
+        c.categories.astype('category') for c in selected_cols.dtypes
+      ]
+      if is_list_like(columns) and len(columns) > 1:

Review comment:
       I think this check is a no-op, don't we know it will be true because we're in the else? (i.e. `isinstance(selected_cols, pd.Series)` is false).




-- 
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 change in pull request #17043: [BEAM-13966] Add pivot(), a non-deferred column operation on categorical columns

Posted by GitBox <gi...@apache.org>.
TheNeuralBit commented on a change in pull request #17043:
URL: https://github.com/apache/beam/pull/17043#discussion_r838926287



##########
File path: sdks/python/apache_beam/dataframe/pandas_doctests_test.py
##########
@@ -876,7 +869,7 @@ def test_top_level(self):
                 'merge_ordered(df1, df2, fill_method="ffill", left_by="group")'
             ],
             # Expected error.
-            'pivot': ["df.pivot(index='foo', columns='bar', values='baz')"],
+            'pivot': ["*"],

Review comment:
       I don't think we want to skip all the pivot tests, can we revert this line and instead add `'pivot': ["*"]` to wont_implement_ok (as suggested above).

##########
File path: sdks/python/apache_beam/dataframe/pandas_doctests_test.py
##########
@@ -366,9 +362,7 @@ def test_dataframe_tests(self):
             # actually raise NotImplementedError
             'pandas.core.frame.DataFrame.pivot_table': ['*'],
             # Expected to raise a ValueError, but we raise NotImplementedError
-            'pandas.core.frame.DataFrame.pivot': [
-                "df.pivot(index='foo', columns='bar', values='baz')"
-            ],

Review comment:
       Similarly here

##########
File path: sdks/python/apache_beam/dataframe/pandas_doctests_test.py
##########
@@ -832,7 +826,6 @@ def test_top_level(self):
             'melt': ['*'],
             'merge': ["df1.merge(df2, how='cross')"],
             'merge_asof': ['*'],
-            'pivot': ['*'],

Review comment:
       I think this should be wont_implement_ok instead, right?




-- 
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] edited a comment on pull request #17043: [BEAM-13966] Add pivot(), a non-deferred column operation on categorical columns

Posted by GitBox <gi...@apache.org>.
codecov[bot] edited a comment on pull request #17043:
URL: https://github.com/apache/beam/pull/17043#issuecomment-1062140334


   # [Codecov](https://codecov.io/gh/apache/beam/pull/17043?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 [#17043](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (ba3f2b6) into [master](https://codecov.io/gh/apache/beam/commit/58f8bef66f4da09202b2de9dc3166c7fa1f4fa4e?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (58f8bef) will **increase** coverage by `0.07%`.
   > The diff coverage is `98.14%`.
   
   ```diff
   @@            Coverage Diff             @@
   ##           master   #17043      +/-   ##
   ==========================================
   + Coverage   73.83%   73.90%   +0.07%     
   ==========================================
     Files         667      667              
     Lines       87392    88071     +679     
   ==========================================
   + Hits        64523    65089     +566     
   - Misses      21766    21879     +113     
     Partials     1103     1103              
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | python | `83.64% <98.14%> (-0.01%)` | :arrow_down: |
   
   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/17043?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [sdks/python/apache\_beam/dataframe/frames.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vZGF0YWZyYW1lL2ZyYW1lcy5weQ==) | `95.21% <98.14%> (+0.07%)` | :arrow_up: |
   | [...nternal/clients/cloudbuild/cloudbuild\_v1\_client.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9kYXRhZmxvdy9pbnRlcm5hbC9jbGllbnRzL2Nsb3VkYnVpbGQvY2xvdWRidWlsZF92MV9jbGllbnQucHk=) | `52.21% <0.00%> (-2.97%)` | :arrow_down: |
   | [sdks/python/apache\_beam/utils/interactive\_utils.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vdXRpbHMvaW50ZXJhY3RpdmVfdXRpbHMucHk=) | `92.68% <0.00%> (-2.44%)` | :arrow_down: |
   | [.../python/apache\_beam/transforms/periodicsequence.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vdHJhbnNmb3Jtcy9wZXJpb2RpY3NlcXVlbmNlLnB5) | `96.72% <0.00%> (-1.64%)` | :arrow_down: |
   | [...thon/apache\_beam/runners/worker/sdk\_worker\_main.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy93b3JrZXIvc2RrX3dvcmtlcl9tYWluLnB5) | `74.12% <0.00%> (-1.40%)` | :arrow_down: |
   | [...\_beam/runners/portability/sdk\_container\_builder.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9wb3J0YWJpbGl0eS9zZGtfY29udGFpbmVyX2J1aWxkZXIucHk=) | `37.93% <0.00%> (-1.02%)` | :arrow_down: |
   | [...pache\_beam/runners/interactive/interactive\_beam.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9pbnRlcmFjdGl2ZS9pbnRlcmFjdGl2ZV9iZWFtLnB5) | `80.62% <0.00%> (-0.78%)` | :arrow_down: |
   | [...che\_beam/runners/interactive/interactive\_runner.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9pbnRlcmFjdGl2ZS9pbnRlcmFjdGl2ZV9ydW5uZXIucHk=) | `92.02% <0.00%> (-0.73%)` | :arrow_down: |
   | [sdks/python/apache\_beam/runners/direct/executor.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9kaXJlY3QvZXhlY3V0b3IucHk=) | `96.46% <0.00%> (-0.55%)` | :arrow_down: |
   | [...runners/interactive/display/pcoll\_visualization.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9pbnRlcmFjdGl2ZS9kaXNwbGF5L3Bjb2xsX3Zpc3VhbGl6YXRpb24ucHk=) | `85.85% <0.00%> (-0.51%)` | :arrow_down: |
   | ... and [15 more](https://codecov.io/gh/apache/beam/pull/17043/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [58f8bef...ba3f2b6](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] yeandy commented on a change in pull request #17043: [BEAM-13966] Add pivot(), a non-deferred column operation on categorical columns

Posted by GitBox <gi...@apache.org>.
yeandy commented on a change in pull request #17043:
URL: https://github.com/apache/beam/pull/17043#discussion_r831083595



##########
File path: sdks/python/apache_beam/dataframe/frames_test.py
##########
@@ -2209,6 +2209,112 @@ def test_nlargest_any(self):
     # but not necessarily with the same index
     self.assert_frame_data_equivalent(result, df.population.nlargest(3))
 
+  def test_pivot_non_categorical(self):
+    df = pd.DataFrame({
+        'foo': ['one', 'one', 'one', 'two', 'two', 'two'],
+        'bar': ['A', 'B', 'C', 'A', 'B', 'C'],
+        'baz': [1, 2, 3, 4, 5, 6],
+        'zoo': ['x', 'y', 'z', 'q', 'w', 't']
+    })
+    with self.assertRaisesRegex(
+        frame_base.WontImplementError,
+        r"pivot\(\) of non-categorical type is not supported"):
+      self._evaluate(
+          lambda df: df.pivot(index='foo', columns='bar', values='baz'), df)
+
+  def test_pivot_pandas_example1(self):
+    # Simple test 1
+    df = pd.DataFrame({
+        'foo': ['one', 'one', 'one', 'two', 'two', 'two'],
+        'bar': ['A', 'B', 'C', 'A', 'B', 'C'],
+        'baz': [1, 2, 3, 4, 5, 6],
+        'zoo': ['x', 'y', 'z', 'q', 'w', 't']
+    })
+    df['bar'] = df['bar'].astype(
+        pd.CategoricalDtype(categories=['A', 'B', 'C']))
+    result = self._evaluate(
+        lambda df: df.pivot(index='foo', columns='bar', values='baz'), df)
+    self.assert_frame_data_equivalent(
+        result, df.pivot(index='foo', columns='bar', values='baz'))
+
+  def test_pivot_pandas_example2(self):
+    # Simple test 2
+    df = pd.DataFrame({
+        'foo': ['one', 'one', 'one', 'two', 'two', 'two'],
+        'bar': ['A', 'B', 'C', 'A', 'B', 'C'],
+        'baz': [1, 2, 3, 4, 5, 6],
+        'zoo': ['x', 'y', 'z', 'q', 'w', 't']
+    })
+    df['bar'] = df['bar'].astype(
+        pd.CategoricalDtype(categories=['A', 'B', 'C']))
+    result = self._evaluate(lambda df: df.pivot(index='foo', columns='bar'), df)
+    self.assert_frame_data_equivalent(
+        result['baz'], df.pivot(index='foo', columns='bar')['baz'])

Review comment:
       This test just selects the `baz` column, and the rest of `result` are non-zero, non-nans found in the `zoo` column. So there's no good verification to test. (unless we want to check that they are non-zero, non-nans?)

##########
File path: sdks/python/apache_beam/dataframe/frames_test.py
##########
@@ -2209,6 +2209,112 @@ def test_nlargest_any(self):
     # but not necessarily with the same index
     self.assert_frame_data_equivalent(result, df.population.nlargest(3))
 
+  def test_pivot_non_categorical(self):
+    df = pd.DataFrame({
+        'foo': ['one', 'one', 'one', 'two', 'two', 'two'],
+        'bar': ['A', 'B', 'C', 'A', 'B', 'C'],
+        'baz': [1, 2, 3, 4, 5, 6],
+        'zoo': ['x', 'y', 'z', 'q', 'w', 't']
+    })
+    with self.assertRaisesRegex(
+        frame_base.WontImplementError,
+        r"pivot\(\) of non-categorical type is not supported"):
+      self._evaluate(
+          lambda df: df.pivot(index='foo', columns='bar', values='baz'), df)
+
+  def test_pivot_pandas_example1(self):
+    # Simple test 1
+    df = pd.DataFrame({
+        'foo': ['one', 'one', 'one', 'two', 'two', 'two'],
+        'bar': ['A', 'B', 'C', 'A', 'B', 'C'],
+        'baz': [1, 2, 3, 4, 5, 6],
+        'zoo': ['x', 'y', 'z', 'q', 'w', 't']
+    })
+    df['bar'] = df['bar'].astype(
+        pd.CategoricalDtype(categories=['A', 'B', 'C']))
+    result = self._evaluate(
+        lambda df: df.pivot(index='foo', columns='bar', values='baz'), df)
+    self.assert_frame_data_equivalent(
+        result, df.pivot(index='foo', columns='bar', values='baz'))

Review comment:
       SG, moved!




-- 
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] edited a comment on pull request #17043: [BEAM-13966] Add pivot(), a non-deferred column operation on categorical columns

Posted by GitBox <gi...@apache.org>.
codecov[bot] edited a comment on pull request #17043:
URL: https://github.com/apache/beam/pull/17043#issuecomment-1062140334


   # [Codecov](https://codecov.io/gh/apache/beam/pull/17043?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 [#17043](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (0e046fc) into [master](https://codecov.io/gh/apache/beam/commit/58f8bef66f4da09202b2de9dc3166c7fa1f4fa4e?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (58f8bef) will **increase** coverage by `0.06%`.
   > The diff coverage is `97.95%`.
   
   ```diff
   @@            Coverage Diff             @@
   ##           master   #17043      +/-   ##
   ==========================================
   + Coverage   73.83%   73.89%   +0.06%     
   ==========================================
     Files         667      667              
     Lines       87392    88094     +702     
   ==========================================
   + Hits        64523    65101     +578     
   - Misses      21766    21890     +124     
     Partials     1103     1103              
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | python | `83.63% <97.95%> (-0.02%)` | :arrow_down: |
   
   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/17043?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [sdks/python/apache\_beam/dataframe/frames.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vZGF0YWZyYW1lL2ZyYW1lcy5weQ==) | `95.20% <97.95%> (+0.06%)` | :arrow_up: |
   | [.../python/apache\_beam/testing/test\_stream\_service.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vdGVzdGluZy90ZXN0X3N0cmVhbV9zZXJ2aWNlLnB5) | `88.37% <0.00%> (-4.66%)` | :arrow_down: |
   | [...nternal/clients/cloudbuild/cloudbuild\_v1\_client.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9kYXRhZmxvdy9pbnRlcm5hbC9jbGllbnRzL2Nsb3VkYnVpbGQvY2xvdWRidWlsZF92MV9jbGllbnQucHk=) | `52.21% <0.00%> (-2.97%)` | :arrow_down: |
   | [sdks/python/apache\_beam/utils/interactive\_utils.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vdXRpbHMvaW50ZXJhY3RpdmVfdXRpbHMucHk=) | `92.68% <0.00%> (-2.44%)` | :arrow_down: |
   | [...thon/apache\_beam/runners/worker/sdk\_worker\_main.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy93b3JrZXIvc2RrX3dvcmtlcl9tYWluLnB5) | `74.12% <0.00%> (-1.40%)` | :arrow_down: |
   | [...che\_beam/runners/interactive/interactive\_runner.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9pbnRlcmFjdGl2ZS9pbnRlcmFjdGl2ZV9ydW5uZXIucHk=) | `91.42% <0.00%> (-1.33%)` | :arrow_down: |
   | [...\_beam/runners/portability/sdk\_container\_builder.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9wb3J0YWJpbGl0eS9zZGtfY29udGFpbmVyX2J1aWxkZXIucHk=) | `37.93% <0.00%> (-1.02%)` | :arrow_down: |
   | [...n/apache\_beam/runners/interactive/cache\_manager.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9pbnRlcmFjdGl2ZS9jYWNoZV9tYW5hZ2VyLnB5) | `87.58% <0.00%> (-1.01%)` | :arrow_down: |
   | [...pache\_beam/runners/interactive/interactive\_beam.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9pbnRlcmFjdGl2ZS9pbnRlcmFjdGl2ZV9iZWFtLnB5) | `80.62% <0.00%> (-0.78%)` | :arrow_down: |
   | [sdks/python/apache\_beam/runners/direct/executor.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9kaXJlY3QvZXhlY3V0b3IucHk=) | `96.46% <0.00%> (-0.55%)` | :arrow_down: |
   | ... and [28 more](https://codecov.io/gh/apache/beam/pull/17043/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [58f8bef...0e046fc](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] codecov[bot] edited a comment on pull request #17043: [BEAM-13966] Add pivot(), a non-deferred column operation on categorical columns

Posted by GitBox <gi...@apache.org>.
codecov[bot] edited a comment on pull request #17043:
URL: https://github.com/apache/beam/pull/17043#issuecomment-1062140334


   # [Codecov](https://codecov.io/gh/apache/beam/pull/17043?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 [#17043](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (de90f88) into [master](https://codecov.io/gh/apache/beam/commit/58f8bef66f4da09202b2de9dc3166c7fa1f4fa4e?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (58f8bef) will **decrease** coverage by `0.01%`.
   > The diff coverage is `100.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/beam/pull/17043/graphs/tree.svg?width=650&height=150&src=pr&token=qcbbAh8Fj1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master   #17043      +/-   ##
   ==========================================
   - Coverage   73.83%   73.81%   -0.02%     
   ==========================================
     Files         667      667              
     Lines       87392    87320      -72     
   ==========================================
   - Hits        64523    64456      -67     
   + Misses      21766    21761       -5     
     Partials     1103     1103              
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | python | `83.63% <100.00%> (-0.02%)` | :arrow_down: |
   
   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/17043?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [sdks/python/apache\_beam/dataframe/frames.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vZGF0YWZyYW1lL2ZyYW1lcy5weQ==) | `95.17% <100.00%> (+0.03%)` | :arrow_up: |
   | [sdks/python/apache\_beam/internal/metrics/metric.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vaW50ZXJuYWwvbWV0cmljcy9tZXRyaWMucHk=) | `90.00% <0.00%> (-1.00%)` | :arrow_down: |
   | [...pache\_beam/runners/interactive/interactive\_beam.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9pbnRlcmFjdGl2ZS9pbnRlcmFjdGl2ZV9iZWFtLnB5) | `80.62% <0.00%> (-0.78%)` | :arrow_down: |
   | [...che\_beam/runners/interactive/interactive\_runner.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9pbnRlcmFjdGl2ZS9pbnRlcmFjdGl2ZV9ydW5uZXIucHk=) | `92.02% <0.00%> (-0.73%)` | :arrow_down: |
   | [sdks/python/apache\_beam/runners/direct/executor.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9kaXJlY3QvZXhlY3V0b3IucHk=) | `96.46% <0.00%> (-0.55%)` | :arrow_down: |
   | [...runners/interactive/display/pcoll\_visualization.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9pbnRlcmFjdGl2ZS9kaXNwbGF5L3Bjb2xsX3Zpc3VhbGl6YXRpb24ucHk=) | `85.85% <0.00%> (-0.51%)` | :arrow_down: |
   | [...dks/python/apache\_beam/options/pipeline\_options.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vb3B0aW9ucy9waXBlbGluZV9vcHRpb25zLnB5) | `94.85% <0.00%> (-0.43%)` | :arrow_down: |
   | [...eam/runners/interactive/interactive\_environment.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9pbnRlcmFjdGl2ZS9pbnRlcmFjdGl2ZV9lbnZpcm9ubWVudC5weQ==) | `90.44% <0.00%> (-0.30%)` | :arrow_down: |
   | [...ks/python/apache\_beam/runners/worker/sdk\_worker.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy93b3JrZXIvc2RrX3dvcmtlci5weQ==) | `88.90% <0.00%> (-0.16%)` | :arrow_down: |
   | [...nternal/clients/dataflow/dataflow\_v1b3\_messages.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9kYXRhZmxvdy9pbnRlcm5hbC9jbGllbnRzL2RhdGFmbG93L2RhdGFmbG93X3YxYjNfbWVzc2FnZXMucHk=) | `100.00% <0.00%> (ø)` | |
   | ... and [3 more](https://codecov.io/gh/apache/beam/pull/17043/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [58f8bef...de90f88](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] yeandy commented on a change in pull request #17043: [BEAM-13966] Add pivot(), a non-deferred column operation on categorical columns

Posted by GitBox <gi...@apache.org>.
yeandy commented on a change in pull request #17043:
URL: https://github.com/apache/beam/pull/17043#discussion_r837878548



##########
File path: sdks/python/apache_beam/dataframe/frames_test.py
##########
@@ -1295,6 +1295,114 @@ def s_times_shuffled(times, s):
     self._run_test(lambda s: s.pipe(s_times, 2), s)
     self._run_test(lambda s: s.pipe((s_times_shuffled, 's'), 2), s)
 
+  def test_pivot_non_categorical(self):
+    df = pd.DataFrame({
+        'foo': ['one', 'one', 'one', 'two', 'two', 'two'],
+        'bar': ['A', 'B', 'C', 'A', 'B', 'C'],
+        'baz': [1, 2, 3, 4, 5, 6],
+        'zoo': ['x', 'y', 'z', 'q', 'w', 't']
+    })
+    with self.assertRaisesRegex(
+        frame_base.WontImplementError,
+        r"pivot\(\) of non-categorical type is not supported"):
+      self._run_test(
+          lambda df: df.pivot(index='foo', columns='bar', values='baz'), df)
+
+  def test_pivot_pandas_example1(self):
+    # Simple test 1
+    df = pd.DataFrame({
+        'foo': ['one', 'one', 'one', 'two', 'two', 'two'],
+        'bar': ['A', 'B', 'C', 'A', 'B', 'C'],
+        'baz': [1, 2, 3, 4, 5, 6],
+        'zoo': ['x', 'y', 'z', 'q', 'w', 't']
+    })
+    df['bar'] = df['bar'].astype(
+        pd.CategoricalDtype(categories=['A', 'B', 'C']))
+    self._run_test(
+        lambda df: df.pivot(index='foo', columns='bar', values='baz'), df)
+
+  def test_pivot_pandas_example3(self):
+    # Multiple values
+    df = pd.DataFrame({
+        'foo': ['one', 'one', 'one', 'two', 'two', 'two'],
+        'bar': ['A', 'B', 'C', 'A', 'B', 'C'],
+        'baz': [1, 2, 3, 4, 5, 6],
+        'zoo': ['x', 'y', 'z', 'q', 'w', 't']
+    })
+    df['bar'] = df['bar'].astype(
+        pd.CategoricalDtype(categories=['A', 'B', 'C']))
+    self._run_test(
+        lambda df: df.pivot(index='foo', columns='bar', values=['baz', 'zoo']),

Review comment:
       Added!




-- 
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] edited a comment on pull request #17043: [BEAM-13966] Add pivot(), a non-deferred column operation on categorical columns

Posted by GitBox <gi...@apache.org>.
codecov[bot] edited a comment on pull request #17043:
URL: https://github.com/apache/beam/pull/17043#issuecomment-1062140334


   # [Codecov](https://codecov.io/gh/apache/beam/pull/17043?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 [#17043](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (0e046fc) into [master](https://codecov.io/gh/apache/beam/commit/58f8bef66f4da09202b2de9dc3166c7fa1f4fa4e?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (58f8bef) will **increase** coverage by `0.18%`.
   > The diff coverage is `97.95%`.
   
   ```diff
   @@            Coverage Diff             @@
   ##           master   #17043      +/-   ##
   ==========================================
   + Coverage   73.83%   74.01%   +0.18%     
   ==========================================
     Files         667      667              
     Lines       87392    88667    +1275     
   ==========================================
   + Hits        64523    65626    +1103     
   - Misses      21766    21938     +172     
     Partials     1103     1103              
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | python | `83.70% <97.95%> (+0.05%)` | :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/17043?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [sdks/python/apache\_beam/dataframe/frames.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vZGF0YWZyYW1lL2ZyYW1lcy5weQ==) | `95.20% <97.95%> (+0.06%)` | :arrow_up: |
   | [...nternal/clients/cloudbuild/cloudbuild\_v1\_client.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9kYXRhZmxvdy9pbnRlcm5hbC9jbGllbnRzL2Nsb3VkYnVpbGQvY2xvdWRidWlsZF92MV9jbGllbnQucHk=) | `52.21% <0.00%> (-2.97%)` | :arrow_down: |
   | [sdks/python/apache\_beam/utils/interactive\_utils.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vdXRpbHMvaW50ZXJhY3RpdmVfdXRpbHMucHk=) | `92.68% <0.00%> (-2.44%)` | :arrow_down: |
   | [...thon/apache\_beam/runners/worker/sdk\_worker\_main.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy93b3JrZXIvc2RrX3dvcmtlcl9tYWluLnB5) | `74.12% <0.00%> (-1.40%)` | :arrow_down: |
   | [...\_beam/runners/portability/sdk\_container\_builder.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9wb3J0YWJpbGl0eS9zZGtfY29udGFpbmVyX2J1aWxkZXIucHk=) | `37.93% <0.00%> (-1.02%)` | :arrow_down: |
   | [...n/apache\_beam/runners/interactive/cache\_manager.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9pbnRlcmFjdGl2ZS9jYWNoZV9tYW5hZ2VyLnB5) | `87.58% <0.00%> (-1.01%)` | :arrow_down: |
   | [...pache\_beam/runners/interactive/interactive\_beam.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9pbnRlcmFjdGl2ZS9pbnRlcmFjdGl2ZV9iZWFtLnB5) | `80.62% <0.00%> (-0.78%)` | :arrow_down: |
   | [sdks/python/apache\_beam/runners/direct/executor.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9kaXJlY3QvZXhlY3V0b3IucHk=) | `96.46% <0.00%> (-0.55%)` | :arrow_down: |
   | [...runners/interactive/display/pcoll\_visualization.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9pbnRlcmFjdGl2ZS9kaXNwbGF5L3Bjb2xsX3Zpc3VhbGl6YXRpb24ucHk=) | `85.85% <0.00%> (-0.51%)` | :arrow_down: |
   | [sdks/python/apache\_beam/internal/gcp/auth.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vaW50ZXJuYWwvZ2NwL2F1dGgucHk=) | `81.03% <0.00%> (-0.45%)` | :arrow_down: |
   | ... and [28 more](https://codecov.io/gh/apache/beam/pull/17043/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [58f8bef...0e046fc](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] yeandy commented on pull request #17043: [BEAM-13966] Add pivot(), a non-deferred column operation on categorical columns

Posted by GitBox <gi...@apache.org>.
yeandy commented on pull request #17043:
URL: https://github.com/apache/beam/pull/17043#issuecomment-1078197344


   R: @TheNeuralBit Moved tests to `DeferredFrameTest`. PTAL.


-- 
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] edited a comment on pull request #17043: [BEAM-13966] Add pivot(), a non-deferred column operation on categorical columns

Posted by GitBox <gi...@apache.org>.
codecov[bot] edited a comment on pull request #17043:
URL: https://github.com/apache/beam/pull/17043#issuecomment-1062140334


   # [Codecov](https://codecov.io/gh/apache/beam/pull/17043?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 [#17043](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (de90f88) into [master](https://codecov.io/gh/apache/beam/commit/58f8bef66f4da09202b2de9dc3166c7fa1f4fa4e?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (58f8bef) will **decrease** coverage by `0.01%`.
   > The diff coverage is `100.00%`.
   
   > :exclamation: Current head de90f88 differs from pull request most recent head 5c6fd0a. Consider uploading reports for the commit 5c6fd0a to get more accurate results
   
   ```diff
   @@            Coverage Diff             @@
   ##           master   #17043      +/-   ##
   ==========================================
   - Coverage   73.83%   73.81%   -0.02%     
   ==========================================
     Files         667      667              
     Lines       87392    87320      -72     
   ==========================================
   - Hits        64523    64456      -67     
   + Misses      21766    21761       -5     
     Partials     1103     1103              
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | python | `83.63% <100.00%> (-0.02%)` | :arrow_down: |
   
   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/17043?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [sdks/python/apache\_beam/dataframe/frames.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vZGF0YWZyYW1lL2ZyYW1lcy5weQ==) | `95.17% <100.00%> (+0.03%)` | :arrow_up: |
   | [sdks/python/apache\_beam/internal/metrics/metric.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vaW50ZXJuYWwvbWV0cmljcy9tZXRyaWMucHk=) | `90.00% <0.00%> (-1.00%)` | :arrow_down: |
   | [...pache\_beam/runners/interactive/interactive\_beam.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9pbnRlcmFjdGl2ZS9pbnRlcmFjdGl2ZV9iZWFtLnB5) | `80.62% <0.00%> (-0.78%)` | :arrow_down: |
   | [...che\_beam/runners/interactive/interactive\_runner.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9pbnRlcmFjdGl2ZS9pbnRlcmFjdGl2ZV9ydW5uZXIucHk=) | `92.02% <0.00%> (-0.73%)` | :arrow_down: |
   | [sdks/python/apache\_beam/runners/direct/executor.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9kaXJlY3QvZXhlY3V0b3IucHk=) | `96.46% <0.00%> (-0.55%)` | :arrow_down: |
   | [...runners/interactive/display/pcoll\_visualization.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9pbnRlcmFjdGl2ZS9kaXNwbGF5L3Bjb2xsX3Zpc3VhbGl6YXRpb24ucHk=) | `85.85% <0.00%> (-0.51%)` | :arrow_down: |
   | [...dks/python/apache\_beam/options/pipeline\_options.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vb3B0aW9ucy9waXBlbGluZV9vcHRpb25zLnB5) | `94.85% <0.00%> (-0.43%)` | :arrow_down: |
   | [...eam/runners/interactive/interactive\_environment.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9pbnRlcmFjdGl2ZS9pbnRlcmFjdGl2ZV9lbnZpcm9ubWVudC5weQ==) | `90.44% <0.00%> (-0.30%)` | :arrow_down: |
   | [...ks/python/apache\_beam/runners/worker/sdk\_worker.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy93b3JrZXIvc2RrX3dvcmtlci5weQ==) | `88.90% <0.00%> (-0.16%)` | :arrow_down: |
   | [...nternal/clients/dataflow/dataflow\_v1b3\_messages.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9kYXRhZmxvdy9pbnRlcm5hbC9jbGllbnRzL2RhdGFmbG93L2RhdGFmbG93X3YxYjNfbWVzc2FnZXMucHk=) | `100.00% <0.00%> (ø)` | |
   | ... and [3 more](https://codecov.io/gh/apache/beam/pull/17043/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [58f8bef...5c6fd0a](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] yeandy commented on a change in pull request #17043: [BEAM-13966] Add pivot(), a non-deferred column operation on categorical columns

Posted by GitBox <gi...@apache.org>.
yeandy commented on a change in pull request #17043:
URL: https://github.com/apache/beam/pull/17043#discussion_r831083933



##########
File path: sdks/python/apache_beam/dataframe/frames.py
##########
@@ -3655,6 +3655,40 @@ def shift(self, axis, freq, **kwargs):
   describe = _agg_method(pd.DataFrame, 'describe')
   max = _agg_method(pd.DataFrame, 'max')
   min = _agg_method(pd.DataFrame, 'min')
+
+  @frame_base.with_docs_from(pd.DataFrame)
+  @frame_base.args_to_kwargs(pd.DataFrame)
+  @frame_base.populate_defaults(pd.DataFrame)
+  def pivot(self, **kwargs):
+    columns = kwargs.get('columns', None)
+    selected_cols = self._expr.proxy()[columns]
+
+    if isinstance(selected_cols, pd.Series):
+      all_cols_are_categorical = isinstance(
+        selected_cols.dtype, pd.CategoricalDtype
+      )
+    else:
+      all_cols_are_categorical = all(
+        isinstance(c, pd.CategoricalDtype) for c in selected_cols.dtypes
+      )
+    if not all_cols_are_categorical:
+      raise frame_base.WontImplementError(
+          "pivot() of non-categorical type is not supported because "
+          "the type of the output column depends on the data. Please use "
+          "pd.CategoricalDtype with explicit categories.",
+          reason="non-deferred-columns")
+
+    proxy = pd.DataFrame(columns=self._expr.proxy().pivot(**kwargs).columns)
+
+    with expressions.allow_non_parallel_operations():

Review comment:
       Changed!




-- 
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 pull request #17043: [BEAM-13966] Add pivot(), a non-deferred column operation on categorical columns

Posted by GitBox <gi...@apache.org>.
yeandy commented on pull request #17043:
URL: https://github.com/apache/beam/pull/17043#issuecomment-1062178806


   R: @TheNeuralBit 


-- 
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 change in pull request #17043: [BEAM-13966] Add pivot(), a non-deferred column operation on categorical columns

Posted by GitBox <gi...@apache.org>.
TheNeuralBit commented on a change in pull request #17043:
URL: https://github.com/apache/beam/pull/17043#discussion_r838923085



##########
File path: sdks/python/apache_beam/dataframe/frames_test.py
##########
@@ -1295,6 +1295,114 @@ def s_times_shuffled(times, s):
     self._run_test(lambda s: s.pipe(s_times, 2), s)
     self._run_test(lambda s: s.pipe((s_times_shuffled, 's'), 2), s)
 
+  def test_pivot_non_categorical(self):
+    df = pd.DataFrame({
+        'foo': ['one', 'one', 'one', 'two', 'two', 'two'],
+        'bar': ['A', 'B', 'C', 'A', 'B', 'C'],
+        'baz': [1, 2, 3, 4, 5, 6],
+        'zoo': ['x', 'y', 'z', 'q', 'w', 't']
+    })
+    with self.assertRaisesRegex(
+        frame_base.WontImplementError,
+        r"pivot\(\) of non-categorical type is not supported"):
+      self._run_test(
+          lambda df: df.pivot(index='foo', columns='bar', values='baz'), df)
+
+  def test_pivot_pandas_example1(self):
+    # Simple test 1
+    df = pd.DataFrame({
+        'foo': ['one', 'one', 'one', 'two', 'two', 'two'],
+        'bar': ['A', 'B', 'C', 'A', 'B', 'C'],
+        'baz': [1, 2, 3, 4, 5, 6],
+        'zoo': ['x', 'y', 'z', 'q', 'w', 't']
+    })
+    df['bar'] = df['bar'].astype(
+        pd.CategoricalDtype(categories=['A', 'B', 'C']))
+    self._run_test(
+        lambda df: df.pivot(index='foo', columns='bar', values='baz'), df)
+
+  def test_pivot_pandas_example3(self):
+    # Multiple values
+    df = pd.DataFrame({
+        'foo': ['one', 'one', 'one', 'two', 'two', 'two'],
+        'bar': ['A', 'B', 'C', 'A', 'B', 'C'],
+        'baz': [1, 2, 3, 4, 5, 6],
+        'zoo': ['x', 'y', 'z', 'q', 'w', 't']
+    })
+    df['bar'] = df['bar'].astype(
+        pd.CategoricalDtype(categories=['A', 'B', 'C']))
+    self._run_test(
+        lambda df: df.pivot(index='foo', columns='bar', values=['baz', 'zoo']),
+        df)
+
+  def test_pivot_pandas_example4(self):
+    # Multiple columns
+    df = pd.DataFrame({
+        "lev1": [1, 1, 1, 2, 2, 2],
+        "lev2": [1, 1, 2, 1, 1, 2],
+        "lev3": [1, 2, 1, 2, 1, 2],
+        "lev4": [1, 2, 3, 4, 5, 6],
+        "values": [0, 1, 2, 3, 4, 5]
+    })
+    df['lev2'] = df['lev2'].astype(pd.CategoricalDtype(categories=[1, 2]))
+    df['lev3'] = df['lev3'].astype(pd.CategoricalDtype(categories=[1, 2]))
+    df['values'] = df['values'].astype('Int64')
+    self._run_test(
+        lambda df: df.pivot(
+            index="lev1", columns=["lev2", "lev3"], values="values"),
+        df)
+
+  @unittest.skipIf(
+      PD_VERSION < (1, 4), "Bug in DF.pivot with MultiIndex for pandas < 1.4")

Review comment:
       Got it, thanks, so the bug is that `df.pivot(index=None)` fails if index is a MultiIndex. It's a problem for Beam because we'll call it in `pivot_helper` at execution time.
   
   Since this is an execution time error, could you make the pivot() implementation fail early if `PD_VERSION < (1, 4)` and multiple indexes are used?
   
   Sorry to be so nitpicky here, but errors that only happen at execution time can be _very_ frustrating for users (e.g. see this [tweet](https://twitter.com/adamlaiacano/status/1275961053067841539)) so I like to fail early if at all possible.




-- 
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 change in pull request #17043: [BEAM-13966] Add pivot(), a non-deferred column operation on categorical columns

Posted by GitBox <gi...@apache.org>.
TheNeuralBit commented on a change in pull request #17043:
URL: https://github.com/apache/beam/pull/17043#discussion_r838929290



##########
File path: sdks/python/apache_beam/dataframe/frames.py
##########
@@ -3655,6 +3656,109 @@ def shift(self, axis, freq, **kwargs):
   describe = _agg_method(pd.DataFrame, 'describe')
   max = _agg_method(pd.DataFrame, 'max')
   min = _agg_method(pd.DataFrame, 'min')
+
+  @frame_base.with_docs_from(pd.DataFrame)
+  @frame_base.args_to_kwargs(pd.DataFrame)
+  @frame_base.populate_defaults(pd.DataFrame)
+  def pivot(self, index=None, columns=None, values=None, **kwargs):

Review comment:
       Oh please also add a docstring here indicating that any columns specified in `columns` must have CategoricalDType so we can determine the output column names. There's docstring infrastructure that will put this message in the "Differences from pandas" section, e.g. for align: 
   ![image](https://user-images.githubusercontent.com/675055/160923353-f75e81b7-ea4b-48ee-8567-c78016799cdc.png)
   




-- 
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 change in pull request #17043: [BEAM-13966] Add pivot(), a non-deferred column operation on categorical columns

Posted by GitBox <gi...@apache.org>.
yeandy commented on a change in pull request #17043:
URL: https://github.com/apache/beam/pull/17043#discussion_r837878183



##########
File path: sdks/python/apache_beam/dataframe/frames.py
##########
@@ -3655,6 +3656,121 @@ def shift(self, axis, freq, **kwargs):
   describe = _agg_method(pd.DataFrame, 'describe')
   max = _agg_method(pd.DataFrame, 'max')
   min = _agg_method(pd.DataFrame, 'min')
+
+  @frame_base.with_docs_from(pd.DataFrame)
+  @frame_base.args_to_kwargs(pd.DataFrame)
+  @frame_base.populate_defaults(pd.DataFrame)
+  def pivot(self, index=None, columns=None, values=None, **kwargs):
+    def verify_all_categorical(all_cols_are_categorical):
+      if not all_cols_are_categorical:
+        raise frame_base.WontImplementError(
+            "pivot() of non-categorical type is not supported because "
+            "the type of the output column depends on the data. Please use "
+            "pd.CategoricalDtype with explicit categories.",
+            reason="non-deferred-columns")

Review comment:
       Fixed!

##########
File path: sdks/python/apache_beam/dataframe/frames.py
##########
@@ -3655,6 +3656,121 @@ def shift(self, axis, freq, **kwargs):
   describe = _agg_method(pd.DataFrame, 'describe')
   max = _agg_method(pd.DataFrame, 'max')
   min = _agg_method(pd.DataFrame, 'min')
+
+  @frame_base.with_docs_from(pd.DataFrame)
+  @frame_base.args_to_kwargs(pd.DataFrame)
+  @frame_base.populate_defaults(pd.DataFrame)
+  def pivot(self, index=None, columns=None, values=None, **kwargs):
+    def verify_all_categorical(all_cols_are_categorical):
+      if not all_cols_are_categorical:
+        raise frame_base.WontImplementError(
+            "pivot() of non-categorical type is not supported because "
+            "the type of the output column depends on the data. Please use "
+            "pd.CategoricalDtype with explicit categories.",
+            reason="non-deferred-columns")
+
+    # Construct column index
+    if is_list_like(columns) and len(columns) <= 1:
+      columns = columns[0]
+    selected_cols = self._expr.proxy()[columns]
+    if isinstance(selected_cols, pd.Series):
+      all_cols_are_categorical = isinstance(
+        selected_cols.dtype, pd.CategoricalDtype
+      )
+      verify_all_categorical(all_cols_are_categorical)
+
+      # If values not provided, take all remaining columns of dataframe
+      if not values:
+        values = self._expr.proxy() \
+          .drop(index, axis=1).drop(columns, axis=1).columns.values
+
+      # Take the provided values
+      if is_list_like(values) and len(values) > 1:
+        values_in_col_index = values
+        names = [None, columns]
+        col_index = pd.MultiIndex.from_product(
+          [values_in_col_index,
+          selected_cols.dtypes.categories.astype('category')],
+          names=names
+        )
+      else:
+        col_index = pd.CategoricalIndex(
+          selected_cols.dtype.categories,
+          name=columns
+        )
+    else:
+      all_cols_are_categorical = all(
+        isinstance(c, pd.CategoricalDtype) for c in selected_cols.dtypes
+      )
+      verify_all_categorical(all_cols_are_categorical)
+
+      categories = [
+        c.categories.astype('category') for c in selected_cols.dtypes
+      ]
+      if is_list_like(columns) and len(columns) > 1:

Review comment:
       True, good catch!




-- 
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] edited a comment on pull request #17043: [BEAM-13966] Add pivot(), a non-deferred column operation on categorical columns

Posted by GitBox <gi...@apache.org>.
codecov[bot] edited a comment on pull request #17043:
URL: https://github.com/apache/beam/pull/17043#issuecomment-1062140334


   # [Codecov](https://codecov.io/gh/apache/beam/pull/17043?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 [#17043](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (0e046fc) into [master](https://codecov.io/gh/apache/beam/commit/58f8bef66f4da09202b2de9dc3166c7fa1f4fa4e?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (58f8bef) will **increase** coverage by `0.18%`.
   > The diff coverage is `97.95%`.
   
   ```diff
   @@            Coverage Diff             @@
   ##           master   #17043      +/-   ##
   ==========================================
   + Coverage   73.83%   74.01%   +0.18%     
   ==========================================
     Files         667      667              
     Lines       87392    88667    +1275     
   ==========================================
   + Hits        64523    65626    +1103     
   - Misses      21766    21938     +172     
     Partials     1103     1103              
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | python | `83.70% <97.95%> (+0.05%)` | :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/17043?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [sdks/python/apache\_beam/dataframe/frames.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vZGF0YWZyYW1lL2ZyYW1lcy5weQ==) | `95.20% <97.95%> (+0.06%)` | :arrow_up: |
   | [...nternal/clients/cloudbuild/cloudbuild\_v1\_client.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9kYXRhZmxvdy9pbnRlcm5hbC9jbGllbnRzL2Nsb3VkYnVpbGQvY2xvdWRidWlsZF92MV9jbGllbnQucHk=) | `52.21% <0.00%> (-2.97%)` | :arrow_down: |
   | [sdks/python/apache\_beam/utils/interactive\_utils.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vdXRpbHMvaW50ZXJhY3RpdmVfdXRpbHMucHk=) | `92.68% <0.00%> (-2.44%)` | :arrow_down: |
   | [...thon/apache\_beam/runners/worker/sdk\_worker\_main.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy93b3JrZXIvc2RrX3dvcmtlcl9tYWluLnB5) | `74.12% <0.00%> (-1.40%)` | :arrow_down: |
   | [...\_beam/runners/portability/sdk\_container\_builder.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9wb3J0YWJpbGl0eS9zZGtfY29udGFpbmVyX2J1aWxkZXIucHk=) | `37.93% <0.00%> (-1.02%)` | :arrow_down: |
   | [...n/apache\_beam/runners/interactive/cache\_manager.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9pbnRlcmFjdGl2ZS9jYWNoZV9tYW5hZ2VyLnB5) | `87.58% <0.00%> (-1.01%)` | :arrow_down: |
   | [...pache\_beam/runners/interactive/interactive\_beam.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9pbnRlcmFjdGl2ZS9pbnRlcmFjdGl2ZV9iZWFtLnB5) | `80.62% <0.00%> (-0.78%)` | :arrow_down: |
   | [sdks/python/apache\_beam/runners/direct/executor.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9kaXJlY3QvZXhlY3V0b3IucHk=) | `96.46% <0.00%> (-0.55%)` | :arrow_down: |
   | [...runners/interactive/display/pcoll\_visualization.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9pbnRlcmFjdGl2ZS9kaXNwbGF5L3Bjb2xsX3Zpc3VhbGl6YXRpb24ucHk=) | `85.85% <0.00%> (-0.51%)` | :arrow_down: |
   | [sdks/python/apache\_beam/internal/gcp/auth.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vaW50ZXJuYWwvZ2NwL2F1dGgucHk=) | `81.03% <0.00%> (-0.45%)` | :arrow_down: |
   | ... and [28 more](https://codecov.io/gh/apache/beam/pull/17043/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [58f8bef...0e046fc](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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 a change in pull request #17043: [BEAM-13966] Add pivot(), a non-deferred column operation on categorical columns

Posted by GitBox <gi...@apache.org>.
TheNeuralBit commented on a change in pull request #17043:
URL: https://github.com/apache/beam/pull/17043#discussion_r826434766



##########
File path: sdks/python/apache_beam/dataframe/frames_test.py
##########
@@ -2209,6 +2209,112 @@ def test_nlargest_any(self):
     # but not necessarily with the same index
     self.assert_frame_data_equivalent(result, df.population.nlargest(3))
 
+  def test_pivot_non_categorical(self):
+    df = pd.DataFrame({
+        'foo': ['one', 'one', 'one', 'two', 'two', 'two'],
+        'bar': ['A', 'B', 'C', 'A', 'B', 'C'],
+        'baz': [1, 2, 3, 4, 5, 6],
+        'zoo': ['x', 'y', 'z', 'q', 'w', 't']
+    })
+    with self.assertRaisesRegex(
+        frame_base.WontImplementError,
+        r"pivot\(\) of non-categorical type is not supported"):
+      self._evaluate(
+          lambda df: df.pivot(index='foo', columns='bar', values='baz'), df)
+
+  def test_pivot_pandas_example1(self):
+    # Simple test 1
+    df = pd.DataFrame({
+        'foo': ['one', 'one', 'one', 'two', 'two', 'two'],
+        'bar': ['A', 'B', 'C', 'A', 'B', 'C'],
+        'baz': [1, 2, 3, 4, 5, 6],
+        'zoo': ['x', 'y', 'z', 'q', 'w', 't']
+    })
+    df['bar'] = df['bar'].astype(
+        pd.CategoricalDtype(categories=['A', 'B', 'C']))
+    result = self._evaluate(
+        lambda df: df.pivot(index='foo', columns='bar', values='baz'), df)
+    self.assert_frame_data_equivalent(
+        result, df.pivot(index='foo', columns='bar', values='baz'))

Review comment:
       I think it would be preferable to test the positive test cases the standard way (in `DeferredFrameTest` rather than in `BeamSpecificTest`). These cases _should_ work the same as they in standard pandas. We should only test it here if we don't expect the results to be identical for some reason.

##########
File path: sdks/python/apache_beam/dataframe/frames_test.py
##########
@@ -2209,6 +2209,112 @@ def test_nlargest_any(self):
     # but not necessarily with the same index
     self.assert_frame_data_equivalent(result, df.population.nlargest(3))
 
+  def test_pivot_non_categorical(self):
+    df = pd.DataFrame({
+        'foo': ['one', 'one', 'one', 'two', 'two', 'two'],
+        'bar': ['A', 'B', 'C', 'A', 'B', 'C'],
+        'baz': [1, 2, 3, 4, 5, 6],
+        'zoo': ['x', 'y', 'z', 'q', 'w', 't']
+    })
+    with self.assertRaisesRegex(
+        frame_base.WontImplementError,
+        r"pivot\(\) of non-categorical type is not supported"):
+      self._evaluate(
+          lambda df: df.pivot(index='foo', columns='bar', values='baz'), df)
+
+  def test_pivot_pandas_example1(self):
+    # Simple test 1
+    df = pd.DataFrame({
+        'foo': ['one', 'one', 'one', 'two', 'two', 'two'],
+        'bar': ['A', 'B', 'C', 'A', 'B', 'C'],
+        'baz': [1, 2, 3, 4, 5, 6],
+        'zoo': ['x', 'y', 'z', 'q', 'w', 't']
+    })
+    df['bar'] = df['bar'].astype(
+        pd.CategoricalDtype(categories=['A', 'B', 'C']))
+    result = self._evaluate(
+        lambda df: df.pivot(index='foo', columns='bar', values='baz'), df)
+    self.assert_frame_data_equivalent(
+        result, df.pivot(index='foo', columns='bar', values='baz'))
+
+  def test_pivot_pandas_example2(self):
+    # Simple test 2
+    df = pd.DataFrame({
+        'foo': ['one', 'one', 'one', 'two', 'two', 'two'],
+        'bar': ['A', 'B', 'C', 'A', 'B', 'C'],
+        'baz': [1, 2, 3, 4, 5, 6],
+        'zoo': ['x', 'y', 'z', 'q', 'w', 't']
+    })
+    df['bar'] = df['bar'].astype(
+        pd.CategoricalDtype(categories=['A', 'B', 'C']))
+    result = self._evaluate(lambda df: df.pivot(index='foo', columns='bar'), df)
+    self.assert_frame_data_equivalent(
+        result['baz'], df.pivot(index='foo', columns='bar')['baz'])

Review comment:
       Ah, this looks like at least one case that should be tested here, since we're only verifying `baz`. Is there some reasonable verification we can do on the rest of `result` (e.g. should it be all NaNs)?

##########
File path: sdks/python/apache_beam/dataframe/frames.py
##########
@@ -3655,6 +3655,40 @@ def shift(self, axis, freq, **kwargs):
   describe = _agg_method(pd.DataFrame, 'describe')
   max = _agg_method(pd.DataFrame, 'max')
   min = _agg_method(pd.DataFrame, 'min')
+
+  @frame_base.with_docs_from(pd.DataFrame)
+  @frame_base.args_to_kwargs(pd.DataFrame)
+  @frame_base.populate_defaults(pd.DataFrame)
+  def pivot(self, **kwargs):
+    columns = kwargs.get('columns', None)
+    selected_cols = self._expr.proxy()[columns]
+
+    if isinstance(selected_cols, pd.Series):
+      all_cols_are_categorical = isinstance(
+        selected_cols.dtype, pd.CategoricalDtype
+      )
+    else:
+      all_cols_are_categorical = all(
+        isinstance(c, pd.CategoricalDtype) for c in selected_cols.dtypes
+      )
+    if not all_cols_are_categorical:
+      raise frame_base.WontImplementError(
+          "pivot() of non-categorical type is not supported because "
+          "the type of the output column depends on the data. Please use "
+          "pd.CategoricalDtype with explicit categories.",
+          reason="non-deferred-columns")
+
+    proxy = pd.DataFrame(columns=self._expr.proxy().pivot(**kwargs).columns)
+
+    with expressions.allow_non_parallel_operations():

Review comment:
       I don't think it's a good idea to bypass the non-parallel operations here. The places we do that are cases where we know the data has already been reduced substantially (e.g. we just need to combine partial results for a global aggregation). In this case the input _could_ be very large, so we'd want the user to have to acknowledge that they want to use Singleton partitioning.
   
   That being said, I think we should be able to do this without Singleton partitioning. What if we first move the `index` column(s) to the index in one expression (if set), then apply the `columns`/`values` part in a separate expression, with `requires=Index()`?




-- 
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] edited a comment on pull request #17043: [BEAM-13966] Add pivot(), a non-deferred column operation on categorical columns

Posted by GitBox <gi...@apache.org>.
codecov[bot] edited a comment on pull request #17043:
URL: https://github.com/apache/beam/pull/17043#issuecomment-1062140334


   # [Codecov](https://codecov.io/gh/apache/beam/pull/17043?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 [#17043](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (ba3f2b6) into [master](https://codecov.io/gh/apache/beam/commit/58f8bef66f4da09202b2de9dc3166c7fa1f4fa4e?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (58f8bef) will **increase** coverage by `0.07%`.
   > The diff coverage is `98.14%`.
   
   ```diff
   @@            Coverage Diff             @@
   ##           master   #17043      +/-   ##
   ==========================================
   + Coverage   73.83%   73.90%   +0.07%     
   ==========================================
     Files         667      667              
     Lines       87392    88071     +679     
   ==========================================
   + Hits        64523    65089     +566     
   - Misses      21766    21879     +113     
     Partials     1103     1103              
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | python | `83.64% <98.14%> (-0.01%)` | :arrow_down: |
   
   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/17043?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [sdks/python/apache\_beam/dataframe/frames.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vZGF0YWZyYW1lL2ZyYW1lcy5weQ==) | `95.21% <98.14%> (+0.07%)` | :arrow_up: |
   | [...nternal/clients/cloudbuild/cloudbuild\_v1\_client.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9kYXRhZmxvdy9pbnRlcm5hbC9jbGllbnRzL2Nsb3VkYnVpbGQvY2xvdWRidWlsZF92MV9jbGllbnQucHk=) | `52.21% <0.00%> (-2.97%)` | :arrow_down: |
   | [sdks/python/apache\_beam/utils/interactive\_utils.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vdXRpbHMvaW50ZXJhY3RpdmVfdXRpbHMucHk=) | `92.68% <0.00%> (-2.44%)` | :arrow_down: |
   | [.../python/apache\_beam/transforms/periodicsequence.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vdHJhbnNmb3Jtcy9wZXJpb2RpY3NlcXVlbmNlLnB5) | `96.72% <0.00%> (-1.64%)` | :arrow_down: |
   | [...thon/apache\_beam/runners/worker/sdk\_worker\_main.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy93b3JrZXIvc2RrX3dvcmtlcl9tYWluLnB5) | `74.12% <0.00%> (-1.40%)` | :arrow_down: |
   | [...\_beam/runners/portability/sdk\_container\_builder.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9wb3J0YWJpbGl0eS9zZGtfY29udGFpbmVyX2J1aWxkZXIucHk=) | `37.93% <0.00%> (-1.02%)` | :arrow_down: |
   | [...pache\_beam/runners/interactive/interactive\_beam.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9pbnRlcmFjdGl2ZS9pbnRlcmFjdGl2ZV9iZWFtLnB5) | `80.62% <0.00%> (-0.78%)` | :arrow_down: |
   | [...che\_beam/runners/interactive/interactive\_runner.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9pbnRlcmFjdGl2ZS9pbnRlcmFjdGl2ZV9ydW5uZXIucHk=) | `92.02% <0.00%> (-0.73%)` | :arrow_down: |
   | [sdks/python/apache\_beam/runners/direct/executor.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9kaXJlY3QvZXhlY3V0b3IucHk=) | `96.46% <0.00%> (-0.55%)` | :arrow_down: |
   | [...runners/interactive/display/pcoll\_visualization.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9pbnRlcmFjdGl2ZS9kaXNwbGF5L3Bjb2xsX3Zpc3VhbGl6YXRpb24ucHk=) | `85.85% <0.00%> (-0.51%)` | :arrow_down: |
   | ... and [15 more](https://codecov.io/gh/apache/beam/pull/17043/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [58f8bef...ba3f2b6](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] codecov[bot] edited a comment on pull request #17043: [BEAM-13966] Add pivot(), a non-deferred column operation on categorical columns

Posted by GitBox <gi...@apache.org>.
codecov[bot] edited a comment on pull request #17043:
URL: https://github.com/apache/beam/pull/17043#issuecomment-1062140334


   # [Codecov](https://codecov.io/gh/apache/beam/pull/17043?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 [#17043](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (5c6fd0a) into [master](https://codecov.io/gh/apache/beam/commit/58f8bef66f4da09202b2de9dc3166c7fa1f4fa4e?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (58f8bef) will **increase** coverage by `0.07%`.
   > The diff coverage is `98.14%`.
   
   > :exclamation: Current head 5c6fd0a differs from pull request most recent head ba3f2b6. Consider uploading reports for the commit ba3f2b6 to get more accurate results
   
   ```diff
   @@            Coverage Diff             @@
   ##           master   #17043      +/-   ##
   ==========================================
   + Coverage   73.83%   73.90%   +0.07%     
   ==========================================
     Files         667      667              
     Lines       87392    88071     +679     
   ==========================================
   + Hits        64523    65089     +566     
   - Misses      21766    21879     +113     
     Partials     1103     1103              
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | python | `83.64% <98.14%> (-0.01%)` | :arrow_down: |
   
   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/17043?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [sdks/python/apache\_beam/dataframe/frames.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vZGF0YWZyYW1lL2ZyYW1lcy5weQ==) | `95.21% <98.14%> (+0.07%)` | :arrow_up: |
   | [...nternal/clients/cloudbuild/cloudbuild\_v1\_client.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9kYXRhZmxvdy9pbnRlcm5hbC9jbGllbnRzL2Nsb3VkYnVpbGQvY2xvdWRidWlsZF92MV9jbGllbnQucHk=) | `52.21% <0.00%> (-2.97%)` | :arrow_down: |
   | [sdks/python/apache\_beam/utils/interactive\_utils.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vdXRpbHMvaW50ZXJhY3RpdmVfdXRpbHMucHk=) | `92.68% <0.00%> (-2.44%)` | :arrow_down: |
   | [...thon/apache\_beam/runners/worker/sdk\_worker\_main.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy93b3JrZXIvc2RrX3dvcmtlcl9tYWluLnB5) | `74.12% <0.00%> (-1.40%)` | :arrow_down: |
   | [...\_beam/runners/portability/sdk\_container\_builder.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9wb3J0YWJpbGl0eS9zZGtfY29udGFpbmVyX2J1aWxkZXIucHk=) | `37.93% <0.00%> (-1.02%)` | :arrow_down: |
   | [sdks/python/apache\_beam/internal/metrics/metric.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vaW50ZXJuYWwvbWV0cmljcy9tZXRyaWMucHk=) | `90.00% <0.00%> (-1.00%)` | :arrow_down: |
   | [...pache\_beam/runners/interactive/interactive\_beam.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9pbnRlcmFjdGl2ZS9pbnRlcmFjdGl2ZV9iZWFtLnB5) | `80.62% <0.00%> (-0.78%)` | :arrow_down: |
   | [...che\_beam/runners/interactive/interactive\_runner.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9pbnRlcmFjdGl2ZS9pbnRlcmFjdGl2ZV9ydW5uZXIucHk=) | `92.02% <0.00%> (-0.73%)` | :arrow_down: |
   | [sdks/python/apache\_beam/runners/direct/executor.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9kaXJlY3QvZXhlY3V0b3IucHk=) | `96.46% <0.00%> (-0.55%)` | :arrow_down: |
   | [...runners/interactive/display/pcoll\_visualization.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9pbnRlcmFjdGl2ZS9kaXNwbGF5L3Bjb2xsX3Zpc3VhbGl6YXRpb24ucHk=) | `85.85% <0.00%> (-0.51%)` | :arrow_down: |
   | ... and [15 more](https://codecov.io/gh/apache/beam/pull/17043/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [58f8bef...ba3f2b6](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] codecov[bot] edited a comment on pull request #17043: [BEAM-13966] Add pivot(), a non-deferred column operation on categorical columns

Posted by GitBox <gi...@apache.org>.
codecov[bot] edited a comment on pull request #17043:
URL: https://github.com/apache/beam/pull/17043#issuecomment-1062140334


   # [Codecov](https://codecov.io/gh/apache/beam/pull/17043?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 [#17043](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (5c6fd0a) into [master](https://codecov.io/gh/apache/beam/commit/58f8bef66f4da09202b2de9dc3166c7fa1f4fa4e?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (58f8bef) will **increase** coverage by `0.07%`.
   > The diff coverage is `98.14%`.
   
   ```diff
   @@            Coverage Diff             @@
   ##           master   #17043      +/-   ##
   ==========================================
   + Coverage   73.83%   73.90%   +0.07%     
   ==========================================
     Files         667      667              
     Lines       87392    88071     +679     
   ==========================================
   + Hits        64523    65089     +566     
   - Misses      21766    21879     +113     
     Partials     1103     1103              
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | python | `83.64% <98.14%> (-0.01%)` | :arrow_down: |
   
   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/17043?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [sdks/python/apache\_beam/dataframe/frames.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vZGF0YWZyYW1lL2ZyYW1lcy5weQ==) | `95.21% <98.14%> (+0.07%)` | :arrow_up: |
   | [...nternal/clients/cloudbuild/cloudbuild\_v1\_client.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9kYXRhZmxvdy9pbnRlcm5hbC9jbGllbnRzL2Nsb3VkYnVpbGQvY2xvdWRidWlsZF92MV9jbGllbnQucHk=) | `52.21% <0.00%> (-2.97%)` | :arrow_down: |
   | [sdks/python/apache\_beam/utils/interactive\_utils.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vdXRpbHMvaW50ZXJhY3RpdmVfdXRpbHMucHk=) | `92.68% <0.00%> (-2.44%)` | :arrow_down: |
   | [...thon/apache\_beam/runners/worker/sdk\_worker\_main.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy93b3JrZXIvc2RrX3dvcmtlcl9tYWluLnB5) | `74.12% <0.00%> (-1.40%)` | :arrow_down: |
   | [...\_beam/runners/portability/sdk\_container\_builder.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9wb3J0YWJpbGl0eS9zZGtfY29udGFpbmVyX2J1aWxkZXIucHk=) | `37.93% <0.00%> (-1.02%)` | :arrow_down: |
   | [sdks/python/apache\_beam/internal/metrics/metric.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vaW50ZXJuYWwvbWV0cmljcy9tZXRyaWMucHk=) | `90.00% <0.00%> (-1.00%)` | :arrow_down: |
   | [...pache\_beam/runners/interactive/interactive\_beam.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9pbnRlcmFjdGl2ZS9pbnRlcmFjdGl2ZV9iZWFtLnB5) | `80.62% <0.00%> (-0.78%)` | :arrow_down: |
   | [...che\_beam/runners/interactive/interactive\_runner.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9pbnRlcmFjdGl2ZS9pbnRlcmFjdGl2ZV9ydW5uZXIucHk=) | `92.02% <0.00%> (-0.73%)` | :arrow_down: |
   | [sdks/python/apache\_beam/runners/direct/executor.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9kaXJlY3QvZXhlY3V0b3IucHk=) | `96.46% <0.00%> (-0.55%)` | :arrow_down: |
   | [...runners/interactive/display/pcoll\_visualization.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9pbnRlcmFjdGl2ZS9kaXNwbGF5L3Bjb2xsX3Zpc3VhbGl6YXRpb24ucHk=) | `85.85% <0.00%> (-0.51%)` | :arrow_down: |
   | ... and [15 more](https://codecov.io/gh/apache/beam/pull/17043/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [58f8bef...5c6fd0a](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] codecov[bot] commented on pull request #17043: [BEAM-13966] Add pivot(), a non-deferred column operation on categorical columns

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


   # [Codecov](https://codecov.io/gh/apache/beam/pull/17043?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 [#17043](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (de90f88) into [master](https://codecov.io/gh/apache/beam/commit/58f8bef66f4da09202b2de9dc3166c7fa1f4fa4e?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (58f8bef) will **decrease** coverage by `0.01%`.
   > The diff coverage is `100.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/beam/pull/17043/graphs/tree.svg?width=650&height=150&src=pr&token=qcbbAh8Fj1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master   #17043      +/-   ##
   ==========================================
   - Coverage   73.83%   73.81%   -0.02%     
   ==========================================
     Files         667      667              
     Lines       87392    87320      -72     
   ==========================================
   - Hits        64523    64456      -67     
   + Misses      21766    21761       -5     
     Partials     1103     1103              
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | python | `83.63% <100.00%> (-0.02%)` | :arrow_down: |
   
   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/17043?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [sdks/python/apache\_beam/dataframe/frames.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vZGF0YWZyYW1lL2ZyYW1lcy5weQ==) | `95.17% <100.00%> (+0.03%)` | :arrow_up: |
   | [sdks/python/apache\_beam/internal/metrics/metric.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vaW50ZXJuYWwvbWV0cmljcy9tZXRyaWMucHk=) | `90.00% <0.00%> (-1.00%)` | :arrow_down: |
   | [...pache\_beam/runners/interactive/interactive\_beam.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9pbnRlcmFjdGl2ZS9pbnRlcmFjdGl2ZV9iZWFtLnB5) | `80.62% <0.00%> (-0.78%)` | :arrow_down: |
   | [...che\_beam/runners/interactive/interactive\_runner.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9pbnRlcmFjdGl2ZS9pbnRlcmFjdGl2ZV9ydW5uZXIucHk=) | `92.02% <0.00%> (-0.73%)` | :arrow_down: |
   | [sdks/python/apache\_beam/runners/direct/executor.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9kaXJlY3QvZXhlY3V0b3IucHk=) | `96.46% <0.00%> (-0.55%)` | :arrow_down: |
   | [...runners/interactive/display/pcoll\_visualization.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9pbnRlcmFjdGl2ZS9kaXNwbGF5L3Bjb2xsX3Zpc3VhbGl6YXRpb24ucHk=) | `85.85% <0.00%> (-0.51%)` | :arrow_down: |
   | [...dks/python/apache\_beam/options/pipeline\_options.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vb3B0aW9ucy9waXBlbGluZV9vcHRpb25zLnB5) | `94.85% <0.00%> (-0.43%)` | :arrow_down: |
   | [...eam/runners/interactive/interactive\_environment.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9pbnRlcmFjdGl2ZS9pbnRlcmFjdGl2ZV9lbnZpcm9ubWVudC5weQ==) | `90.44% <0.00%> (-0.30%)` | :arrow_down: |
   | [...ks/python/apache\_beam/runners/worker/sdk\_worker.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy93b3JrZXIvc2RrX3dvcmtlci5weQ==) | `88.90% <0.00%> (-0.16%)` | :arrow_down: |
   | [...nternal/clients/dataflow/dataflow\_v1b3\_messages.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9kYXRhZmxvdy9pbnRlcm5hbC9jbGllbnRzL2RhdGFmbG93L2RhdGFmbG93X3YxYjNfbWVzc2FnZXMucHk=) | `100.00% <0.00%> (ø)` | |
   | ... and [3 more](https://codecov.io/gh/apache/beam/pull/17043/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [58f8bef...de90f88](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] codecov[bot] edited a comment on pull request #17043: [BEAM-13966] Add pivot(), a non-deferred column operation on categorical columns

Posted by GitBox <gi...@apache.org>.
codecov[bot] edited a comment on pull request #17043:
URL: https://github.com/apache/beam/pull/17043#issuecomment-1062140334


   # [Codecov](https://codecov.io/gh/apache/beam/pull/17043?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 [#17043](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (ba3f2b6) into [master](https://codecov.io/gh/apache/beam/commit/58f8bef66f4da09202b2de9dc3166c7fa1f4fa4e?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (58f8bef) will **increase** coverage by `0.07%`.
   > The diff coverage is `98.14%`.
   
   > :exclamation: Current head ba3f2b6 differs from pull request most recent head 0e046fc. Consider uploading reports for the commit 0e046fc to get more accurate results
   
   ```diff
   @@            Coverage Diff             @@
   ##           master   #17043      +/-   ##
   ==========================================
   + Coverage   73.83%   73.90%   +0.07%     
   ==========================================
     Files         667      667              
     Lines       87392    88071     +679     
   ==========================================
   + Hits        64523    65089     +566     
   - Misses      21766    21879     +113     
     Partials     1103     1103              
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | python | `83.64% <98.14%> (-0.01%)` | :arrow_down: |
   
   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/17043?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [sdks/python/apache\_beam/dataframe/frames.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vZGF0YWZyYW1lL2ZyYW1lcy5weQ==) | `95.21% <98.14%> (+0.07%)` | :arrow_up: |
   | [...nternal/clients/cloudbuild/cloudbuild\_v1\_client.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9kYXRhZmxvdy9pbnRlcm5hbC9jbGllbnRzL2Nsb3VkYnVpbGQvY2xvdWRidWlsZF92MV9jbGllbnQucHk=) | `52.21% <0.00%> (-2.97%)` | :arrow_down: |
   | [sdks/python/apache\_beam/utils/interactive\_utils.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vdXRpbHMvaW50ZXJhY3RpdmVfdXRpbHMucHk=) | `92.68% <0.00%> (-2.44%)` | :arrow_down: |
   | [.../python/apache\_beam/transforms/periodicsequence.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vdHJhbnNmb3Jtcy9wZXJpb2RpY3NlcXVlbmNlLnB5) | `96.72% <0.00%> (-1.64%)` | :arrow_down: |
   | [...thon/apache\_beam/runners/worker/sdk\_worker\_main.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy93b3JrZXIvc2RrX3dvcmtlcl9tYWluLnB5) | `74.12% <0.00%> (-1.40%)` | :arrow_down: |
   | [...\_beam/runners/portability/sdk\_container\_builder.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9wb3J0YWJpbGl0eS9zZGtfY29udGFpbmVyX2J1aWxkZXIucHk=) | `37.93% <0.00%> (-1.02%)` | :arrow_down: |
   | [...pache\_beam/runners/interactive/interactive\_beam.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9pbnRlcmFjdGl2ZS9pbnRlcmFjdGl2ZV9iZWFtLnB5) | `80.62% <0.00%> (-0.78%)` | :arrow_down: |
   | [...che\_beam/runners/interactive/interactive\_runner.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9pbnRlcmFjdGl2ZS9pbnRlcmFjdGl2ZV9ydW5uZXIucHk=) | `92.02% <0.00%> (-0.73%)` | :arrow_down: |
   | [sdks/python/apache\_beam/runners/direct/executor.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9kaXJlY3QvZXhlY3V0b3IucHk=) | `96.46% <0.00%> (-0.55%)` | :arrow_down: |
   | [...runners/interactive/display/pcoll\_visualization.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9pbnRlcmFjdGl2ZS9kaXNwbGF5L3Bjb2xsX3Zpc3VhbGl6YXRpb24ucHk=) | `85.85% <0.00%> (-0.51%)` | :arrow_down: |
   | ... and [15 more](https://codecov.io/gh/apache/beam/pull/17043/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [58f8bef...0e046fc](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] yeandy commented on a change in pull request #17043: [BEAM-13966] Add pivot(), a non-deferred column operation on categorical columns

Posted by GitBox <gi...@apache.org>.
yeandy commented on a change in pull request #17043:
URL: https://github.com/apache/beam/pull/17043#discussion_r837878672



##########
File path: sdks/python/apache_beam/dataframe/frames.py
##########
@@ -3655,6 +3656,121 @@ def shift(self, axis, freq, **kwargs):
   describe = _agg_method(pd.DataFrame, 'describe')
   max = _agg_method(pd.DataFrame, 'max')
   min = _agg_method(pd.DataFrame, 'min')
+
+  @frame_base.with_docs_from(pd.DataFrame)
+  @frame_base.args_to_kwargs(pd.DataFrame)
+  @frame_base.populate_defaults(pd.DataFrame)
+  def pivot(self, index=None, columns=None, values=None, **kwargs):
+    def verify_all_categorical(all_cols_are_categorical):
+      if not all_cols_are_categorical:
+        raise frame_base.WontImplementError(
+            "pivot() of non-categorical type is not supported because "
+            "the type of the output column depends on the data. Please use "
+            "pd.CategoricalDtype with explicit categories.",
+            reason="non-deferred-columns")
+
+    # Construct column index
+    if is_list_like(columns) and len(columns) <= 1:
+      columns = columns[0]
+    selected_cols = self._expr.proxy()[columns]
+    if isinstance(selected_cols, pd.Series):
+      all_cols_are_categorical = isinstance(
+        selected_cols.dtype, pd.CategoricalDtype
+      )
+      verify_all_categorical(all_cols_are_categorical)
+
+      # If values not provided, take all remaining columns of dataframe
+      if not values:
+        values = self._expr.proxy() \
+          .drop(index, axis=1).drop(columns, axis=1).columns.values
+
+      # Take the provided values
+      if is_list_like(values) and len(values) > 1:
+        values_in_col_index = values
+        names = [None, columns]
+        col_index = pd.MultiIndex.from_product(
+          [values_in_col_index,
+          selected_cols.dtypes.categories.astype('category')],
+          names=names
+        )
+      else:
+        col_index = pd.CategoricalIndex(
+          selected_cols.dtype.categories,
+          name=columns
+        )
+    else:
+      all_cols_are_categorical = all(
+        isinstance(c, pd.CategoricalDtype) for c in selected_cols.dtypes
+      )
+      verify_all_categorical(all_cols_are_categorical)
+
+      categories = [
+        c.categories.astype('category') for c in selected_cols.dtypes
+      ]
+      if is_list_like(columns) and len(columns) > 1:
+        col_index = pd.MultiIndex.from_product(categories, names=columns)
+      else:
+        col_index = pd.CategoricalIndex(
+            selected_cols.dtype.categories,
+            name=columns
+        )
+
+    # Construct row index
+    if index:
+      per_partition = expressions.ComputedExpression(
+          'pivot-per-partition',
+          lambda df: df.set_index(keys=index), [self._expr],
+          preserves_partition_by=partitionings.Singleton(),
+          requires_partition_by=partitionings.Arbitrary()
+      )
+      if is_list_like(index):

Review comment:
       I've updated the logic to extract the index from a temporary df created from calling `pivot` from the current proxy. Thus we don't have to do this check if `index` is a list or not.




-- 
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] edited a comment on pull request #17043: [BEAM-13966] Add pivot(), a non-deferred column operation on categorical columns

Posted by GitBox <gi...@apache.org>.
codecov[bot] edited a comment on pull request #17043:
URL: https://github.com/apache/beam/pull/17043#issuecomment-1062140334


   # [Codecov](https://codecov.io/gh/apache/beam/pull/17043?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 [#17043](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (0e046fc) into [master](https://codecov.io/gh/apache/beam/commit/58f8bef66f4da09202b2de9dc3166c7fa1f4fa4e?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (58f8bef) will **increase** coverage by `0.06%`.
   > The diff coverage is `97.95%`.
   
   ```diff
   @@            Coverage Diff             @@
   ##           master   #17043      +/-   ##
   ==========================================
   + Coverage   73.83%   73.89%   +0.06%     
   ==========================================
     Files         667      667              
     Lines       87392    88094     +702     
   ==========================================
   + Hits        64523    65101     +578     
   - Misses      21766    21890     +124     
     Partials     1103     1103              
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | python | `83.63% <97.95%> (-0.02%)` | :arrow_down: |
   
   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/17043?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [sdks/python/apache\_beam/dataframe/frames.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vZGF0YWZyYW1lL2ZyYW1lcy5weQ==) | `95.20% <97.95%> (+0.06%)` | :arrow_up: |
   | [.../python/apache\_beam/testing/test\_stream\_service.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vdGVzdGluZy90ZXN0X3N0cmVhbV9zZXJ2aWNlLnB5) | `88.37% <0.00%> (-4.66%)` | :arrow_down: |
   | [...nternal/clients/cloudbuild/cloudbuild\_v1\_client.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9kYXRhZmxvdy9pbnRlcm5hbC9jbGllbnRzL2Nsb3VkYnVpbGQvY2xvdWRidWlsZF92MV9jbGllbnQucHk=) | `52.21% <0.00%> (-2.97%)` | :arrow_down: |
   | [sdks/python/apache\_beam/utils/interactive\_utils.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vdXRpbHMvaW50ZXJhY3RpdmVfdXRpbHMucHk=) | `92.68% <0.00%> (-2.44%)` | :arrow_down: |
   | [...thon/apache\_beam/runners/worker/sdk\_worker\_main.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy93b3JrZXIvc2RrX3dvcmtlcl9tYWluLnB5) | `74.12% <0.00%> (-1.40%)` | :arrow_down: |
   | [...che\_beam/runners/interactive/interactive\_runner.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9pbnRlcmFjdGl2ZS9pbnRlcmFjdGl2ZV9ydW5uZXIucHk=) | `91.42% <0.00%> (-1.33%)` | :arrow_down: |
   | [...\_beam/runners/portability/sdk\_container\_builder.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9wb3J0YWJpbGl0eS9zZGtfY29udGFpbmVyX2J1aWxkZXIucHk=) | `37.93% <0.00%> (-1.02%)` | :arrow_down: |
   | [...n/apache\_beam/runners/interactive/cache\_manager.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9pbnRlcmFjdGl2ZS9jYWNoZV9tYW5hZ2VyLnB5) | `87.58% <0.00%> (-1.01%)` | :arrow_down: |
   | [...pache\_beam/runners/interactive/interactive\_beam.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9pbnRlcmFjdGl2ZS9pbnRlcmFjdGl2ZV9iZWFtLnB5) | `80.62% <0.00%> (-0.78%)` | :arrow_down: |
   | [sdks/python/apache\_beam/runners/direct/executor.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9kaXJlY3QvZXhlY3V0b3IucHk=) | `96.46% <0.00%> (-0.55%)` | :arrow_down: |
   | ... and [28 more](https://codecov.io/gh/apache/beam/pull/17043/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [58f8bef...0e046fc](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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] yeandy commented on pull request #17043: [BEAM-13966] Add pivot(), a non-deferred column operation on categorical columns

Posted by GitBox <gi...@apache.org>.
yeandy commented on pull request #17043:
URL: https://github.com/apache/beam/pull/17043#issuecomment-1083112921


   Run Python PreCommit


-- 
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] edited a comment on pull request #17043: [BEAM-13966] Add pivot(), a non-deferred column operation on categorical columns

Posted by GitBox <gi...@apache.org>.
codecov[bot] edited a comment on pull request #17043:
URL: https://github.com/apache/beam/pull/17043#issuecomment-1062140334


   # [Codecov](https://codecov.io/gh/apache/beam/pull/17043?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 [#17043](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (5c6fd0a) into [master](https://codecov.io/gh/apache/beam/commit/58f8bef66f4da09202b2de9dc3166c7fa1f4fa4e?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (58f8bef) will **increase** coverage by `0.07%`.
   > The diff coverage is `98.14%`.
   
   ```diff
   @@            Coverage Diff             @@
   ##           master   #17043      +/-   ##
   ==========================================
   + Coverage   73.83%   73.90%   +0.07%     
   ==========================================
     Files         667      667              
     Lines       87392    88071     +679     
   ==========================================
   + Hits        64523    65089     +566     
   - Misses      21766    21879     +113     
     Partials     1103     1103              
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | python | `83.64% <98.14%> (-0.01%)` | :arrow_down: |
   
   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/17043?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [sdks/python/apache\_beam/dataframe/frames.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vZGF0YWZyYW1lL2ZyYW1lcy5weQ==) | `95.21% <98.14%> (+0.07%)` | :arrow_up: |
   | [...nternal/clients/cloudbuild/cloudbuild\_v1\_client.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9kYXRhZmxvdy9pbnRlcm5hbC9jbGllbnRzL2Nsb3VkYnVpbGQvY2xvdWRidWlsZF92MV9jbGllbnQucHk=) | `52.21% <0.00%> (-2.97%)` | :arrow_down: |
   | [sdks/python/apache\_beam/utils/interactive\_utils.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vdXRpbHMvaW50ZXJhY3RpdmVfdXRpbHMucHk=) | `92.68% <0.00%> (-2.44%)` | :arrow_down: |
   | [...thon/apache\_beam/runners/worker/sdk\_worker\_main.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy93b3JrZXIvc2RrX3dvcmtlcl9tYWluLnB5) | `74.12% <0.00%> (-1.40%)` | :arrow_down: |
   | [...\_beam/runners/portability/sdk\_container\_builder.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9wb3J0YWJpbGl0eS9zZGtfY29udGFpbmVyX2J1aWxkZXIucHk=) | `37.93% <0.00%> (-1.02%)` | :arrow_down: |
   | [sdks/python/apache\_beam/internal/metrics/metric.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vaW50ZXJuYWwvbWV0cmljcy9tZXRyaWMucHk=) | `90.00% <0.00%> (-1.00%)` | :arrow_down: |
   | [...pache\_beam/runners/interactive/interactive\_beam.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9pbnRlcmFjdGl2ZS9pbnRlcmFjdGl2ZV9iZWFtLnB5) | `80.62% <0.00%> (-0.78%)` | :arrow_down: |
   | [...che\_beam/runners/interactive/interactive\_runner.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9pbnRlcmFjdGl2ZS9pbnRlcmFjdGl2ZV9ydW5uZXIucHk=) | `92.02% <0.00%> (-0.73%)` | :arrow_down: |
   | [sdks/python/apache\_beam/runners/direct/executor.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9kaXJlY3QvZXhlY3V0b3IucHk=) | `96.46% <0.00%> (-0.55%)` | :arrow_down: |
   | [...runners/interactive/display/pcoll\_visualization.py](https://codecov.io/gh/apache/beam/pull/17043/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-c2Rrcy9weXRob24vYXBhY2hlX2JlYW0vcnVubmVycy9pbnRlcmFjdGl2ZS9kaXNwbGF5L3Bjb2xsX3Zpc3VhbGl6YXRpb24ucHk=) | `85.85% <0.00%> (-0.51%)` | :arrow_down: |
   | ... and [15 more](https://codecov.io/gh/apache/beam/pull/17043/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) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [58f8bef...5c6fd0a](https://codecov.io/gh/apache/beam/pull/17043?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?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