aboutsummaryrefslogtreecommitdiff
path: root/python
diff options
context:
space:
mode:
authorCheng Hao <hao.cheng@intel.com>2015-07-21 00:48:07 -0700
committerDavies Liu <davies.liu@gmail.com>2015-07-21 00:48:07 -0700
commit8c8f0ef59e12b6f13d5a0bf2d7bf1248b5c1e369 (patch)
treef4a7c28f662757bc341407642c7f90357f1d4b79 /python
parentd38c5029a2ca845e2782096044a6412b653c9f95 (diff)
downloadspark-8c8f0ef59e12b6f13d5a0bf2d7bf1248b5c1e369.tar.gz
spark-8c8f0ef59e12b6f13d5a0bf2d7bf1248b5c1e369.tar.bz2
spark-8c8f0ef59e12b6f13d5a0bf2d7bf1248b5c1e369.zip
[SPARK-8255] [SPARK-8256] [SQL] Add regex_extract/regex_replace
Add expressions `regex_extract` & `regex_replace` Author: Cheng Hao <hao.cheng@intel.com> Closes #7468 from chenghao-intel/regexp and squashes the following commits: e5ea476 [Cheng Hao] minor update for documentation ef96fd6 [Cheng Hao] update the code gen 72cf28f [Cheng Hao] Add more log for compilation error 4e11381 [Cheng Hao] Add regexp_replace / regexp_extract support
Diffstat (limited to 'python')
-rw-r--r--python/pyspark/sql/functions.py30
1 files changed, 30 insertions, 0 deletions
diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py
index 031745a1c4..3c134faa0a 100644
--- a/python/pyspark/sql/functions.py
+++ b/python/pyspark/sql/functions.py
@@ -46,6 +46,8 @@ __all__ = [
'monotonicallyIncreasingId',
'rand',
'randn',
+ 'regexp_extract',
+ 'regexp_replace',
'sha1',
'sha2',
'sparkPartitionId',
@@ -345,6 +347,34 @@ def levenshtein(left, right):
@ignore_unicode_prefix
@since(1.5)
+def regexp_extract(str, pattern, idx):
+ """Extract a specific(idx) group identified by a java regex, from the specified string column.
+
+ >>> df = sqlContext.createDataFrame([('100-200',)], ['str'])
+ >>> df.select(regexp_extract('str', '(\d+)-(\d+)', 1).alias('d')).collect()
+ [Row(d=u'100')]
+ """
+ sc = SparkContext._active_spark_context
+ jc = sc._jvm.functions.regexp_extract(_to_java_column(str), pattern, idx)
+ return Column(jc)
+
+
+@ignore_unicode_prefix
+@since(1.5)
+def regexp_replace(str, pattern, replacement):
+ """Replace all substrings of the specified string value that match regexp with rep.
+
+ >>> df = sqlContext.createDataFrame([('100-200',)], ['str'])
+ >>> df.select(regexp_replace('str', '(\\d+)', '##').alias('d')).collect()
+ [Row(d=u'##-##')]
+ """
+ sc = SparkContext._active_spark_context
+ jc = sc._jvm.functions.regexp_replace(_to_java_column(str), pattern, replacement)
+ return Column(jc)
+
+
+@ignore_unicode_prefix
+@since(1.5)
def md5(col):
"""Calculates the MD5 digest and returns the value as a 32 character hex string.