[
https://issues.apache.org/jira/browse/BEAM-7018?focusedWorklogId=282129&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-282129
]
ASF GitHub Bot logged work on BEAM-7018:
----------------------------------------
Author: ASF GitHub Bot
Created on: 24/Jul/19 16:30
Start Date: 24/Jul/19 16:30
Worklog Time Spent: 10m
Work Description: mszb commented on pull request #8859: [BEAM-7018] Added
Regex transform for PythonSDK
URL: https://github.com/apache/beam/pull/8859#discussion_r306906519
##########
File path: sdks/python/apache_beam/transforms/util.py
##########
@@ -864,3 +867,225 @@ def add_window_info(element,
timestamp=DoFn.TimestampParam,
def expand(self, pcoll):
return pcoll | ParDo(self.add_window_info)
+
+
+class Regex(object):
+ """
+ PTransform to use Regular Expression to process the elements in a
+ PCollection.
+ """
+ @staticmethod
+ def _regex_compile(regex):
+ """Return re.compile if the regex has a string value"""
+ if isinstance(regex, str):
+ regex = re.compile(regex)
+ return regex
+
+ @staticmethod
+ @ptransform_fn
+ def _matches_all_object(pcoll, regex, group=None):
+ """
+ PTransform to find the matches as per the regex.
+ """
+ def _process(element):
+ m = regex.finditer(element)
+ results = list()
+ for _, match in enumerate(m, start=1):
+ results.append(match.group())
+ for groupNum in range(0, len(match.groups())):
+ results.append(match.group(groupNum + 1))
+ if results:
+ yield results
+ return pcoll | FlatMap(_process)
+
+ @staticmethod
+ @ptransform_fn
+ def _matches_kv_object(pcoll, regex, keyGroup, valueGroup):
+ """
+ PTransfrom to find the matches as per regex and return as a KV pair.
+ """
+ def _process(element):
+ match = regex.match(element)
+ if match:
+ yield (match.group(keyGroup), match.group(valueGroup))
+
+ return pcoll | FlatMap(_process)
+
+ @staticmethod
+ @typehints.with_input_types(str)
+ @typehints.with_output_types(str)
+ @ptransform_fn
+ def matches(pcoll, regex, group=None):
+ """
+ Returns the matches if the entire line matched the Regex. Returns the
+ entire line (group 0 by default). Group can be integer value and a string
+ value.
+
+ Args:
+ regex: the regular expression string or (re.compile) pattern.
+ group: (optional) name of the group, it can be integer or a string value.
+ """
+ regex = Regex._regex_compile(regex)
+ group = group or 0
+
+ def _process(element):
+ m = regex.match(element)
+ if m:
+ yield m.group(group)
+ return pcoll | FlatMap(_process)
+
+ @staticmethod
+ @typehints.with_input_types(str)
+ @typehints.with_output_types(str)
+ @ptransform_fn
+ def all_matches(pcoll, regex):
+ """
+ Returns the matches if the entire line matches the Regex. Returns all
+ groups.
+
+ Args:
+ regex: the regular expression string or (re.compile) pattern.
+ """
+ regex = Regex._regex_compile(regex)
+ return pcoll | Regex._matches_all_object(regex=regex)
+
+ @staticmethod
+ @typehints.with_input_types(str)
+ @typehints.with_output_types(typehints.KV[K, V])
+ @ptransform_fn
+ def matches_kv(pcoll, regex, keyGroup, valueGroup):
+ """
+ Returns the matches if the entire line matches the Regex. Returns the
+ specified groups as the key and value pair.
+
+ Args:
+ regex: the regular expression string or (re.compile) pattern.
+ keyGroup: The Regex group to use as the key. Can be int or str.
+ valueGroup: The Regex group to use the value. Can be int or str.
+ """
+ regex = Regex._regex_compile(regex)
+ return pcoll | Regex._matches_kv_object(regex=regex, keyGroup=keyGroup,
+ valueGroup=valueGroup)
+
+ @staticmethod
+ @typehints.with_input_types(str)
+ @typehints.with_output_types(str)
+ @ptransform_fn
+ def find(pcoll, regex, group=None):
+ """
+ Returns the matches if a portion of the line matches the Regex. Returns
+ the entire line (group 0 by default). Group can be integer value and a
+ string value.
+
+ Args:
+ regex: the regular expression string or (re.compile) pattern.
+ group: (optional) name of the group, it can be integer or a string value.
+ """
+ regex = Regex._regex_compile(regex)
+ group = group or 0
+
+ def _process(element):
+ r = regex.search(element)
+ if r:
+ yield r.group(group)
+ return pcoll | FlatMap(_process)
+
+ @staticmethod
+ @typehints.with_input_types(str)
+ @typehints.with_output_types(str)
+ @ptransform_fn
+ def find_all(pcoll, regex):
Review comment:
So what I understand from your feedback is:
1. for the 'Regex.find_all()', we need to change `re.finditer` to
`re.findall` and return the result as a list.
2. create a new method `Regex.finditer()` which returns "match objects"
instances..
The issue with the 'MatchObject' is we can not pickle it.
Showing error `PicklingError: Can't pickle <built-in method match of
_sre.SRE_Pattern`
So either we can make a custom match object class and return it or uses KV
pairs? Where K as the string of group and V as a tuple containing all groups?
For example:
lets say sting is "abb ax abbb" & regex expression is "a(b*)", so the
results would be like:
`[
KV("abb", ('bb',)),
KV("a", ('',)),
KV("abbb", ('bbb',)
]`
or maybe we can include group 0 in the values as well.
`[
KV("abb", ('abb', 'bb',)),
KV("a", ('ab', '')),
KV("abbb", ('abbb', 'bbb')
]`
Your thoughts?
----------------------------------------------------------------
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.
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 282129)
Time Spent: 9.5h (was: 9h 20m)
> Regex transform for Python SDK
> ------------------------------
>
> Key: BEAM-7018
> URL: https://issues.apache.org/jira/browse/BEAM-7018
> Project: Beam
> Issue Type: New Feature
> Components: sdk-py-core
> Reporter: Rose Nguyen
> Assignee: Shehzaad Nakhoda
> Priority: Minor
> Time Spent: 9.5h
> Remaining Estimate: 0h
>
> PTransorms to use Regular Expressions to process elements in a PCollection
> It should offer the same API as its Java counterpart:
> [https://github.com/apache/beam/blob/11a977b8b26eff2274d706541127c19dc93131a2/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Regex.java]
--
This message was sent by Atlassian JIRA
(v7.6.14#76016)