-
Notifications
You must be signed in to change notification settings - Fork 4.5k
[BEAM-13947] Add split() and rsplit(), non-deferred column operations on categorical columns #16677
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 1 commit
6b382a3
5f6123b
0437851
43ff9b8
20fda68
a08094c
c64e42c
92df15a
8e29388
f094db7
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
- Loading branch information
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -4721,13 +4721,84 @@ 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') | ||
| # TODO: Add for pandas 1.4.0 | ||
| # def _split_helper( | ||
| # self, | ||
| # rsplit=False, | ||
| # pat=None, | ||
| # expand=False, | ||
| # regex=None, | ||
| # **kwargs | ||
| # ): | ||
| def _split_helper(self, rsplit=False, pat=None, expand=False, **kwargs): | ||
| # Not creating separate columns | ||
| if not expand: | ||
| proxy = self._expr.proxy() | ||
| else: | ||
| # Creating separate columns, so data type is more strict | ||
| 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') | ||
| split_cats = [ | ||
| cat.split(sep=kwargs.get('pat'), maxsplit=kwargs.get('n', -1)) | ||
| for cat in dtype.categories | ||
| ] | ||
|
|
||
| # TODO: Replace for pandas 1.4.0 | ||
| # Treat pat as literal string | ||
| # if not regex or (regex is None and len(pat) == 1): | ||
| # split_cats = [ | ||
| # cat.split( | ||
| # sep=kwargs.get('pat'), | ||
| # maxsplit=kwargs.get('n', -1) | ||
| # ) for cat in dtype.categories | ||
| # ] | ||
| # Treat pat as regex | ||
| # else: | ||
| # 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: proxy.combine_first( | ||
| (s.str.split(pat=pat, expand=expand, **kwargs) | ||
| if not rsplit else s.str.rsplit(pat=pat, expand=expand, **kwargs)) | ||
| ) | ||
|
|
||
| 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) | ||
yeandy marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| # TODO: Add for pandas 1.4.0 | ||
| # def split(self, pat=None, expand=False, regex=None, **kwargs): | ||
| def split(self, pat=None, expand=False, **kwargs): | ||
| return self._split_helper(rsplit=False, pat=pat, expand=expand, **kwargs) | ||
|
|
||
| @frame_base.with_docs_from(pd.core.strings.StringMethods) | ||
| @frame_base.args_to_kwargs(pd.core.strings.StringMethods) | ||
| # TODO: Add for pandas 1.4.0 | ||
| # def rsplit(self, pat=None, expand=False, regex=None, **kwargs): | ||
| def rsplit(self, pat=None, expand=False, **kwargs): | ||
|
||
| return self._split_helper(rsplit=True, pat=pat, expand=expand, **kwargs) | ||
|
|
||
|
|
||
| ELEMENTWISE_STRING_METHODS = [ | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.