You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@beam.apache.org by "ASF GitHub Bot (Jira)" <ji...@apache.org> on 2022/03/03 02:29:00 UTC

[jira] [Work logged] (BEAM-13947) Implement Series.str.split() and Series.str.rsplit() for DataFrame API

     [ https://issues.apache.org/jira/browse/BEAM-13947?focusedWorklogId=735737&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-735737 ]

ASF GitHub Bot logged work on BEAM-13947:
-----------------------------------------

                Author: ASF GitHub Bot
            Created on: 03/Mar/22 02:28
            Start Date: 03/Mar/22 02:28
    Worklog Time Spent: 10m 
      Work Description: TheNeuralBit commented on a change in pull request #16677:
URL: https://github.com/apache/beam/pull/16677#discussion_r818260116



##########
File path: sdks/python/apache_beam/dataframe/frames.py
##########
@@ -4721,13 +4721,91 @@ def repeat(self, repeats):
       pd.core.strings.StringMethods, 'get_dummies',
       reason='non-deferred-columns')
 
-  split = frame_base.wont_implement_method(
-      pd.core.strings.StringMethods, 'split',
-      reason='non-deferred-columns')
+  def _split_helper(
+    self, rsplit=False, pat=None, expand=False, regex=None, **kwargs):
+
+    # Adding arguments to kwargs. regex introduced in pandas 1.4
+    # but only for split, not rsplit
+    kwargs['pat'] = pat
+    kwargs['expand'] = expand
+    if PD_VERSION >= (1, 4) and not rsplit:
+      kwargs['regex'] = regex

Review comment:
       Odd that `regex` is in `split` only, that's frustrating.
   
   It _might_ be cleaner to just pass through all the arguments via `**kwargs`, and pull out the ones you need to branch on.

##########
File path: sdks/python/apache_beam/dataframe/frames.py
##########
@@ -4721,13 +4721,91 @@ def repeat(self, repeats):
       pd.core.strings.StringMethods, 'get_dummies',
       reason='non-deferred-columns')
 
-  split = frame_base.wont_implement_method(
-      pd.core.strings.StringMethods, 'split',
-      reason='non-deferred-columns')
+  def _split_helper(
+    self, rsplit=False, pat=None, expand=False, regex=None, **kwargs):
+
+    # Adding arguments to kwargs. regex introduced in pandas 1.4
+    # but only for split, not rsplit
+    kwargs['pat'] = pat
+    kwargs['expand'] = expand
+    if PD_VERSION >= (1, 4) and not rsplit:
+      kwargs['regex'] = regex
+
+    if not expand:
+      # Not creating separate columns
+      proxy = self._expr.proxy()
+      func = lambda s: pd.concat([proxy,
+        (s.str.split(**kwargs) if not rsplit else s.str.rsplit(**kwargs))]

Review comment:
       could you branch on `rsplit` outside of the function definition?

##########
File path: sdks/python/apache_beam/dataframe/frames.py
##########
@@ -4721,13 +4721,91 @@ def repeat(self, repeats):
       pd.core.strings.StringMethods, 'get_dummies',
       reason='non-deferred-columns')
 
-  split = frame_base.wont_implement_method(
-      pd.core.strings.StringMethods, 'split',
-      reason='non-deferred-columns')
+  def _split_helper(
+    self, rsplit=False, pat=None, expand=False, regex=None, **kwargs):
+
+    # Adding arguments to kwargs. regex introduced in pandas 1.4
+    # but only for split, not rsplit
+    kwargs['pat'] = pat
+    kwargs['expand'] = expand
+    if PD_VERSION >= (1, 4) and not rsplit:
+      kwargs['regex'] = regex
+
+    if not expand:
+      # Not creating separate columns
+      proxy = self._expr.proxy()
+      func = lambda s: pd.concat([proxy,
+        (s.str.split(**kwargs) if not rsplit else s.str.rsplit(**kwargs))]
+      )
+    else:
+      # Creating separate columns, so must be more strict on dtype
+      dtype = self._expr.proxy().dtype
+      if not isinstance(dtype, pd.CategoricalDtype):
+        method_name = 'rsplit' if rsplit else 'split'
+        raise frame_base.WontImplementError(
+            method_name + "() of non-categorical type is not supported because "

Review comment:
       nit: I would use an f-string here, this is fine too though.
   ```suggestion
               f"{method_name}() of non-categorical type is not supported because "
   ```

##########
File path: sdks/python/apache_beam/dataframe/frames.py
##########
@@ -4721,13 +4721,82 @@ def repeat(self, repeats):
       pd.core.strings.StringMethods, 'get_dummies',
       reason='non-deferred-columns')
 
-  split = frame_base.wont_implement_method(
-      pd.core.strings.StringMethods, 'split',
-      reason='non-deferred-columns')
+  def _split_helper(
+    self, rsplit=False, pat=None, expand=False, regex=None, **kwargs):
+    if not expand:
+      # Not creating separate columns
+      proxy = self._expr.proxy()
+      func = lambda s: pd.concat([proxy,
+        (s.str.split(pat=pat, expand=expand, regex=regex, **kwargs)
+        if not rsplit else s.str.rsplit(pat=pat, expand=expand, **kwargs))]
+      )
+    else:
+      # Creating separate columns, so must be more strict on dtype
+      dtype = self._expr.proxy().dtype
+      if not isinstance(dtype, pd.CategoricalDtype):
+        method_name = 'rsplit' if rsplit else 'split'
+        raise frame_base.WontImplementError(
+            method_name + "() 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")
 
-  rsplit = frame_base.wont_implement_method(
-      pd.core.strings.StringMethods, 'rsplit',
-      reason='non-deferred-columns')
+      if regex is False or (
+        regex is None and isinstance(pat, str) and len(pat) == 1):
+        # Treat pat as literal string
+        split_cats = [
+          cat.split(
+            sep=kwargs.get('pat'),
+            maxsplit=kwargs.get('n', -1)
+          ) for cat in dtype.categories
+        ]
+      else:
+        # Treat pat as regex
+        split_cats = [
+          re.split(
+            pattern=pat,
+            string=cat,
+            maxsplit=kwargs.get('n', 0)
+          ) for cat in dtype.categories
+        ]
+
+      max_splits = len(max(split_cats, key=len))
+      proxy = pd.DataFrame(columns=range(max_splits))
+
+      func = lambda s: pd.concat([proxy,
+        (s.str.split(pat=pat, expand=expand, regex=regex, **kwargs)
+        if not rsplit else s.str.rsplit(pat=pat, expand=expand, **kwargs))]
+      ).replace(np.nan, value=None)

Review comment:
       yeah this should be doable, I'm not sure if it can be done in a one-liner though. You could define a multi-line function, something like
   
   ```py
   def func(s):
     result = s.str.split(**kwargs)
     s[~s.isna()].str.replace(np.nan, value=None, inplace=True)
     return result
   ```
   
   Making it multi-line might improve readability anyway

##########
File path: sdks/python/apache_beam/dataframe/frames.py
##########
@@ -4721,13 +4721,91 @@ def repeat(self, repeats):
       pd.core.strings.StringMethods, 'get_dummies',
       reason='non-deferred-columns')
 
-  split = frame_base.wont_implement_method(
-      pd.core.strings.StringMethods, 'split',
-      reason='non-deferred-columns')
+  def _split_helper(
+    self, rsplit=False, pat=None, expand=False, regex=None, **kwargs):
+
+    # Adding arguments to kwargs. regex introduced in pandas 1.4
+    # but only for split, not rsplit
+    kwargs['pat'] = pat
+    kwargs['expand'] = expand
+    if PD_VERSION >= (1, 4) and not rsplit:
+      kwargs['regex'] = regex
+
+    if not expand:
+      # Not creating separate columns
+      proxy = self._expr.proxy()
+      func = lambda s: pd.concat([proxy,
+        (s.str.split(**kwargs) if not rsplit else s.str.rsplit(**kwargs))]
+      )
+    else:
+      # Creating separate columns, so must be more strict on dtype
+      dtype = self._expr.proxy().dtype
+      if not isinstance(dtype, pd.CategoricalDtype):
+        method_name = 'rsplit' if rsplit else 'split'
+        raise frame_base.WontImplementError(
+            method_name + "() 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")
 
-  rsplit = frame_base.wont_implement_method(
-      pd.core.strings.StringMethods, 'rsplit',
-      reason='non-deferred-columns')
+      if regex is False or (
+        regex is None and (
+          (not pat) or (isinstance(pat, str) and len(pat) == 1)
+        )
+      ):
+        # Treat pat as literal string
+        split_cats = [
+          cat.split(
+            sep=kwargs.get('pat'),
+            maxsplit=kwargs.get('n', -1)
+          ) for cat in dtype.categories
+        ]
+      else:
+        # Treat pat as regex
+        split_cats = [
+          re.split(
+            pattern=pat,
+            string=cat,
+            maxsplit=kwargs.get('n', 0)
+          ) for cat in dtype.categories
+        ]
+
+      max_splits = len(max(split_cats, key=len))
+      proxy = pd.DataFrame(columns=range(max_splits))
+
+      func = lambda s: pd.concat([proxy,
+        (s.str.split(**kwargs) if not rsplit else s.str.rsplit(**kwargs))]
+      ).replace(np.nan, value=None)
+
+    return frame_base.DeferredFrame.wrap(
+        expressions.ComputedExpression(
+            'split',
+            func,
+            [self._expr],
+            proxy=proxy,
+            requires_partition_by=partitionings.Arbitrary(),
+            preserves_partition_by=partitionings.Arbitrary()))
+
+  @frame_base.with_docs_from(pd.core.strings.StringMethods)
+  @frame_base.args_to_kwargs(pd.core.strings.StringMethods)

Review comment:
       Could you use populate_defaults here?




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


Issue Time Tracking
-------------------

    Worklog Id:     (was: 735737)
    Time Spent: 5.5h  (was: 5h 20m)

> Implement Series.str.split() and Series.str.rsplit() for DataFrame API
> ----------------------------------------------------------------------
>
>                 Key: BEAM-13947
>                 URL: https://issues.apache.org/jira/browse/BEAM-13947
>             Project: Beam
>          Issue Type: Sub-task
>          Components: dsl-dataframe, sdk-py-core
>            Reporter: Andy Ye
>            Assignee: Andy Ye
>            Priority: P3
>              Labels: dataframe-api
>          Time Spent: 5.5h
>  Remaining Estimate: 0h
>




--
This message was sent by Atlassian Jira
(v8.20.1#820001)