spark-instrumented-optimizer/python/pyspark/sql/functions.py

3229 lines
133 KiB
Python
Raw Normal View History

[SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames - The old implicit would convert RDDs directly to DataFrames, and that added too many methods. - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed Python changes: - toDataFrame -> toDF - Dsl -> functions package - addColumn -> withColumn - renameColumn -> withColumnRenamed - add toDF functions to RDD on SQLContext init - add flatMap to DataFrame Author: Reynold Xin <rxin@databricks.com> Author: Davies Liu <davies@databricks.com> Closes #4556 from rxin/SPARK-5752 and squashes the following commits: 5ef9910 [Reynold Xin] More fix 61d3fca [Reynold Xin] Merge branch 'df5' of github.com:davies/spark into SPARK-5752 ff5832c [Reynold Xin] Fix python 749c675 [Reynold Xin] count(*) fixes. 5806df0 [Reynold Xin] Fix build break again. d941f3d [Reynold Xin] Fixed explode compilation break. fe1267a [Davies Liu] flatMap c4afb8e [Reynold Xin] style d9de47f [Davies Liu] add comment b783994 [Davies Liu] add comment for toDF e2154e5 [Davies Liu] schema() -> schema 3a1004f [Davies Liu] Dsl -> functions, toDF() fb256af [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 0dd74eb [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames 97dd47c [Davies Liu] fix mistake 6168f74 [Davies Liu] fix test 1fc0199 [Davies Liu] fix test a075cd5 [Davies Liu] clean up, toPandas 663d314 [Davies Liu] add test for agg('*') 9e214d5 [Reynold Xin] count(*) fixes. 1ed7136 [Reynold Xin] Fix build break again. 921b2e3 [Reynold Xin] Fixed explode compilation break. 14698d4 [Davies Liu] flatMap ba3e12d [Reynold Xin] style d08c92d [Davies Liu] add comment 5c8b524 [Davies Liu] add comment for toDF a4e5e66 [Davies Liu] schema() -> schema d377fc9 [Davies Liu] Dsl -> functions, toDF() 6b3086c [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 807e8b1 [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames
2015-02-14 02:03:22 -05:00
#
# Licensed to the Apache Software Foundation (ASF) under one or more
# contributor license agreements. See the NOTICE file distributed with
# this work for additional information regarding copyright ownership.
# The ASF licenses this file to You under the Apache License, Version 2.0
# (the "License"); you may not use this file except in compliance with
# the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
#
"""
A collections of builtin functions
"""
[SPARK-4897] [PySpark] Python 3 support This PR update PySpark to support Python 3 (tested with 3.4). Known issue: unpickle array from Pyrolite is broken in Python 3, those tests are skipped. TODO: ec2/spark-ec2.py is not fully tested with python3. Author: Davies Liu <davies@databricks.com> Author: twneale <twneale@gmail.com> Author: Josh Rosen <joshrosen@databricks.com> Closes #5173 from davies/python3 and squashes the following commits: d7d6323 [Davies Liu] fix tests 6c52a98 [Davies Liu] fix mllib test 99e334f [Davies Liu] update timeout b716610 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 cafd5ec [Davies Liu] adddress comments from @mengxr bf225d7 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 179fc8d [Davies Liu] tuning flaky tests 8c8b957 [Davies Liu] fix ResourceWarning in Python 3 5c57c95 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 4006829 [Davies Liu] fix test 2fc0066 [Davies Liu] add python3 path 71535e9 [Davies Liu] fix xrange and divide 5a55ab4 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 125f12c [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 ed498c8 [Davies Liu] fix compatibility with python 3 820e649 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 e8ce8c9 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 ad7c374 [Davies Liu] fix mllib test and warning ef1fc2f [Davies Liu] fix tests 4eee14a [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 20112ff [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 59bb492 [Davies Liu] fix tests 1da268c [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 ca0fdd3 [Davies Liu] fix code style 9563a15 [Davies Liu] add imap back for python 2 0b1ec04 [Davies Liu] make python examples work with Python 3 d2fd566 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 a716d34 [Davies Liu] test with python 3.4 f1700e8 [Davies Liu] fix test in python3 671b1db [Davies Liu] fix test in python3 692ff47 [Davies Liu] fix flaky test 7b9699f [Davies Liu] invalidate import cache for Python 3.3+ 9c58497 [Davies Liu] fix kill worker 309bfbf [Davies Liu] keep compatibility 5707476 [Davies Liu] cleanup, fix hash of string in 3.3+ 8662d5b [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 f53e1f0 [Davies Liu] fix tests 70b6b73 [Davies Liu] compile ec2/spark_ec2.py in python 3 a39167e [Davies Liu] support customize class in __main__ 814c77b [Davies Liu] run unittests with python 3 7f4476e [Davies Liu] mllib tests passed d737924 [Davies Liu] pass ml tests 375ea17 [Davies Liu] SQL tests pass 6cc42a9 [Davies Liu] rename 431a8de [Davies Liu] streaming tests pass 78901a7 [Davies Liu] fix hash of serializer in Python 3 24b2f2e [Davies Liu] pass all RDD tests 35f48fe [Davies Liu] run future again 1eebac2 [Davies Liu] fix conflict in ec2/spark_ec2.py 6e3c21d [Davies Liu] make cloudpickle work with Python3 2fb2db3 [Josh Rosen] Guard more changes behind sys.version; still doesn't run 1aa5e8f [twneale] Turned out `pickle.DictionaryType is dict` == True, so swapped it out 7354371 [twneale] buffer --> memoryview I'm not super sure if this a valid change, but the 2.7 docs recommend using memoryview over buffer where possible, so hoping it'll work. b69ccdf [twneale] Uses the pure python pickle._Pickler instead of c-extension _pickle.Pickler. It appears pyspark 2.7 uses the pure python pickler as well, so this shouldn't degrade pickling performance (?). f40d925 [twneale] xrange --> range e104215 [twneale] Replaces 2.7 types.InstsanceType with 3.4 `object`....could be horribly wrong depending on how types.InstanceType is used elsewhere in the package--see http://bugs.python.org/issue8206 79de9d0 [twneale] Replaces python2.7 `file` with 3.4 _io.TextIOWrapper 2adb42d [Josh Rosen] Fix up some import differences between Python 2 and 3 854be27 [Josh Rosen] Run `futurize` on Python code: 7c5b4ce [Josh Rosen] Remove Python 3 check in shell.py.
2015-04-16 19:20:57 -04:00
import sys
import functools
[SPARK-22313][PYTHON] Mark/print deprecation warnings as DeprecationWarning for deprecated APIs ## What changes were proposed in this pull request? This PR proposes to mark the existing warnings as `DeprecationWarning` and print out warnings for deprecated functions. This could be actually useful for Spark app developers. I use (old) PyCharm and this IDE can detect this specific `DeprecationWarning` in some cases: **Before** <img src="https://user-images.githubusercontent.com/6477701/31762664-df68d9f8-b4f6-11e7-8773-f0468f70a2cc.png" height="45" /> **After** <img src="https://user-images.githubusercontent.com/6477701/31762662-de4d6868-b4f6-11e7-98dc-3c8446a0c28a.png" height="70" /> For console usage, `DeprecationWarning` is usually disabled (see https://docs.python.org/2/library/warnings.html#warning-categories and https://docs.python.org/3/library/warnings.html#warning-categories): ``` >>> import warnings >>> filter(lambda f: f[2] == DeprecationWarning, warnings.filters) [('ignore', <_sre.SRE_Pattern object at 0x10ba58c00>, <type 'exceptions.DeprecationWarning'>, <_sre.SRE_Pattern object at 0x10bb04138>, 0), ('ignore', None, <type 'exceptions.DeprecationWarning'>, None, 0)] ``` so, it won't actually mess up the terminal much unless it is intended. If this is intendedly enabled, it'd should as below: ``` >>> import warnings >>> warnings.simplefilter('always', DeprecationWarning) >>> >>> from pyspark.sql import functions >>> functions.approxCountDistinct("a") .../spark/python/pyspark/sql/functions.py:232: DeprecationWarning: Deprecated in 2.1, use approx_count_distinct instead. "Deprecated in 2.1, use approx_count_distinct instead.", DeprecationWarning) ... ``` These instances were found by: ``` cd python/pyspark grep -r "Deprecated" . grep -r "deprecated" . grep -r "deprecate" . ``` ## How was this patch tested? Manually tested. Author: hyukjinkwon <gurwls223@gmail.com> Closes #19535 from HyukjinKwon/deprecated-warning.
2017-10-23 23:44:47 -04:00
import warnings
[SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames - The old implicit would convert RDDs directly to DataFrames, and that added too many methods. - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed Python changes: - toDataFrame -> toDF - Dsl -> functions package - addColumn -> withColumn - renameColumn -> withColumnRenamed - add toDF functions to RDD on SQLContext init - add flatMap to DataFrame Author: Reynold Xin <rxin@databricks.com> Author: Davies Liu <davies@databricks.com> Closes #4556 from rxin/SPARK-5752 and squashes the following commits: 5ef9910 [Reynold Xin] More fix 61d3fca [Reynold Xin] Merge branch 'df5' of github.com:davies/spark into SPARK-5752 ff5832c [Reynold Xin] Fix python 749c675 [Reynold Xin] count(*) fixes. 5806df0 [Reynold Xin] Fix build break again. d941f3d [Reynold Xin] Fixed explode compilation break. fe1267a [Davies Liu] flatMap c4afb8e [Reynold Xin] style d9de47f [Davies Liu] add comment b783994 [Davies Liu] add comment for toDF e2154e5 [Davies Liu] schema() -> schema 3a1004f [Davies Liu] Dsl -> functions, toDF() fb256af [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 0dd74eb [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames 97dd47c [Davies Liu] fix mistake 6168f74 [Davies Liu] fix test 1fc0199 [Davies Liu] fix test a075cd5 [Davies Liu] clean up, toPandas 663d314 [Davies Liu] add test for agg('*') 9e214d5 [Reynold Xin] count(*) fixes. 1ed7136 [Reynold Xin] Fix build break again. 921b2e3 [Reynold Xin] Fixed explode compilation break. 14698d4 [Davies Liu] flatMap ba3e12d [Reynold Xin] style d08c92d [Davies Liu] add comment 5c8b524 [Davies Liu] add comment for toDF a4e5e66 [Davies Liu] schema() -> schema d377fc9 [Davies Liu] Dsl -> functions, toDF() 6b3086c [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 807e8b1 [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames
2015-02-14 02:03:22 -05:00
[SPARK-4897] [PySpark] Python 3 support This PR update PySpark to support Python 3 (tested with 3.4). Known issue: unpickle array from Pyrolite is broken in Python 3, those tests are skipped. TODO: ec2/spark-ec2.py is not fully tested with python3. Author: Davies Liu <davies@databricks.com> Author: twneale <twneale@gmail.com> Author: Josh Rosen <joshrosen@databricks.com> Closes #5173 from davies/python3 and squashes the following commits: d7d6323 [Davies Liu] fix tests 6c52a98 [Davies Liu] fix mllib test 99e334f [Davies Liu] update timeout b716610 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 cafd5ec [Davies Liu] adddress comments from @mengxr bf225d7 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 179fc8d [Davies Liu] tuning flaky tests 8c8b957 [Davies Liu] fix ResourceWarning in Python 3 5c57c95 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 4006829 [Davies Liu] fix test 2fc0066 [Davies Liu] add python3 path 71535e9 [Davies Liu] fix xrange and divide 5a55ab4 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 125f12c [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 ed498c8 [Davies Liu] fix compatibility with python 3 820e649 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 e8ce8c9 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 ad7c374 [Davies Liu] fix mllib test and warning ef1fc2f [Davies Liu] fix tests 4eee14a [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 20112ff [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 59bb492 [Davies Liu] fix tests 1da268c [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 ca0fdd3 [Davies Liu] fix code style 9563a15 [Davies Liu] add imap back for python 2 0b1ec04 [Davies Liu] make python examples work with Python 3 d2fd566 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 a716d34 [Davies Liu] test with python 3.4 f1700e8 [Davies Liu] fix test in python3 671b1db [Davies Liu] fix test in python3 692ff47 [Davies Liu] fix flaky test 7b9699f [Davies Liu] invalidate import cache for Python 3.3+ 9c58497 [Davies Liu] fix kill worker 309bfbf [Davies Liu] keep compatibility 5707476 [Davies Liu] cleanup, fix hash of string in 3.3+ 8662d5b [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 f53e1f0 [Davies Liu] fix tests 70b6b73 [Davies Liu] compile ec2/spark_ec2.py in python 3 a39167e [Davies Liu] support customize class in __main__ 814c77b [Davies Liu] run unittests with python 3 7f4476e [Davies Liu] mllib tests passed d737924 [Davies Liu] pass ml tests 375ea17 [Davies Liu] SQL tests pass 6cc42a9 [Davies Liu] rename 431a8de [Davies Liu] streaming tests pass 78901a7 [Davies Liu] fix hash of serializer in Python 3 24b2f2e [Davies Liu] pass all RDD tests 35f48fe [Davies Liu] run future again 1eebac2 [Davies Liu] fix conflict in ec2/spark_ec2.py 6e3c21d [Davies Liu] make cloudpickle work with Python3 2fb2db3 [Josh Rosen] Guard more changes behind sys.version; still doesn't run 1aa5e8f [twneale] Turned out `pickle.DictionaryType is dict` == True, so swapped it out 7354371 [twneale] buffer --> memoryview I'm not super sure if this a valid change, but the 2.7 docs recommend using memoryview over buffer where possible, so hoping it'll work. b69ccdf [twneale] Uses the pure python pickle._Pickler instead of c-extension _pickle.Pickler. It appears pyspark 2.7 uses the pure python pickler as well, so this shouldn't degrade pickling performance (?). f40d925 [twneale] xrange --> range e104215 [twneale] Replaces 2.7 types.InstsanceType with 3.4 `object`....could be horribly wrong depending on how types.InstanceType is used elsewhere in the package--see http://bugs.python.org/issue8206 79de9d0 [twneale] Replaces python2.7 `file` with 3.4 _io.TextIOWrapper 2adb42d [Josh Rosen] Fix up some import differences between Python 2 and 3 854be27 [Josh Rosen] Run `futurize` on Python code: 7c5b4ce [Josh Rosen] Remove Python 3 check in shell.py.
2015-04-16 19:20:57 -04:00
if sys.version < "3":
from itertools import imap as map
[SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames - The old implicit would convert RDDs directly to DataFrames, and that added too many methods. - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed Python changes: - toDataFrame -> toDF - Dsl -> functions package - addColumn -> withColumn - renameColumn -> withColumnRenamed - add toDF functions to RDD on SQLContext init - add flatMap to DataFrame Author: Reynold Xin <rxin@databricks.com> Author: Davies Liu <davies@databricks.com> Closes #4556 from rxin/SPARK-5752 and squashes the following commits: 5ef9910 [Reynold Xin] More fix 61d3fca [Reynold Xin] Merge branch 'df5' of github.com:davies/spark into SPARK-5752 ff5832c [Reynold Xin] Fix python 749c675 [Reynold Xin] count(*) fixes. 5806df0 [Reynold Xin] Fix build break again. d941f3d [Reynold Xin] Fixed explode compilation break. fe1267a [Davies Liu] flatMap c4afb8e [Reynold Xin] style d9de47f [Davies Liu] add comment b783994 [Davies Liu] add comment for toDF e2154e5 [Davies Liu] schema() -> schema 3a1004f [Davies Liu] Dsl -> functions, toDF() fb256af [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 0dd74eb [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames 97dd47c [Davies Liu] fix mistake 6168f74 [Davies Liu] fix test 1fc0199 [Davies Liu] fix test a075cd5 [Davies Liu] clean up, toPandas 663d314 [Davies Liu] add test for agg('*') 9e214d5 [Reynold Xin] count(*) fixes. 1ed7136 [Reynold Xin] Fix build break again. 921b2e3 [Reynold Xin] Fixed explode compilation break. 14698d4 [Davies Liu] flatMap ba3e12d [Reynold Xin] style d08c92d [Davies Liu] add comment 5c8b524 [Davies Liu] add comment for toDF a4e5e66 [Davies Liu] schema() -> schema d377fc9 [Davies Liu] Dsl -> functions, toDF() 6b3086c [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 807e8b1 [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames
2015-02-14 02:03:22 -05:00
if sys.version >= '3':
basestring = str
from pyspark import since, SparkContext
from pyspark.rdd import ignore_unicode_prefix, PythonEvalType
[SPARK-26979][PYTHON][FOLLOW-UP] Make binary math/string functions take string as columns as well ## What changes were proposed in this pull request? This is a followup of https://github.com/apache/spark/pull/23882 to handle binary math/string functions. For instance, see the cases below: **Before:** ```python >>> from pyspark.sql.functions import lit, ascii >>> spark.range(1).select(lit('a').alias("value")).select(ascii("value")) Traceback (most recent call last): File "<stdin>", line 1, in <module> File "/.../spark/python/pyspark/sql/functions.py", line 51, in _ jc = getattr(sc._jvm.functions, name)(col._jc if isinstance(col, Column) else col) File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1286, in __call__ File "/.../spark/python/pyspark/sql/utils.py", line 63, in deco return f(*a, **kw) File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/protocol.py", line 332, in get_return_value py4j.protocol.Py4JError: An error occurred while calling z:org.apache.spark.sql.functions.ascii. Trace: py4j.Py4JException: Method ascii([class java.lang.String]) does not exist at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:318) at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:339) at py4j.Gateway.invoke(Gateway.java:276) at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132) at py4j.commands.CallCommand.execute(CallCommand.java:79) at py4j.GatewayConnection.run(GatewayConnection.java:238) at java.lang.Thread.run(Thread.java:748) ``` ```python >>> from pyspark.sql.functions import atan2 >>> spark.range(1).select(atan2("id", "id")) Traceback (most recent call last): File "<stdin>", line 1, in <module> File "/.../spark/python/pyspark/sql/functions.py", line 78, in _ jc = getattr(sc._jvm.functions, name)(col1._jc if isinstance(col1, Column) else float(col1), ValueError: could not convert string to float: id ``` **After:** ```python >>> from pyspark.sql.functions import lit, ascii >>> spark.range(1).select(lit('a').alias("value")).select(ascii("value")) DataFrame[ascii(value): int] ``` ```python >>> from pyspark.sql.functions import atan2 >>> spark.range(1).select(atan2("id", "id")) DataFrame[ATAN2(id, id): double] ``` Note that, - This PR causes a slight behaviour changes for math functions. For instance, numbers as strings (e.g., `"1"`) were supported as arguments of binary math functions before. After this PR, it recognises it as column names. - I also intentionally didn't document this behaviour changes since we're going ahead for Spark 3.0 and I don't think numbers as strings make much sense in math functions. - There is another exception `when`, which takes string as literal values as below. This PR doeesn't fix this ambiguity. ```python >>> spark.range(1).select(when(lit(True), col("id"))).show() ``` ``` +--------------------------+ |CASE WHEN true THEN id END| +--------------------------+ | 0| +--------------------------+ ``` ```python >>> spark.range(1).select(when(lit(True), "id")).show() ``` ``` +--------------------------+ |CASE WHEN true THEN id END| +--------------------------+ | id| +--------------------------+ ``` This PR also fixes as below: https://github.com/apache/spark/pull/23882 fixed it to: - Rename `_create_function` to `_create_name_function` - Define new `_create_function` to take strings as column names. This PR, I proposes to: - Revert `_create_name_function` name to `_create_function`. - Define new `_create_function_over_column` to take strings as column names. ## How was this patch tested? Some unit tests were added for binary math / string functions. Closes #24121 from HyukjinKwon/SPARK-26979. Authored-by: Hyukjin Kwon <gurwls223@apache.org> Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-19 19:06:10 -04:00
from pyspark.sql.column import Column, _to_java_column, _to_seq, _create_column_from_literal, \
_create_column_from_name
from pyspark.sql.dataframe import DataFrame
from pyspark.sql.types import StringType, DataType
# Keep UserDefinedFunction import for backwards compatible import; moved in SPARK-22409
from pyspark.sql.udf import UserDefinedFunction, _create_udf
[SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames - The old implicit would convert RDDs directly to DataFrames, and that added too many methods. - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed Python changes: - toDataFrame -> toDF - Dsl -> functions package - addColumn -> withColumn - renameColumn -> withColumnRenamed - add toDF functions to RDD on SQLContext init - add flatMap to DataFrame Author: Reynold Xin <rxin@databricks.com> Author: Davies Liu <davies@databricks.com> Closes #4556 from rxin/SPARK-5752 and squashes the following commits: 5ef9910 [Reynold Xin] More fix 61d3fca [Reynold Xin] Merge branch 'df5' of github.com:davies/spark into SPARK-5752 ff5832c [Reynold Xin] Fix python 749c675 [Reynold Xin] count(*) fixes. 5806df0 [Reynold Xin] Fix build break again. d941f3d [Reynold Xin] Fixed explode compilation break. fe1267a [Davies Liu] flatMap c4afb8e [Reynold Xin] style d9de47f [Davies Liu] add comment b783994 [Davies Liu] add comment for toDF e2154e5 [Davies Liu] schema() -> schema 3a1004f [Davies Liu] Dsl -> functions, toDF() fb256af [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 0dd74eb [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames 97dd47c [Davies Liu] fix mistake 6168f74 [Davies Liu] fix test 1fc0199 [Davies Liu] fix test a075cd5 [Davies Liu] clean up, toPandas 663d314 [Davies Liu] add test for agg('*') 9e214d5 [Reynold Xin] count(*) fixes. 1ed7136 [Reynold Xin] Fix build break again. 921b2e3 [Reynold Xin] Fixed explode compilation break. 14698d4 [Davies Liu] flatMap ba3e12d [Reynold Xin] style d08c92d [Davies Liu] add comment 5c8b524 [Davies Liu] add comment for toDF a4e5e66 [Davies Liu] schema() -> schema d377fc9 [Davies Liu] Dsl -> functions, toDF() 6b3086c [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 807e8b1 [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames
2015-02-14 02:03:22 -05:00
[SPARK-26979][PYTHON][FOLLOW-UP] Make binary math/string functions take string as columns as well ## What changes were proposed in this pull request? This is a followup of https://github.com/apache/spark/pull/23882 to handle binary math/string functions. For instance, see the cases below: **Before:** ```python >>> from pyspark.sql.functions import lit, ascii >>> spark.range(1).select(lit('a').alias("value")).select(ascii("value")) Traceback (most recent call last): File "<stdin>", line 1, in <module> File "/.../spark/python/pyspark/sql/functions.py", line 51, in _ jc = getattr(sc._jvm.functions, name)(col._jc if isinstance(col, Column) else col) File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1286, in __call__ File "/.../spark/python/pyspark/sql/utils.py", line 63, in deco return f(*a, **kw) File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/protocol.py", line 332, in get_return_value py4j.protocol.Py4JError: An error occurred while calling z:org.apache.spark.sql.functions.ascii. Trace: py4j.Py4JException: Method ascii([class java.lang.String]) does not exist at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:318) at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:339) at py4j.Gateway.invoke(Gateway.java:276) at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132) at py4j.commands.CallCommand.execute(CallCommand.java:79) at py4j.GatewayConnection.run(GatewayConnection.java:238) at java.lang.Thread.run(Thread.java:748) ``` ```python >>> from pyspark.sql.functions import atan2 >>> spark.range(1).select(atan2("id", "id")) Traceback (most recent call last): File "<stdin>", line 1, in <module> File "/.../spark/python/pyspark/sql/functions.py", line 78, in _ jc = getattr(sc._jvm.functions, name)(col1._jc if isinstance(col1, Column) else float(col1), ValueError: could not convert string to float: id ``` **After:** ```python >>> from pyspark.sql.functions import lit, ascii >>> spark.range(1).select(lit('a').alias("value")).select(ascii("value")) DataFrame[ascii(value): int] ``` ```python >>> from pyspark.sql.functions import atan2 >>> spark.range(1).select(atan2("id", "id")) DataFrame[ATAN2(id, id): double] ``` Note that, - This PR causes a slight behaviour changes for math functions. For instance, numbers as strings (e.g., `"1"`) were supported as arguments of binary math functions before. After this PR, it recognises it as column names. - I also intentionally didn't document this behaviour changes since we're going ahead for Spark 3.0 and I don't think numbers as strings make much sense in math functions. - There is another exception `when`, which takes string as literal values as below. This PR doeesn't fix this ambiguity. ```python >>> spark.range(1).select(when(lit(True), col("id"))).show() ``` ``` +--------------------------+ |CASE WHEN true THEN id END| +--------------------------+ | 0| +--------------------------+ ``` ```python >>> spark.range(1).select(when(lit(True), "id")).show() ``` ``` +--------------------------+ |CASE WHEN true THEN id END| +--------------------------+ | id| +--------------------------+ ``` This PR also fixes as below: https://github.com/apache/spark/pull/23882 fixed it to: - Rename `_create_function` to `_create_name_function` - Define new `_create_function` to take strings as column names. This PR, I proposes to: - Revert `_create_name_function` name to `_create_function`. - Define new `_create_function_over_column` to take strings as column names. ## How was this patch tested? Some unit tests were added for binary math / string functions. Closes #24121 from HyukjinKwon/SPARK-26979. Authored-by: Hyukjin Kwon <gurwls223@apache.org> Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-19 19:06:10 -04:00
# Note to developers: all of PySpark functions here take string as column names whenever possible.
# Namely, if columns are referred as arguments, they can be always both Column or string,
# even though there might be few exceptions for legacy or inevitable reasons.
# If you are fixing other language APIs together, also please note that Scala side is not the case
# since it requires to make every single overridden definition.
[SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames - The old implicit would convert RDDs directly to DataFrames, and that added too many methods. - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed Python changes: - toDataFrame -> toDF - Dsl -> functions package - addColumn -> withColumn - renameColumn -> withColumnRenamed - add toDF functions to RDD on SQLContext init - add flatMap to DataFrame Author: Reynold Xin <rxin@databricks.com> Author: Davies Liu <davies@databricks.com> Closes #4556 from rxin/SPARK-5752 and squashes the following commits: 5ef9910 [Reynold Xin] More fix 61d3fca [Reynold Xin] Merge branch 'df5' of github.com:davies/spark into SPARK-5752 ff5832c [Reynold Xin] Fix python 749c675 [Reynold Xin] count(*) fixes. 5806df0 [Reynold Xin] Fix build break again. d941f3d [Reynold Xin] Fixed explode compilation break. fe1267a [Davies Liu] flatMap c4afb8e [Reynold Xin] style d9de47f [Davies Liu] add comment b783994 [Davies Liu] add comment for toDF e2154e5 [Davies Liu] schema() -> schema 3a1004f [Davies Liu] Dsl -> functions, toDF() fb256af [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 0dd74eb [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames 97dd47c [Davies Liu] fix mistake 6168f74 [Davies Liu] fix test 1fc0199 [Davies Liu] fix test a075cd5 [Davies Liu] clean up, toPandas 663d314 [Davies Liu] add test for agg('*') 9e214d5 [Reynold Xin] count(*) fixes. 1ed7136 [Reynold Xin] Fix build break again. 921b2e3 [Reynold Xin] Fixed explode compilation break. 14698d4 [Davies Liu] flatMap ba3e12d [Reynold Xin] style d08c92d [Davies Liu] add comment 5c8b524 [Davies Liu] add comment for toDF a4e5e66 [Davies Liu] schema() -> schema d377fc9 [Davies Liu] Dsl -> functions, toDF() 6b3086c [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 807e8b1 [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames
2015-02-14 02:03:22 -05:00
[SPARK-26979][PYTHON][FOLLOW-UP] Make binary math/string functions take string as columns as well ## What changes were proposed in this pull request? This is a followup of https://github.com/apache/spark/pull/23882 to handle binary math/string functions. For instance, see the cases below: **Before:** ```python >>> from pyspark.sql.functions import lit, ascii >>> spark.range(1).select(lit('a').alias("value")).select(ascii("value")) Traceback (most recent call last): File "<stdin>", line 1, in <module> File "/.../spark/python/pyspark/sql/functions.py", line 51, in _ jc = getattr(sc._jvm.functions, name)(col._jc if isinstance(col, Column) else col) File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1286, in __call__ File "/.../spark/python/pyspark/sql/utils.py", line 63, in deco return f(*a, **kw) File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/protocol.py", line 332, in get_return_value py4j.protocol.Py4JError: An error occurred while calling z:org.apache.spark.sql.functions.ascii. Trace: py4j.Py4JException: Method ascii([class java.lang.String]) does not exist at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:318) at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:339) at py4j.Gateway.invoke(Gateway.java:276) at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132) at py4j.commands.CallCommand.execute(CallCommand.java:79) at py4j.GatewayConnection.run(GatewayConnection.java:238) at java.lang.Thread.run(Thread.java:748) ``` ```python >>> from pyspark.sql.functions import atan2 >>> spark.range(1).select(atan2("id", "id")) Traceback (most recent call last): File "<stdin>", line 1, in <module> File "/.../spark/python/pyspark/sql/functions.py", line 78, in _ jc = getattr(sc._jvm.functions, name)(col1._jc if isinstance(col1, Column) else float(col1), ValueError: could not convert string to float: id ``` **After:** ```python >>> from pyspark.sql.functions import lit, ascii >>> spark.range(1).select(lit('a').alias("value")).select(ascii("value")) DataFrame[ascii(value): int] ``` ```python >>> from pyspark.sql.functions import atan2 >>> spark.range(1).select(atan2("id", "id")) DataFrame[ATAN2(id, id): double] ``` Note that, - This PR causes a slight behaviour changes for math functions. For instance, numbers as strings (e.g., `"1"`) were supported as arguments of binary math functions before. After this PR, it recognises it as column names. - I also intentionally didn't document this behaviour changes since we're going ahead for Spark 3.0 and I don't think numbers as strings make much sense in math functions. - There is another exception `when`, which takes string as literal values as below. This PR doeesn't fix this ambiguity. ```python >>> spark.range(1).select(when(lit(True), col("id"))).show() ``` ``` +--------------------------+ |CASE WHEN true THEN id END| +--------------------------+ | 0| +--------------------------+ ``` ```python >>> spark.range(1).select(when(lit(True), "id")).show() ``` ``` +--------------------------+ |CASE WHEN true THEN id END| +--------------------------+ | id| +--------------------------+ ``` This PR also fixes as below: https://github.com/apache/spark/pull/23882 fixed it to: - Rename `_create_function` to `_create_name_function` - Define new `_create_function` to take strings as column names. This PR, I proposes to: - Revert `_create_name_function` name to `_create_function`. - Define new `_create_function_over_column` to take strings as column names. ## How was this patch tested? Some unit tests were added for binary math / string functions. Closes #24121 from HyukjinKwon/SPARK-26979. Authored-by: Hyukjin Kwon <gurwls223@apache.org> Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-19 19:06:10 -04:00
def _create_function(name, doc=""):
"""Create a PySpark function by its name"""
[SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames - The old implicit would convert RDDs directly to DataFrames, and that added too many methods. - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed Python changes: - toDataFrame -> toDF - Dsl -> functions package - addColumn -> withColumn - renameColumn -> withColumnRenamed - add toDF functions to RDD on SQLContext init - add flatMap to DataFrame Author: Reynold Xin <rxin@databricks.com> Author: Davies Liu <davies@databricks.com> Closes #4556 from rxin/SPARK-5752 and squashes the following commits: 5ef9910 [Reynold Xin] More fix 61d3fca [Reynold Xin] Merge branch 'df5' of github.com:davies/spark into SPARK-5752 ff5832c [Reynold Xin] Fix python 749c675 [Reynold Xin] count(*) fixes. 5806df0 [Reynold Xin] Fix build break again. d941f3d [Reynold Xin] Fixed explode compilation break. fe1267a [Davies Liu] flatMap c4afb8e [Reynold Xin] style d9de47f [Davies Liu] add comment b783994 [Davies Liu] add comment for toDF e2154e5 [Davies Liu] schema() -> schema 3a1004f [Davies Liu] Dsl -> functions, toDF() fb256af [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 0dd74eb [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames 97dd47c [Davies Liu] fix mistake 6168f74 [Davies Liu] fix test 1fc0199 [Davies Liu] fix test a075cd5 [Davies Liu] clean up, toPandas 663d314 [Davies Liu] add test for agg('*') 9e214d5 [Reynold Xin] count(*) fixes. 1ed7136 [Reynold Xin] Fix build break again. 921b2e3 [Reynold Xin] Fixed explode compilation break. 14698d4 [Davies Liu] flatMap ba3e12d [Reynold Xin] style d08c92d [Davies Liu] add comment 5c8b524 [Davies Liu] add comment for toDF a4e5e66 [Davies Liu] schema() -> schema d377fc9 [Davies Liu] Dsl -> functions, toDF() 6b3086c [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 807e8b1 [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames
2015-02-14 02:03:22 -05:00
def _(col):
sc = SparkContext._active_spark_context
jc = getattr(sc._jvm.functions, name)(col._jc if isinstance(col, Column) else col)
[SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames - The old implicit would convert RDDs directly to DataFrames, and that added too many methods. - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed Python changes: - toDataFrame -> toDF - Dsl -> functions package - addColumn -> withColumn - renameColumn -> withColumnRenamed - add toDF functions to RDD on SQLContext init - add flatMap to DataFrame Author: Reynold Xin <rxin@databricks.com> Author: Davies Liu <davies@databricks.com> Closes #4556 from rxin/SPARK-5752 and squashes the following commits: 5ef9910 [Reynold Xin] More fix 61d3fca [Reynold Xin] Merge branch 'df5' of github.com:davies/spark into SPARK-5752 ff5832c [Reynold Xin] Fix python 749c675 [Reynold Xin] count(*) fixes. 5806df0 [Reynold Xin] Fix build break again. d941f3d [Reynold Xin] Fixed explode compilation break. fe1267a [Davies Liu] flatMap c4afb8e [Reynold Xin] style d9de47f [Davies Liu] add comment b783994 [Davies Liu] add comment for toDF e2154e5 [Davies Liu] schema() -> schema 3a1004f [Davies Liu] Dsl -> functions, toDF() fb256af [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 0dd74eb [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames 97dd47c [Davies Liu] fix mistake 6168f74 [Davies Liu] fix test 1fc0199 [Davies Liu] fix test a075cd5 [Davies Liu] clean up, toPandas 663d314 [Davies Liu] add test for agg('*') 9e214d5 [Reynold Xin] count(*) fixes. 1ed7136 [Reynold Xin] Fix build break again. 921b2e3 [Reynold Xin] Fixed explode compilation break. 14698d4 [Davies Liu] flatMap ba3e12d [Reynold Xin] style d08c92d [Davies Liu] add comment 5c8b524 [Davies Liu] add comment for toDF a4e5e66 [Davies Liu] schema() -> schema d377fc9 [Davies Liu] Dsl -> functions, toDF() 6b3086c [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 807e8b1 [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames
2015-02-14 02:03:22 -05:00
return Column(jc)
_.__name__ = name
_.__doc__ = doc
return _
[SPARK-26979][PYTHON][FOLLOW-UP] Make binary math/string functions take string as columns as well ## What changes were proposed in this pull request? This is a followup of https://github.com/apache/spark/pull/23882 to handle binary math/string functions. For instance, see the cases below: **Before:** ```python >>> from pyspark.sql.functions import lit, ascii >>> spark.range(1).select(lit('a').alias("value")).select(ascii("value")) Traceback (most recent call last): File "<stdin>", line 1, in <module> File "/.../spark/python/pyspark/sql/functions.py", line 51, in _ jc = getattr(sc._jvm.functions, name)(col._jc if isinstance(col, Column) else col) File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1286, in __call__ File "/.../spark/python/pyspark/sql/utils.py", line 63, in deco return f(*a, **kw) File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/protocol.py", line 332, in get_return_value py4j.protocol.Py4JError: An error occurred while calling z:org.apache.spark.sql.functions.ascii. Trace: py4j.Py4JException: Method ascii([class java.lang.String]) does not exist at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:318) at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:339) at py4j.Gateway.invoke(Gateway.java:276) at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132) at py4j.commands.CallCommand.execute(CallCommand.java:79) at py4j.GatewayConnection.run(GatewayConnection.java:238) at java.lang.Thread.run(Thread.java:748) ``` ```python >>> from pyspark.sql.functions import atan2 >>> spark.range(1).select(atan2("id", "id")) Traceback (most recent call last): File "<stdin>", line 1, in <module> File "/.../spark/python/pyspark/sql/functions.py", line 78, in _ jc = getattr(sc._jvm.functions, name)(col1._jc if isinstance(col1, Column) else float(col1), ValueError: could not convert string to float: id ``` **After:** ```python >>> from pyspark.sql.functions import lit, ascii >>> spark.range(1).select(lit('a').alias("value")).select(ascii("value")) DataFrame[ascii(value): int] ``` ```python >>> from pyspark.sql.functions import atan2 >>> spark.range(1).select(atan2("id", "id")) DataFrame[ATAN2(id, id): double] ``` Note that, - This PR causes a slight behaviour changes for math functions. For instance, numbers as strings (e.g., `"1"`) were supported as arguments of binary math functions before. After this PR, it recognises it as column names. - I also intentionally didn't document this behaviour changes since we're going ahead for Spark 3.0 and I don't think numbers as strings make much sense in math functions. - There is another exception `when`, which takes string as literal values as below. This PR doeesn't fix this ambiguity. ```python >>> spark.range(1).select(when(lit(True), col("id"))).show() ``` ``` +--------------------------+ |CASE WHEN true THEN id END| +--------------------------+ | 0| +--------------------------+ ``` ```python >>> spark.range(1).select(when(lit(True), "id")).show() ``` ``` +--------------------------+ |CASE WHEN true THEN id END| +--------------------------+ | id| +--------------------------+ ``` This PR also fixes as below: https://github.com/apache/spark/pull/23882 fixed it to: - Rename `_create_function` to `_create_name_function` - Define new `_create_function` to take strings as column names. This PR, I proposes to: - Revert `_create_name_function` name to `_create_function`. - Define new `_create_function_over_column` to take strings as column names. ## How was this patch tested? Some unit tests were added for binary math / string functions. Closes #24121 from HyukjinKwon/SPARK-26979. Authored-by: Hyukjin Kwon <gurwls223@apache.org> Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-19 19:06:10 -04:00
def _create_function_over_column(name, doc=""):
"""Similar with `_create_function` but creates a PySpark function that takes a column
(as string as well). This is mainly for PySpark functions to take strings as
column names.
"""
[SPARK-26979][PYTHON] Add missing string column name support for some SQL functions ## What changes were proposed in this pull request? Most SQL functions defined in `spark.sql.functions` have two calling patterns, one with a Column object as input, and another with a string representing a column name, which is then converted into a Column object internally. There are, however, a few notable exceptions: - lower() - upper() - abs() - bitwiseNOT() - ltrim() - rtrim() - trim() - ascii() - base64() - unbase64() While this doesn't break anything, as you can easily create a Column object yourself prior to passing it to one of these functions, it has two undesirable consequences: 1. It is surprising - it breaks coder's expectations when they are first starting with Spark. Every API should be as consistent as possible, so as to make the learning curve smoother and to reduce causes for human error; 2. It gets in the way of stylistic conventions. Most of the time it makes Python code more readable to use literal names, and the API provides ample support for that, but these few exceptions prevent this pattern from being universally applicable. This patch is meant to fix the aforementioned problem. ### Effect This patch **enables** support for passing column names as input to those functions mentioned above. ### Side effects This PR also **fixes** an issue with some functions being defined multiple times by using `_create_function()`. ### How it works `_create_function()` was redefined to always convert the argument to a Column object. The old implementation has been kept under `_create_name_function()`, and is still being used to generate the following special functions: - lit() - col() - column() - asc() - desc() - asc_nulls_first() - asc_nulls_last() - desc_nulls_first() - desc_nulls_last() This is because these functions can only take a column name as their argument. This is not a problem, as their semantics require so. ## How was this patch tested? Ran ./dev/run-tests and tested it manually. Closes #23882 from asmello/col-name-support-pyspark. Authored-by: André Sá de Mello <amello@palantir.com> Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-17 13:58:16 -04:00
def _(col):
sc = SparkContext._active_spark_context
jc = getattr(sc._jvm.functions, name)(_to_java_column(col))
return Column(jc)
_.__name__ = name
_.__doc__ = doc
return _
[SPARK-22313][PYTHON] Mark/print deprecation warnings as DeprecationWarning for deprecated APIs ## What changes were proposed in this pull request? This PR proposes to mark the existing warnings as `DeprecationWarning` and print out warnings for deprecated functions. This could be actually useful for Spark app developers. I use (old) PyCharm and this IDE can detect this specific `DeprecationWarning` in some cases: **Before** <img src="https://user-images.githubusercontent.com/6477701/31762664-df68d9f8-b4f6-11e7-8773-f0468f70a2cc.png" height="45" /> **After** <img src="https://user-images.githubusercontent.com/6477701/31762662-de4d6868-b4f6-11e7-98dc-3c8446a0c28a.png" height="70" /> For console usage, `DeprecationWarning` is usually disabled (see https://docs.python.org/2/library/warnings.html#warning-categories and https://docs.python.org/3/library/warnings.html#warning-categories): ``` >>> import warnings >>> filter(lambda f: f[2] == DeprecationWarning, warnings.filters) [('ignore', <_sre.SRE_Pattern object at 0x10ba58c00>, <type 'exceptions.DeprecationWarning'>, <_sre.SRE_Pattern object at 0x10bb04138>, 0), ('ignore', None, <type 'exceptions.DeprecationWarning'>, None, 0)] ``` so, it won't actually mess up the terminal much unless it is intended. If this is intendedly enabled, it'd should as below: ``` >>> import warnings >>> warnings.simplefilter('always', DeprecationWarning) >>> >>> from pyspark.sql import functions >>> functions.approxCountDistinct("a") .../spark/python/pyspark/sql/functions.py:232: DeprecationWarning: Deprecated in 2.1, use approx_count_distinct instead. "Deprecated in 2.1, use approx_count_distinct instead.", DeprecationWarning) ... ``` These instances were found by: ``` cd python/pyspark grep -r "Deprecated" . grep -r "deprecated" . grep -r "deprecate" . ``` ## How was this patch tested? Manually tested. Author: hyukjinkwon <gurwls223@gmail.com> Closes #19535 from HyukjinKwon/deprecated-warning.
2017-10-23 23:44:47 -04:00
def _wrap_deprecated_function(func, message):
""" Wrap the deprecated function to print out deprecation warnings"""
def _(col):
warnings.warn(message, DeprecationWarning)
return func(col)
return functools.wraps(func)(_)
def _create_binary_mathfunction(name, doc=""):
""" Create a binary mathfunction by name"""
def _(col1, col2):
sc = SparkContext._active_spark_context
[SPARK-26979][PYTHON][FOLLOW-UP] Make binary math/string functions take string as columns as well ## What changes were proposed in this pull request? This is a followup of https://github.com/apache/spark/pull/23882 to handle binary math/string functions. For instance, see the cases below: **Before:** ```python >>> from pyspark.sql.functions import lit, ascii >>> spark.range(1).select(lit('a').alias("value")).select(ascii("value")) Traceback (most recent call last): File "<stdin>", line 1, in <module> File "/.../spark/python/pyspark/sql/functions.py", line 51, in _ jc = getattr(sc._jvm.functions, name)(col._jc if isinstance(col, Column) else col) File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1286, in __call__ File "/.../spark/python/pyspark/sql/utils.py", line 63, in deco return f(*a, **kw) File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/protocol.py", line 332, in get_return_value py4j.protocol.Py4JError: An error occurred while calling z:org.apache.spark.sql.functions.ascii. Trace: py4j.Py4JException: Method ascii([class java.lang.String]) does not exist at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:318) at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:339) at py4j.Gateway.invoke(Gateway.java:276) at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132) at py4j.commands.CallCommand.execute(CallCommand.java:79) at py4j.GatewayConnection.run(GatewayConnection.java:238) at java.lang.Thread.run(Thread.java:748) ``` ```python >>> from pyspark.sql.functions import atan2 >>> spark.range(1).select(atan2("id", "id")) Traceback (most recent call last): File "<stdin>", line 1, in <module> File "/.../spark/python/pyspark/sql/functions.py", line 78, in _ jc = getattr(sc._jvm.functions, name)(col1._jc if isinstance(col1, Column) else float(col1), ValueError: could not convert string to float: id ``` **After:** ```python >>> from pyspark.sql.functions import lit, ascii >>> spark.range(1).select(lit('a').alias("value")).select(ascii("value")) DataFrame[ascii(value): int] ``` ```python >>> from pyspark.sql.functions import atan2 >>> spark.range(1).select(atan2("id", "id")) DataFrame[ATAN2(id, id): double] ``` Note that, - This PR causes a slight behaviour changes for math functions. For instance, numbers as strings (e.g., `"1"`) were supported as arguments of binary math functions before. After this PR, it recognises it as column names. - I also intentionally didn't document this behaviour changes since we're going ahead for Spark 3.0 and I don't think numbers as strings make much sense in math functions. - There is another exception `when`, which takes string as literal values as below. This PR doeesn't fix this ambiguity. ```python >>> spark.range(1).select(when(lit(True), col("id"))).show() ``` ``` +--------------------------+ |CASE WHEN true THEN id END| +--------------------------+ | 0| +--------------------------+ ``` ```python >>> spark.range(1).select(when(lit(True), "id")).show() ``` ``` +--------------------------+ |CASE WHEN true THEN id END| +--------------------------+ | id| +--------------------------+ ``` This PR also fixes as below: https://github.com/apache/spark/pull/23882 fixed it to: - Rename `_create_function` to `_create_name_function` - Define new `_create_function` to take strings as column names. This PR, I proposes to: - Revert `_create_name_function` name to `_create_function`. - Define new `_create_function_over_column` to take strings as column names. ## How was this patch tested? Some unit tests were added for binary math / string functions. Closes #24121 from HyukjinKwon/SPARK-26979. Authored-by: Hyukjin Kwon <gurwls223@apache.org> Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-19 19:06:10 -04:00
# For legacy reasons, the arguments here can be implicitly converted into floats,
# if they are not columns or strings.
if isinstance(col1, Column):
arg1 = col1._jc
elif isinstance(col1, basestring):
arg1 = _create_column_from_name(col1)
else:
arg1 = float(col1)
if isinstance(col2, Column):
arg2 = col2._jc
elif isinstance(col2, basestring):
arg2 = _create_column_from_name(col2)
else:
arg2 = float(col2)
jc = getattr(sc._jvm.functions, name)(arg1, arg2)
return Column(jc)
_.__name__ = name
_.__doc__ = doc
return _
def _create_window_function(name, doc=''):
""" Create a window function by name """
def _():
sc = SparkContext._active_spark_context
jc = getattr(sc._jvm.functions, name)()
return Column(jc)
_.__name__ = name
_.__doc__ = 'Window function: ' + doc
return _
_lit_doc = """
Creates a :class:`Column` of literal value.
>>> df.select(lit(5).alias('height')).withColumn('spark_user', lit(True)).take(1)
[Row(height=5, spark_user=True)]
"""
[SPARK-26979][PYTHON][FOLLOW-UP] Make binary math/string functions take string as columns as well ## What changes were proposed in this pull request? This is a followup of https://github.com/apache/spark/pull/23882 to handle binary math/string functions. For instance, see the cases below: **Before:** ```python >>> from pyspark.sql.functions import lit, ascii >>> spark.range(1).select(lit('a').alias("value")).select(ascii("value")) Traceback (most recent call last): File "<stdin>", line 1, in <module> File "/.../spark/python/pyspark/sql/functions.py", line 51, in _ jc = getattr(sc._jvm.functions, name)(col._jc if isinstance(col, Column) else col) File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1286, in __call__ File "/.../spark/python/pyspark/sql/utils.py", line 63, in deco return f(*a, **kw) File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/protocol.py", line 332, in get_return_value py4j.protocol.Py4JError: An error occurred while calling z:org.apache.spark.sql.functions.ascii. Trace: py4j.Py4JException: Method ascii([class java.lang.String]) does not exist at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:318) at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:339) at py4j.Gateway.invoke(Gateway.java:276) at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132) at py4j.commands.CallCommand.execute(CallCommand.java:79) at py4j.GatewayConnection.run(GatewayConnection.java:238) at java.lang.Thread.run(Thread.java:748) ``` ```python >>> from pyspark.sql.functions import atan2 >>> spark.range(1).select(atan2("id", "id")) Traceback (most recent call last): File "<stdin>", line 1, in <module> File "/.../spark/python/pyspark/sql/functions.py", line 78, in _ jc = getattr(sc._jvm.functions, name)(col1._jc if isinstance(col1, Column) else float(col1), ValueError: could not convert string to float: id ``` **After:** ```python >>> from pyspark.sql.functions import lit, ascii >>> spark.range(1).select(lit('a').alias("value")).select(ascii("value")) DataFrame[ascii(value): int] ``` ```python >>> from pyspark.sql.functions import atan2 >>> spark.range(1).select(atan2("id", "id")) DataFrame[ATAN2(id, id): double] ``` Note that, - This PR causes a slight behaviour changes for math functions. For instance, numbers as strings (e.g., `"1"`) were supported as arguments of binary math functions before. After this PR, it recognises it as column names. - I also intentionally didn't document this behaviour changes since we're going ahead for Spark 3.0 and I don't think numbers as strings make much sense in math functions. - There is another exception `when`, which takes string as literal values as below. This PR doeesn't fix this ambiguity. ```python >>> spark.range(1).select(when(lit(True), col("id"))).show() ``` ``` +--------------------------+ |CASE WHEN true THEN id END| +--------------------------+ | 0| +--------------------------+ ``` ```python >>> spark.range(1).select(when(lit(True), "id")).show() ``` ``` +--------------------------+ |CASE WHEN true THEN id END| +--------------------------+ | id| +--------------------------+ ``` This PR also fixes as below: https://github.com/apache/spark/pull/23882 fixed it to: - Rename `_create_function` to `_create_name_function` - Define new `_create_function` to take strings as column names. This PR, I proposes to: - Revert `_create_name_function` name to `_create_function`. - Define new `_create_function_over_column` to take strings as column names. ## How was this patch tested? Some unit tests were added for binary math / string functions. Closes #24121 from HyukjinKwon/SPARK-26979. Authored-by: Hyukjin Kwon <gurwls223@apache.org> Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-19 19:06:10 -04:00
_functions = {
'lit': _lit_doc,
[SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames - The old implicit would convert RDDs directly to DataFrames, and that added too many methods. - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed Python changes: - toDataFrame -> toDF - Dsl -> functions package - addColumn -> withColumn - renameColumn -> withColumnRenamed - add toDF functions to RDD on SQLContext init - add flatMap to DataFrame Author: Reynold Xin <rxin@databricks.com> Author: Davies Liu <davies@databricks.com> Closes #4556 from rxin/SPARK-5752 and squashes the following commits: 5ef9910 [Reynold Xin] More fix 61d3fca [Reynold Xin] Merge branch 'df5' of github.com:davies/spark into SPARK-5752 ff5832c [Reynold Xin] Fix python 749c675 [Reynold Xin] count(*) fixes. 5806df0 [Reynold Xin] Fix build break again. d941f3d [Reynold Xin] Fixed explode compilation break. fe1267a [Davies Liu] flatMap c4afb8e [Reynold Xin] style d9de47f [Davies Liu] add comment b783994 [Davies Liu] add comment for toDF e2154e5 [Davies Liu] schema() -> schema 3a1004f [Davies Liu] Dsl -> functions, toDF() fb256af [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 0dd74eb [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames 97dd47c [Davies Liu] fix mistake 6168f74 [Davies Liu] fix test 1fc0199 [Davies Liu] fix test a075cd5 [Davies Liu] clean up, toPandas 663d314 [Davies Liu] add test for agg('*') 9e214d5 [Reynold Xin] count(*) fixes. 1ed7136 [Reynold Xin] Fix build break again. 921b2e3 [Reynold Xin] Fixed explode compilation break. 14698d4 [Davies Liu] flatMap ba3e12d [Reynold Xin] style d08c92d [Davies Liu] add comment 5c8b524 [Davies Liu] add comment for toDF a4e5e66 [Davies Liu] schema() -> schema d377fc9 [Davies Liu] Dsl -> functions, toDF() 6b3086c [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 807e8b1 [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames
2015-02-14 02:03:22 -05:00
'col': 'Returns a :class:`Column` based on the given column name.',
'column': 'Returns a :class:`Column` based on the given column name.',
'asc': 'Returns a sort expression based on the ascending order of the given column name.',
'desc': 'Returns a sort expression based on the descending order of the given column name.',
[SPARK-26979][PYTHON] Add missing string column name support for some SQL functions ## What changes were proposed in this pull request? Most SQL functions defined in `spark.sql.functions` have two calling patterns, one with a Column object as input, and another with a string representing a column name, which is then converted into a Column object internally. There are, however, a few notable exceptions: - lower() - upper() - abs() - bitwiseNOT() - ltrim() - rtrim() - trim() - ascii() - base64() - unbase64() While this doesn't break anything, as you can easily create a Column object yourself prior to passing it to one of these functions, it has two undesirable consequences: 1. It is surprising - it breaks coder's expectations when they are first starting with Spark. Every API should be as consistent as possible, so as to make the learning curve smoother and to reduce causes for human error; 2. It gets in the way of stylistic conventions. Most of the time it makes Python code more readable to use literal names, and the API provides ample support for that, but these few exceptions prevent this pattern from being universally applicable. This patch is meant to fix the aforementioned problem. ### Effect This patch **enables** support for passing column names as input to those functions mentioned above. ### Side effects This PR also **fixes** an issue with some functions being defined multiple times by using `_create_function()`. ### How it works `_create_function()` was redefined to always convert the argument to a Column object. The old implementation has been kept under `_create_name_function()`, and is still being used to generate the following special functions: - lit() - col() - column() - asc() - desc() - asc_nulls_first() - asc_nulls_last() - desc_nulls_first() - desc_nulls_last() This is because these functions can only take a column name as their argument. This is not a problem, as their semantics require so. ## How was this patch tested? Ran ./dev/run-tests and tested it manually. Closes #23882 from asmello/col-name-support-pyspark. Authored-by: André Sá de Mello <amello@palantir.com> Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-17 13:58:16 -04:00
}
[SPARK-26979][PYTHON][FOLLOW-UP] Make binary math/string functions take string as columns as well ## What changes were proposed in this pull request? This is a followup of https://github.com/apache/spark/pull/23882 to handle binary math/string functions. For instance, see the cases below: **Before:** ```python >>> from pyspark.sql.functions import lit, ascii >>> spark.range(1).select(lit('a').alias("value")).select(ascii("value")) Traceback (most recent call last): File "<stdin>", line 1, in <module> File "/.../spark/python/pyspark/sql/functions.py", line 51, in _ jc = getattr(sc._jvm.functions, name)(col._jc if isinstance(col, Column) else col) File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1286, in __call__ File "/.../spark/python/pyspark/sql/utils.py", line 63, in deco return f(*a, **kw) File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/protocol.py", line 332, in get_return_value py4j.protocol.Py4JError: An error occurred while calling z:org.apache.spark.sql.functions.ascii. Trace: py4j.Py4JException: Method ascii([class java.lang.String]) does not exist at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:318) at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:339) at py4j.Gateway.invoke(Gateway.java:276) at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132) at py4j.commands.CallCommand.execute(CallCommand.java:79) at py4j.GatewayConnection.run(GatewayConnection.java:238) at java.lang.Thread.run(Thread.java:748) ``` ```python >>> from pyspark.sql.functions import atan2 >>> spark.range(1).select(atan2("id", "id")) Traceback (most recent call last): File "<stdin>", line 1, in <module> File "/.../spark/python/pyspark/sql/functions.py", line 78, in _ jc = getattr(sc._jvm.functions, name)(col1._jc if isinstance(col1, Column) else float(col1), ValueError: could not convert string to float: id ``` **After:** ```python >>> from pyspark.sql.functions import lit, ascii >>> spark.range(1).select(lit('a').alias("value")).select(ascii("value")) DataFrame[ascii(value): int] ``` ```python >>> from pyspark.sql.functions import atan2 >>> spark.range(1).select(atan2("id", "id")) DataFrame[ATAN2(id, id): double] ``` Note that, - This PR causes a slight behaviour changes for math functions. For instance, numbers as strings (e.g., `"1"`) were supported as arguments of binary math functions before. After this PR, it recognises it as column names. - I also intentionally didn't document this behaviour changes since we're going ahead for Spark 3.0 and I don't think numbers as strings make much sense in math functions. - There is another exception `when`, which takes string as literal values as below. This PR doeesn't fix this ambiguity. ```python >>> spark.range(1).select(when(lit(True), col("id"))).show() ``` ``` +--------------------------+ |CASE WHEN true THEN id END| +--------------------------+ | 0| +--------------------------+ ``` ```python >>> spark.range(1).select(when(lit(True), "id")).show() ``` ``` +--------------------------+ |CASE WHEN true THEN id END| +--------------------------+ | id| +--------------------------+ ``` This PR also fixes as below: https://github.com/apache/spark/pull/23882 fixed it to: - Rename `_create_function` to `_create_name_function` - Define new `_create_function` to take strings as column names. This PR, I proposes to: - Revert `_create_name_function` name to `_create_function`. - Define new `_create_function_over_column` to take strings as column names. ## How was this patch tested? Some unit tests were added for binary math / string functions. Closes #24121 from HyukjinKwon/SPARK-26979. Authored-by: Hyukjin Kwon <gurwls223@apache.org> Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-19 19:06:10 -04:00
_functions_over_column = {
[SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames - The old implicit would convert RDDs directly to DataFrames, and that added too many methods. - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed Python changes: - toDataFrame -> toDF - Dsl -> functions package - addColumn -> withColumn - renameColumn -> withColumnRenamed - add toDF functions to RDD on SQLContext init - add flatMap to DataFrame Author: Reynold Xin <rxin@databricks.com> Author: Davies Liu <davies@databricks.com> Closes #4556 from rxin/SPARK-5752 and squashes the following commits: 5ef9910 [Reynold Xin] More fix 61d3fca [Reynold Xin] Merge branch 'df5' of github.com:davies/spark into SPARK-5752 ff5832c [Reynold Xin] Fix python 749c675 [Reynold Xin] count(*) fixes. 5806df0 [Reynold Xin] Fix build break again. d941f3d [Reynold Xin] Fixed explode compilation break. fe1267a [Davies Liu] flatMap c4afb8e [Reynold Xin] style d9de47f [Davies Liu] add comment b783994 [Davies Liu] add comment for toDF e2154e5 [Davies Liu] schema() -> schema 3a1004f [Davies Liu] Dsl -> functions, toDF() fb256af [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 0dd74eb [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames 97dd47c [Davies Liu] fix mistake 6168f74 [Davies Liu] fix test 1fc0199 [Davies Liu] fix test a075cd5 [Davies Liu] clean up, toPandas 663d314 [Davies Liu] add test for agg('*') 9e214d5 [Reynold Xin] count(*) fixes. 1ed7136 [Reynold Xin] Fix build break again. 921b2e3 [Reynold Xin] Fixed explode compilation break. 14698d4 [Davies Liu] flatMap ba3e12d [Reynold Xin] style d08c92d [Davies Liu] add comment 5c8b524 [Davies Liu] add comment for toDF a4e5e66 [Davies Liu] schema() -> schema d377fc9 [Davies Liu] Dsl -> functions, toDF() 6b3086c [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 807e8b1 [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames
2015-02-14 02:03:22 -05:00
'sqrt': 'Computes the square root of the specified float value.',
'abs': 'Computes the absolute value.',
[SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames - The old implicit would convert RDDs directly to DataFrames, and that added too many methods. - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed Python changes: - toDataFrame -> toDF - Dsl -> functions package - addColumn -> withColumn - renameColumn -> withColumnRenamed - add toDF functions to RDD on SQLContext init - add flatMap to DataFrame Author: Reynold Xin <rxin@databricks.com> Author: Davies Liu <davies@databricks.com> Closes #4556 from rxin/SPARK-5752 and squashes the following commits: 5ef9910 [Reynold Xin] More fix 61d3fca [Reynold Xin] Merge branch 'df5' of github.com:davies/spark into SPARK-5752 ff5832c [Reynold Xin] Fix python 749c675 [Reynold Xin] count(*) fixes. 5806df0 [Reynold Xin] Fix build break again. d941f3d [Reynold Xin] Fixed explode compilation break. fe1267a [Davies Liu] flatMap c4afb8e [Reynold Xin] style d9de47f [Davies Liu] add comment b783994 [Davies Liu] add comment for toDF e2154e5 [Davies Liu] schema() -> schema 3a1004f [Davies Liu] Dsl -> functions, toDF() fb256af [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 0dd74eb [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames 97dd47c [Davies Liu] fix mistake 6168f74 [Davies Liu] fix test 1fc0199 [Davies Liu] fix test a075cd5 [Davies Liu] clean up, toPandas 663d314 [Davies Liu] add test for agg('*') 9e214d5 [Reynold Xin] count(*) fixes. 1ed7136 [Reynold Xin] Fix build break again. 921b2e3 [Reynold Xin] Fixed explode compilation break. 14698d4 [Davies Liu] flatMap ba3e12d [Reynold Xin] style d08c92d [Davies Liu] add comment 5c8b524 [Davies Liu] add comment for toDF a4e5e66 [Davies Liu] schema() -> schema d377fc9 [Davies Liu] Dsl -> functions, toDF() 6b3086c [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 807e8b1 [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames
2015-02-14 02:03:22 -05:00
'max': 'Aggregate function: returns the maximum value of the expression in a group.',
'min': 'Aggregate function: returns the minimum value of the expression in a group.',
'count': 'Aggregate function: returns the number of items in a group.',
'sum': 'Aggregate function: returns the sum of all values in the expression.',
'avg': 'Aggregate function: returns the average of the values in a group.',
'mean': 'Aggregate function: returns the average of the values in a group.',
'sumDistinct': 'Aggregate function: returns the sum of distinct values in the expression.',
}
[SPARK-26979][PYTHON][FOLLOW-UP] Make binary math/string functions take string as columns as well ## What changes were proposed in this pull request? This is a followup of https://github.com/apache/spark/pull/23882 to handle binary math/string functions. For instance, see the cases below: **Before:** ```python >>> from pyspark.sql.functions import lit, ascii >>> spark.range(1).select(lit('a').alias("value")).select(ascii("value")) Traceback (most recent call last): File "<stdin>", line 1, in <module> File "/.../spark/python/pyspark/sql/functions.py", line 51, in _ jc = getattr(sc._jvm.functions, name)(col._jc if isinstance(col, Column) else col) File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1286, in __call__ File "/.../spark/python/pyspark/sql/utils.py", line 63, in deco return f(*a, **kw) File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/protocol.py", line 332, in get_return_value py4j.protocol.Py4JError: An error occurred while calling z:org.apache.spark.sql.functions.ascii. Trace: py4j.Py4JException: Method ascii([class java.lang.String]) does not exist at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:318) at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:339) at py4j.Gateway.invoke(Gateway.java:276) at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132) at py4j.commands.CallCommand.execute(CallCommand.java:79) at py4j.GatewayConnection.run(GatewayConnection.java:238) at java.lang.Thread.run(Thread.java:748) ``` ```python >>> from pyspark.sql.functions import atan2 >>> spark.range(1).select(atan2("id", "id")) Traceback (most recent call last): File "<stdin>", line 1, in <module> File "/.../spark/python/pyspark/sql/functions.py", line 78, in _ jc = getattr(sc._jvm.functions, name)(col1._jc if isinstance(col1, Column) else float(col1), ValueError: could not convert string to float: id ``` **After:** ```python >>> from pyspark.sql.functions import lit, ascii >>> spark.range(1).select(lit('a').alias("value")).select(ascii("value")) DataFrame[ascii(value): int] ``` ```python >>> from pyspark.sql.functions import atan2 >>> spark.range(1).select(atan2("id", "id")) DataFrame[ATAN2(id, id): double] ``` Note that, - This PR causes a slight behaviour changes for math functions. For instance, numbers as strings (e.g., `"1"`) were supported as arguments of binary math functions before. After this PR, it recognises it as column names. - I also intentionally didn't document this behaviour changes since we're going ahead for Spark 3.0 and I don't think numbers as strings make much sense in math functions. - There is another exception `when`, which takes string as literal values as below. This PR doeesn't fix this ambiguity. ```python >>> spark.range(1).select(when(lit(True), col("id"))).show() ``` ``` +--------------------------+ |CASE WHEN true THEN id END| +--------------------------+ | 0| +--------------------------+ ``` ```python >>> spark.range(1).select(when(lit(True), "id")).show() ``` ``` +--------------------------+ |CASE WHEN true THEN id END| +--------------------------+ | id| +--------------------------+ ``` This PR also fixes as below: https://github.com/apache/spark/pull/23882 fixed it to: - Rename `_create_function` to `_create_name_function` - Define new `_create_function` to take strings as column names. This PR, I proposes to: - Revert `_create_name_function` name to `_create_function`. - Define new `_create_function_over_column` to take strings as column names. ## How was this patch tested? Some unit tests were added for binary math / string functions. Closes #24121 from HyukjinKwon/SPARK-26979. Authored-by: Hyukjin Kwon <gurwls223@apache.org> Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-19 19:06:10 -04:00
_functions_1_4_over_column = {
# unary math functions
'acos': ':return: inverse cosine of `col`, as if computed by `java.lang.Math.acos()`',
'asin': ':return: inverse sine of `col`, as if computed by `java.lang.Math.asin()`',
'atan': ':return: inverse tangent of `col`, as if computed by `java.lang.Math.atan()`',
'cbrt': 'Computes the cube-root of the given value.',
'ceil': 'Computes the ceiling of the given value.',
'cos': """:param col: angle in radians
:return: cosine of the angle, as if computed by `java.lang.Math.cos()`.""",
'cosh': """:param col: hyperbolic angle
:return: hyperbolic cosine of the angle, as if computed by `java.lang.Math.cosh()`""",
'exp': 'Computes the exponential of the given value.',
'expm1': 'Computes the exponential of the given value minus one.',
'floor': 'Computes the floor of the given value.',
'log': 'Computes the natural logarithm of the given value.',
'log10': 'Computes the logarithm of the given value in Base 10.',
'log1p': 'Computes the natural logarithm of the given value plus one.',
'rint': 'Returns the double value that is closest in value to the argument and' +
' is equal to a mathematical integer.',
'signum': 'Computes the signum of the given value.',
'sin': """:param col: angle in radians
:return: sine of the angle, as if computed by `java.lang.Math.sin()`""",
'sinh': """:param col: hyperbolic angle
:return: hyperbolic sine of the given value,
as if computed by `java.lang.Math.sinh()`""",
'tan': """:param col: angle in radians
:return: tangent of the given value, as if computed by `java.lang.Math.tan()`""",
'tanh': """:param col: hyperbolic angle
:return: hyperbolic tangent of the given value,
as if computed by `java.lang.Math.tanh()`""",
'toDegrees': '.. note:: Deprecated in 2.1, use :func:`degrees` instead.',
'toRadians': '.. note:: Deprecated in 2.1, use :func:`radians` instead.',
'bitwiseNOT': 'Computes bitwise not.',
[SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames - The old implicit would convert RDDs directly to DataFrames, and that added too many methods. - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed Python changes: - toDataFrame -> toDF - Dsl -> functions package - addColumn -> withColumn - renameColumn -> withColumnRenamed - add toDF functions to RDD on SQLContext init - add flatMap to DataFrame Author: Reynold Xin <rxin@databricks.com> Author: Davies Liu <davies@databricks.com> Closes #4556 from rxin/SPARK-5752 and squashes the following commits: 5ef9910 [Reynold Xin] More fix 61d3fca [Reynold Xin] Merge branch 'df5' of github.com:davies/spark into SPARK-5752 ff5832c [Reynold Xin] Fix python 749c675 [Reynold Xin] count(*) fixes. 5806df0 [Reynold Xin] Fix build break again. d941f3d [Reynold Xin] Fixed explode compilation break. fe1267a [Davies Liu] flatMap c4afb8e [Reynold Xin] style d9de47f [Davies Liu] add comment b783994 [Davies Liu] add comment for toDF e2154e5 [Davies Liu] schema() -> schema 3a1004f [Davies Liu] Dsl -> functions, toDF() fb256af [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 0dd74eb [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames 97dd47c [Davies Liu] fix mistake 6168f74 [Davies Liu] fix test 1fc0199 [Davies Liu] fix test a075cd5 [Davies Liu] clean up, toPandas 663d314 [Davies Liu] add test for agg('*') 9e214d5 [Reynold Xin] count(*) fixes. 1ed7136 [Reynold Xin] Fix build break again. 921b2e3 [Reynold Xin] Fixed explode compilation break. 14698d4 [Davies Liu] flatMap ba3e12d [Reynold Xin] style d08c92d [Davies Liu] add comment 5c8b524 [Davies Liu] add comment for toDF a4e5e66 [Davies Liu] schema() -> schema d377fc9 [Davies Liu] Dsl -> functions, toDF() 6b3086c [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 807e8b1 [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames
2015-02-14 02:03:22 -05:00
}
[SPARK-26979][PYTHON][FOLLOW-UP] Make binary math/string functions take string as columns as well ## What changes were proposed in this pull request? This is a followup of https://github.com/apache/spark/pull/23882 to handle binary math/string functions. For instance, see the cases below: **Before:** ```python >>> from pyspark.sql.functions import lit, ascii >>> spark.range(1).select(lit('a').alias("value")).select(ascii("value")) Traceback (most recent call last): File "<stdin>", line 1, in <module> File "/.../spark/python/pyspark/sql/functions.py", line 51, in _ jc = getattr(sc._jvm.functions, name)(col._jc if isinstance(col, Column) else col) File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1286, in __call__ File "/.../spark/python/pyspark/sql/utils.py", line 63, in deco return f(*a, **kw) File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/protocol.py", line 332, in get_return_value py4j.protocol.Py4JError: An error occurred while calling z:org.apache.spark.sql.functions.ascii. Trace: py4j.Py4JException: Method ascii([class java.lang.String]) does not exist at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:318) at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:339) at py4j.Gateway.invoke(Gateway.java:276) at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132) at py4j.commands.CallCommand.execute(CallCommand.java:79) at py4j.GatewayConnection.run(GatewayConnection.java:238) at java.lang.Thread.run(Thread.java:748) ``` ```python >>> from pyspark.sql.functions import atan2 >>> spark.range(1).select(atan2("id", "id")) Traceback (most recent call last): File "<stdin>", line 1, in <module> File "/.../spark/python/pyspark/sql/functions.py", line 78, in _ jc = getattr(sc._jvm.functions, name)(col1._jc if isinstance(col1, Column) else float(col1), ValueError: could not convert string to float: id ``` **After:** ```python >>> from pyspark.sql.functions import lit, ascii >>> spark.range(1).select(lit('a').alias("value")).select(ascii("value")) DataFrame[ascii(value): int] ``` ```python >>> from pyspark.sql.functions import atan2 >>> spark.range(1).select(atan2("id", "id")) DataFrame[ATAN2(id, id): double] ``` Note that, - This PR causes a slight behaviour changes for math functions. For instance, numbers as strings (e.g., `"1"`) were supported as arguments of binary math functions before. After this PR, it recognises it as column names. - I also intentionally didn't document this behaviour changes since we're going ahead for Spark 3.0 and I don't think numbers as strings make much sense in math functions. - There is another exception `when`, which takes string as literal values as below. This PR doeesn't fix this ambiguity. ```python >>> spark.range(1).select(when(lit(True), col("id"))).show() ``` ``` +--------------------------+ |CASE WHEN true THEN id END| +--------------------------+ | 0| +--------------------------+ ``` ```python >>> spark.range(1).select(when(lit(True), "id")).show() ``` ``` +--------------------------+ |CASE WHEN true THEN id END| +--------------------------+ | id| +--------------------------+ ``` This PR also fixes as below: https://github.com/apache/spark/pull/23882 fixed it to: - Rename `_create_function` to `_create_name_function` - Define new `_create_function` to take strings as column names. This PR, I proposes to: - Revert `_create_name_function` name to `_create_function`. - Define new `_create_function_over_column` to take strings as column names. ## How was this patch tested? Some unit tests were added for binary math / string functions. Closes #24121 from HyukjinKwon/SPARK-26979. Authored-by: Hyukjin Kwon <gurwls223@apache.org> Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-19 19:06:10 -04:00
_functions_2_4 = {
'asc_nulls_first': 'Returns a sort expression based on the ascending order of the given' +
' column name, and null values return before non-null values.',
'asc_nulls_last': 'Returns a sort expression based on the ascending order of the given' +
' column name, and null values appear after non-null values.',
'desc_nulls_first': 'Returns a sort expression based on the descending order of the given' +
' column name, and null values appear before non-null values.',
'desc_nulls_last': 'Returns a sort expression based on the descending order of the given' +
' column name, and null values appear after non-null values',
}
_collect_list_doc = """
Aggregate function: returns a list of objects with duplicates.
.. note:: The function is non-deterministic because the order of collected results depends
on order of rows which may be non-deterministic after a shuffle.
>>> df2 = spark.createDataFrame([(2,), (5,), (5,)], ('age',))
>>> df2.agg(collect_list('age')).collect()
[Row(collect_list(age)=[2, 5, 5])]
"""
_collect_set_doc = """
Aggregate function: returns a set of objects with duplicate elements eliminated.
.. note:: The function is non-deterministic because the order of collected results depends
on order of rows which may be non-deterministic after a shuffle.
>>> df2 = spark.createDataFrame([(2,), (5,), (5,)], ('age',))
>>> df2.agg(collect_set('age')).collect()
[Row(collect_set(age)=[5, 2])]
"""
[SPARK-26979][PYTHON][FOLLOW-UP] Make binary math/string functions take string as columns as well ## What changes were proposed in this pull request? This is a followup of https://github.com/apache/spark/pull/23882 to handle binary math/string functions. For instance, see the cases below: **Before:** ```python >>> from pyspark.sql.functions import lit, ascii >>> spark.range(1).select(lit('a').alias("value")).select(ascii("value")) Traceback (most recent call last): File "<stdin>", line 1, in <module> File "/.../spark/python/pyspark/sql/functions.py", line 51, in _ jc = getattr(sc._jvm.functions, name)(col._jc if isinstance(col, Column) else col) File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1286, in __call__ File "/.../spark/python/pyspark/sql/utils.py", line 63, in deco return f(*a, **kw) File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/protocol.py", line 332, in get_return_value py4j.protocol.Py4JError: An error occurred while calling z:org.apache.spark.sql.functions.ascii. Trace: py4j.Py4JException: Method ascii([class java.lang.String]) does not exist at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:318) at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:339) at py4j.Gateway.invoke(Gateway.java:276) at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132) at py4j.commands.CallCommand.execute(CallCommand.java:79) at py4j.GatewayConnection.run(GatewayConnection.java:238) at java.lang.Thread.run(Thread.java:748) ``` ```python >>> from pyspark.sql.functions import atan2 >>> spark.range(1).select(atan2("id", "id")) Traceback (most recent call last): File "<stdin>", line 1, in <module> File "/.../spark/python/pyspark/sql/functions.py", line 78, in _ jc = getattr(sc._jvm.functions, name)(col1._jc if isinstance(col1, Column) else float(col1), ValueError: could not convert string to float: id ``` **After:** ```python >>> from pyspark.sql.functions import lit, ascii >>> spark.range(1).select(lit('a').alias("value")).select(ascii("value")) DataFrame[ascii(value): int] ``` ```python >>> from pyspark.sql.functions import atan2 >>> spark.range(1).select(atan2("id", "id")) DataFrame[ATAN2(id, id): double] ``` Note that, - This PR causes a slight behaviour changes for math functions. For instance, numbers as strings (e.g., `"1"`) were supported as arguments of binary math functions before. After this PR, it recognises it as column names. - I also intentionally didn't document this behaviour changes since we're going ahead for Spark 3.0 and I don't think numbers as strings make much sense in math functions. - There is another exception `when`, which takes string as literal values as below. This PR doeesn't fix this ambiguity. ```python >>> spark.range(1).select(when(lit(True), col("id"))).show() ``` ``` +--------------------------+ |CASE WHEN true THEN id END| +--------------------------+ | 0| +--------------------------+ ``` ```python >>> spark.range(1).select(when(lit(True), "id")).show() ``` ``` +--------------------------+ |CASE WHEN true THEN id END| +--------------------------+ | id| +--------------------------+ ``` This PR also fixes as below: https://github.com/apache/spark/pull/23882 fixed it to: - Rename `_create_function` to `_create_name_function` - Define new `_create_function` to take strings as column names. This PR, I proposes to: - Revert `_create_name_function` name to `_create_function`. - Define new `_create_function_over_column` to take strings as column names. ## How was this patch tested? Some unit tests were added for binary math / string functions. Closes #24121 from HyukjinKwon/SPARK-26979. Authored-by: Hyukjin Kwon <gurwls223@apache.org> Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-19 19:06:10 -04:00
_functions_1_6_over_column = {
# unary math functions
'stddev': 'Aggregate function: alias for stddev_samp.',
'stddev_samp': 'Aggregate function: returns the unbiased sample standard deviation of' +
' the expression in a group.',
'stddev_pop': 'Aggregate function: returns population standard deviation of' +
' the expression in a group.',
'variance': 'Aggregate function: alias for var_samp.',
'var_samp': 'Aggregate function: returns the unbiased sample variance of' +
' the values in a group.',
'var_pop': 'Aggregate function: returns the population variance of the values in a group.',
'skewness': 'Aggregate function: returns the skewness of the values in a group.',
'kurtosis': 'Aggregate function: returns the kurtosis of the values in a group.',
'collect_list': _collect_list_doc,
'collect_set': _collect_set_doc
}
[SPARK-26979][PYTHON][FOLLOW-UP] Make binary math/string functions take string as columns as well ## What changes were proposed in this pull request? This is a followup of https://github.com/apache/spark/pull/23882 to handle binary math/string functions. For instance, see the cases below: **Before:** ```python >>> from pyspark.sql.functions import lit, ascii >>> spark.range(1).select(lit('a').alias("value")).select(ascii("value")) Traceback (most recent call last): File "<stdin>", line 1, in <module> File "/.../spark/python/pyspark/sql/functions.py", line 51, in _ jc = getattr(sc._jvm.functions, name)(col._jc if isinstance(col, Column) else col) File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1286, in __call__ File "/.../spark/python/pyspark/sql/utils.py", line 63, in deco return f(*a, **kw) File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/protocol.py", line 332, in get_return_value py4j.protocol.Py4JError: An error occurred while calling z:org.apache.spark.sql.functions.ascii. Trace: py4j.Py4JException: Method ascii([class java.lang.String]) does not exist at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:318) at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:339) at py4j.Gateway.invoke(Gateway.java:276) at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132) at py4j.commands.CallCommand.execute(CallCommand.java:79) at py4j.GatewayConnection.run(GatewayConnection.java:238) at java.lang.Thread.run(Thread.java:748) ``` ```python >>> from pyspark.sql.functions import atan2 >>> spark.range(1).select(atan2("id", "id")) Traceback (most recent call last): File "<stdin>", line 1, in <module> File "/.../spark/python/pyspark/sql/functions.py", line 78, in _ jc = getattr(sc._jvm.functions, name)(col1._jc if isinstance(col1, Column) else float(col1), ValueError: could not convert string to float: id ``` **After:** ```python >>> from pyspark.sql.functions import lit, ascii >>> spark.range(1).select(lit('a').alias("value")).select(ascii("value")) DataFrame[ascii(value): int] ``` ```python >>> from pyspark.sql.functions import atan2 >>> spark.range(1).select(atan2("id", "id")) DataFrame[ATAN2(id, id): double] ``` Note that, - This PR causes a slight behaviour changes for math functions. For instance, numbers as strings (e.g., `"1"`) were supported as arguments of binary math functions before. After this PR, it recognises it as column names. - I also intentionally didn't document this behaviour changes since we're going ahead for Spark 3.0 and I don't think numbers as strings make much sense in math functions. - There is another exception `when`, which takes string as literal values as below. This PR doeesn't fix this ambiguity. ```python >>> spark.range(1).select(when(lit(True), col("id"))).show() ``` ``` +--------------------------+ |CASE WHEN true THEN id END| +--------------------------+ | 0| +--------------------------+ ``` ```python >>> spark.range(1).select(when(lit(True), "id")).show() ``` ``` +--------------------------+ |CASE WHEN true THEN id END| +--------------------------+ | id| +--------------------------+ ``` This PR also fixes as below: https://github.com/apache/spark/pull/23882 fixed it to: - Rename `_create_function` to `_create_name_function` - Define new `_create_function` to take strings as column names. This PR, I proposes to: - Revert `_create_name_function` name to `_create_function`. - Define new `_create_function_over_column` to take strings as column names. ## How was this patch tested? Some unit tests were added for binary math / string functions. Closes #24121 from HyukjinKwon/SPARK-26979. Authored-by: Hyukjin Kwon <gurwls223@apache.org> Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-19 19:06:10 -04:00
_functions_2_1_over_column = {
# unary math functions
'degrees': """
Converts an angle measured in radians to an approximately equivalent angle
measured in degrees.
:param col: angle in radians
:return: angle in degrees, as if computed by `java.lang.Math.toDegrees()`
""",
'radians': """
Converts an angle measured in degrees to an approximately equivalent angle
measured in radians.
:param col: angle in degrees
:return: angle in radians, as if computed by `java.lang.Math.toRadians()`
""",
}
# math functions that take two arguments as input
_binary_mathfunctions = {
'atan2': """
:param col1: coordinate on y-axis
:param col2: coordinate on x-axis
:return: the `theta` component of the point
(`r`, `theta`)
in polar coordinates that corresponds to the point
(`x`, `y`) in Cartesian coordinates,
as if computed by `java.lang.Math.atan2()`
""",
'hypot': 'Computes ``sqrt(a^2 + b^2)`` without intermediate overflow or underflow.',
[SPARK-8218][SQL] Add binary log math function JIRA: https://issues.apache.org/jira/browse/SPARK-8218 Because there is already `log` unary function defined, the binary log function is called `logarithm` for now. Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #6725 from viirya/expr_binary_log and squashes the following commits: bf96bd9 [Liang-Chi Hsieh] Compare log result in string. 102070d [Liang-Chi Hsieh] Round log result to better comparing in python test. fd01863 [Liang-Chi Hsieh] For comments. beed631 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_binary_log 6089d11 [Liang-Chi Hsieh] Remove unnecessary override. 8cf37b7 [Liang-Chi Hsieh] For comments. bc89597 [Liang-Chi Hsieh] For comments. db7dc38 [Liang-Chi Hsieh] Use ctor instead of companion object. 0634ef7 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_binary_log 1750034 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_binary_log 3d75bfc [Liang-Chi Hsieh] Fix scala style. 5b39c02 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_binary_log 23c54a3 [Liang-Chi Hsieh] Fix scala style. ebc9929 [Liang-Chi Hsieh] Let Logarithm accept one parameter too. 605574d [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_binary_log 21c3bfd [Liang-Chi Hsieh] Fix scala style. c6c187f [Liang-Chi Hsieh] For comments. c795342 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_binary_log f373bac [Liang-Chi Hsieh] Add binary log expression.
2015-06-18 02:31:30 -04:00
'pow': 'Returns the value of the first argument raised to the power of the second argument.',
}
_window_functions = {
'row_number':
"""returns a sequential number starting at 1 within a window partition.""",
'dense_rank':
"""returns the rank of rows within a window partition, without any gaps.
The difference between rank and dense_rank is that dense_rank leaves no gaps in ranking
sequence when there are ties. That is, if you were ranking a competition using dense_rank
and had three people tie for second place, you would say that all three were in second
place and that the next person came in third. Rank would give me sequential numbers, making
the person that came in third place (after the ties) would register as coming in fifth.
This is equivalent to the DENSE_RANK function in SQL.""",
'rank':
"""returns the rank of rows within a window partition.
The difference between rank and dense_rank is that dense_rank leaves no gaps in ranking
sequence when there are ties. That is, if you were ranking a competition using dense_rank
and had three people tie for second place, you would say that all three were in second
place and that the next person came in third. Rank would give me sequential numbers, making
the person that came in third place (after the ties) would register as coming in fifth.
This is equivalent to the RANK function in SQL.""",
'cume_dist':
"""returns the cumulative distribution of values within a window partition,
i.e. the fraction of rows that are below the current row.""",
'percent_rank':
"""returns the relative rank (i.e. percentile) of rows within a window partition.""",
}
[SPARK-22313][PYTHON] Mark/print deprecation warnings as DeprecationWarning for deprecated APIs ## What changes were proposed in this pull request? This PR proposes to mark the existing warnings as `DeprecationWarning` and print out warnings for deprecated functions. This could be actually useful for Spark app developers. I use (old) PyCharm and this IDE can detect this specific `DeprecationWarning` in some cases: **Before** <img src="https://user-images.githubusercontent.com/6477701/31762664-df68d9f8-b4f6-11e7-8773-f0468f70a2cc.png" height="45" /> **After** <img src="https://user-images.githubusercontent.com/6477701/31762662-de4d6868-b4f6-11e7-98dc-3c8446a0c28a.png" height="70" /> For console usage, `DeprecationWarning` is usually disabled (see https://docs.python.org/2/library/warnings.html#warning-categories and https://docs.python.org/3/library/warnings.html#warning-categories): ``` >>> import warnings >>> filter(lambda f: f[2] == DeprecationWarning, warnings.filters) [('ignore', <_sre.SRE_Pattern object at 0x10ba58c00>, <type 'exceptions.DeprecationWarning'>, <_sre.SRE_Pattern object at 0x10bb04138>, 0), ('ignore', None, <type 'exceptions.DeprecationWarning'>, None, 0)] ``` so, it won't actually mess up the terminal much unless it is intended. If this is intendedly enabled, it'd should as below: ``` >>> import warnings >>> warnings.simplefilter('always', DeprecationWarning) >>> >>> from pyspark.sql import functions >>> functions.approxCountDistinct("a") .../spark/python/pyspark/sql/functions.py:232: DeprecationWarning: Deprecated in 2.1, use approx_count_distinct instead. "Deprecated in 2.1, use approx_count_distinct instead.", DeprecationWarning) ... ``` These instances were found by: ``` cd python/pyspark grep -r "Deprecated" . grep -r "deprecated" . grep -r "deprecate" . ``` ## How was this patch tested? Manually tested. Author: hyukjinkwon <gurwls223@gmail.com> Closes #19535 from HyukjinKwon/deprecated-warning.
2017-10-23 23:44:47 -04:00
# Wraps deprecated functions (keys) with the messages (values).
_functions_deprecated = {
}
[SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames - The old implicit would convert RDDs directly to DataFrames, and that added too many methods. - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed Python changes: - toDataFrame -> toDF - Dsl -> functions package - addColumn -> withColumn - renameColumn -> withColumnRenamed - add toDF functions to RDD on SQLContext init - add flatMap to DataFrame Author: Reynold Xin <rxin@databricks.com> Author: Davies Liu <davies@databricks.com> Closes #4556 from rxin/SPARK-5752 and squashes the following commits: 5ef9910 [Reynold Xin] More fix 61d3fca [Reynold Xin] Merge branch 'df5' of github.com:davies/spark into SPARK-5752 ff5832c [Reynold Xin] Fix python 749c675 [Reynold Xin] count(*) fixes. 5806df0 [Reynold Xin] Fix build break again. d941f3d [Reynold Xin] Fixed explode compilation break. fe1267a [Davies Liu] flatMap c4afb8e [Reynold Xin] style d9de47f [Davies Liu] add comment b783994 [Davies Liu] add comment for toDF e2154e5 [Davies Liu] schema() -> schema 3a1004f [Davies Liu] Dsl -> functions, toDF() fb256af [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 0dd74eb [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames 97dd47c [Davies Liu] fix mistake 6168f74 [Davies Liu] fix test 1fc0199 [Davies Liu] fix test a075cd5 [Davies Liu] clean up, toPandas 663d314 [Davies Liu] add test for agg('*') 9e214d5 [Reynold Xin] count(*) fixes. 1ed7136 [Reynold Xin] Fix build break again. 921b2e3 [Reynold Xin] Fixed explode compilation break. 14698d4 [Davies Liu] flatMap ba3e12d [Reynold Xin] style d08c92d [Davies Liu] add comment 5c8b524 [Davies Liu] add comment for toDF a4e5e66 [Davies Liu] schema() -> schema d377fc9 [Davies Liu] Dsl -> functions, toDF() 6b3086c [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 807e8b1 [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames
2015-02-14 02:03:22 -05:00
for _name, _doc in _functions.items():
globals()[_name] = since(1.3)(_create_function(_name, _doc))
[SPARK-26979][PYTHON][FOLLOW-UP] Make binary math/string functions take string as columns as well ## What changes were proposed in this pull request? This is a followup of https://github.com/apache/spark/pull/23882 to handle binary math/string functions. For instance, see the cases below: **Before:** ```python >>> from pyspark.sql.functions import lit, ascii >>> spark.range(1).select(lit('a').alias("value")).select(ascii("value")) Traceback (most recent call last): File "<stdin>", line 1, in <module> File "/.../spark/python/pyspark/sql/functions.py", line 51, in _ jc = getattr(sc._jvm.functions, name)(col._jc if isinstance(col, Column) else col) File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1286, in __call__ File "/.../spark/python/pyspark/sql/utils.py", line 63, in deco return f(*a, **kw) File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/protocol.py", line 332, in get_return_value py4j.protocol.Py4JError: An error occurred while calling z:org.apache.spark.sql.functions.ascii. Trace: py4j.Py4JException: Method ascii([class java.lang.String]) does not exist at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:318) at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:339) at py4j.Gateway.invoke(Gateway.java:276) at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132) at py4j.commands.CallCommand.execute(CallCommand.java:79) at py4j.GatewayConnection.run(GatewayConnection.java:238) at java.lang.Thread.run(Thread.java:748) ``` ```python >>> from pyspark.sql.functions import atan2 >>> spark.range(1).select(atan2("id", "id")) Traceback (most recent call last): File "<stdin>", line 1, in <module> File "/.../spark/python/pyspark/sql/functions.py", line 78, in _ jc = getattr(sc._jvm.functions, name)(col1._jc if isinstance(col1, Column) else float(col1), ValueError: could not convert string to float: id ``` **After:** ```python >>> from pyspark.sql.functions import lit, ascii >>> spark.range(1).select(lit('a').alias("value")).select(ascii("value")) DataFrame[ascii(value): int] ``` ```python >>> from pyspark.sql.functions import atan2 >>> spark.range(1).select(atan2("id", "id")) DataFrame[ATAN2(id, id): double] ``` Note that, - This PR causes a slight behaviour changes for math functions. For instance, numbers as strings (e.g., `"1"`) were supported as arguments of binary math functions before. After this PR, it recognises it as column names. - I also intentionally didn't document this behaviour changes since we're going ahead for Spark 3.0 and I don't think numbers as strings make much sense in math functions. - There is another exception `when`, which takes string as literal values as below. This PR doeesn't fix this ambiguity. ```python >>> spark.range(1).select(when(lit(True), col("id"))).show() ``` ``` +--------------------------+ |CASE WHEN true THEN id END| +--------------------------+ | 0| +--------------------------+ ``` ```python >>> spark.range(1).select(when(lit(True), "id")).show() ``` ``` +--------------------------+ |CASE WHEN true THEN id END| +--------------------------+ | id| +--------------------------+ ``` This PR also fixes as below: https://github.com/apache/spark/pull/23882 fixed it to: - Rename `_create_function` to `_create_name_function` - Define new `_create_function` to take strings as column names. This PR, I proposes to: - Revert `_create_name_function` name to `_create_function`. - Define new `_create_function_over_column` to take strings as column names. ## How was this patch tested? Some unit tests were added for binary math / string functions. Closes #24121 from HyukjinKwon/SPARK-26979. Authored-by: Hyukjin Kwon <gurwls223@apache.org> Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-19 19:06:10 -04:00
for _name, _doc in _functions_over_column.items():
globals()[_name] = since(1.3)(_create_function_over_column(_name, _doc))
for _name, _doc in _functions_1_4_over_column.items():
globals()[_name] = since(1.4)(_create_function_over_column(_name, _doc))
for _name, _doc in _binary_mathfunctions.items():
globals()[_name] = since(1.4)(_create_binary_mathfunction(_name, _doc))
for _name, _doc in _window_functions.items():
globals()[_name] = since(1.6)(_create_window_function(_name, _doc))
[SPARK-26979][PYTHON][FOLLOW-UP] Make binary math/string functions take string as columns as well ## What changes were proposed in this pull request? This is a followup of https://github.com/apache/spark/pull/23882 to handle binary math/string functions. For instance, see the cases below: **Before:** ```python >>> from pyspark.sql.functions import lit, ascii >>> spark.range(1).select(lit('a').alias("value")).select(ascii("value")) Traceback (most recent call last): File "<stdin>", line 1, in <module> File "/.../spark/python/pyspark/sql/functions.py", line 51, in _ jc = getattr(sc._jvm.functions, name)(col._jc if isinstance(col, Column) else col) File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1286, in __call__ File "/.../spark/python/pyspark/sql/utils.py", line 63, in deco return f(*a, **kw) File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/protocol.py", line 332, in get_return_value py4j.protocol.Py4JError: An error occurred while calling z:org.apache.spark.sql.functions.ascii. Trace: py4j.Py4JException: Method ascii([class java.lang.String]) does not exist at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:318) at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:339) at py4j.Gateway.invoke(Gateway.java:276) at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132) at py4j.commands.CallCommand.execute(CallCommand.java:79) at py4j.GatewayConnection.run(GatewayConnection.java:238) at java.lang.Thread.run(Thread.java:748) ``` ```python >>> from pyspark.sql.functions import atan2 >>> spark.range(1).select(atan2("id", "id")) Traceback (most recent call last): File "<stdin>", line 1, in <module> File "/.../spark/python/pyspark/sql/functions.py", line 78, in _ jc = getattr(sc._jvm.functions, name)(col1._jc if isinstance(col1, Column) else float(col1), ValueError: could not convert string to float: id ``` **After:** ```python >>> from pyspark.sql.functions import lit, ascii >>> spark.range(1).select(lit('a').alias("value")).select(ascii("value")) DataFrame[ascii(value): int] ``` ```python >>> from pyspark.sql.functions import atan2 >>> spark.range(1).select(atan2("id", "id")) DataFrame[ATAN2(id, id): double] ``` Note that, - This PR causes a slight behaviour changes for math functions. For instance, numbers as strings (e.g., `"1"`) were supported as arguments of binary math functions before. After this PR, it recognises it as column names. - I also intentionally didn't document this behaviour changes since we're going ahead for Spark 3.0 and I don't think numbers as strings make much sense in math functions. - There is another exception `when`, which takes string as literal values as below. This PR doeesn't fix this ambiguity. ```python >>> spark.range(1).select(when(lit(True), col("id"))).show() ``` ``` +--------------------------+ |CASE WHEN true THEN id END| +--------------------------+ | 0| +--------------------------+ ``` ```python >>> spark.range(1).select(when(lit(True), "id")).show() ``` ``` +--------------------------+ |CASE WHEN true THEN id END| +--------------------------+ | id| +--------------------------+ ``` This PR also fixes as below: https://github.com/apache/spark/pull/23882 fixed it to: - Rename `_create_function` to `_create_name_function` - Define new `_create_function` to take strings as column names. This PR, I proposes to: - Revert `_create_name_function` name to `_create_function`. - Define new `_create_function_over_column` to take strings as column names. ## How was this patch tested? Some unit tests were added for binary math / string functions. Closes #24121 from HyukjinKwon/SPARK-26979. Authored-by: Hyukjin Kwon <gurwls223@apache.org> Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-19 19:06:10 -04:00
for _name, _doc in _functions_1_6_over_column.items():
globals()[_name] = since(1.6)(_create_function_over_column(_name, _doc))
for _name, _doc in _functions_2_1_over_column.items():
globals()[_name] = since(2.1)(_create_function_over_column(_name, _doc))
[SPARK-22313][PYTHON] Mark/print deprecation warnings as DeprecationWarning for deprecated APIs ## What changes were proposed in this pull request? This PR proposes to mark the existing warnings as `DeprecationWarning` and print out warnings for deprecated functions. This could be actually useful for Spark app developers. I use (old) PyCharm and this IDE can detect this specific `DeprecationWarning` in some cases: **Before** <img src="https://user-images.githubusercontent.com/6477701/31762664-df68d9f8-b4f6-11e7-8773-f0468f70a2cc.png" height="45" /> **After** <img src="https://user-images.githubusercontent.com/6477701/31762662-de4d6868-b4f6-11e7-98dc-3c8446a0c28a.png" height="70" /> For console usage, `DeprecationWarning` is usually disabled (see https://docs.python.org/2/library/warnings.html#warning-categories and https://docs.python.org/3/library/warnings.html#warning-categories): ``` >>> import warnings >>> filter(lambda f: f[2] == DeprecationWarning, warnings.filters) [('ignore', <_sre.SRE_Pattern object at 0x10ba58c00>, <type 'exceptions.DeprecationWarning'>, <_sre.SRE_Pattern object at 0x10bb04138>, 0), ('ignore', None, <type 'exceptions.DeprecationWarning'>, None, 0)] ``` so, it won't actually mess up the terminal much unless it is intended. If this is intendedly enabled, it'd should as below: ``` >>> import warnings >>> warnings.simplefilter('always', DeprecationWarning) >>> >>> from pyspark.sql import functions >>> functions.approxCountDistinct("a") .../spark/python/pyspark/sql/functions.py:232: DeprecationWarning: Deprecated in 2.1, use approx_count_distinct instead. "Deprecated in 2.1, use approx_count_distinct instead.", DeprecationWarning) ... ``` These instances were found by: ``` cd python/pyspark grep -r "Deprecated" . grep -r "deprecated" . grep -r "deprecate" . ``` ## How was this patch tested? Manually tested. Author: hyukjinkwon <gurwls223@gmail.com> Closes #19535 from HyukjinKwon/deprecated-warning.
2017-10-23 23:44:47 -04:00
for _name, _message in _functions_deprecated.items():
globals()[_name] = _wrap_deprecated_function(globals()[_name], _message)
[SPARK-26979][PYTHON][FOLLOW-UP] Make binary math/string functions take string as columns as well ## What changes were proposed in this pull request? This is a followup of https://github.com/apache/spark/pull/23882 to handle binary math/string functions. For instance, see the cases below: **Before:** ```python >>> from pyspark.sql.functions import lit, ascii >>> spark.range(1).select(lit('a').alias("value")).select(ascii("value")) Traceback (most recent call last): File "<stdin>", line 1, in <module> File "/.../spark/python/pyspark/sql/functions.py", line 51, in _ jc = getattr(sc._jvm.functions, name)(col._jc if isinstance(col, Column) else col) File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1286, in __call__ File "/.../spark/python/pyspark/sql/utils.py", line 63, in deco return f(*a, **kw) File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/protocol.py", line 332, in get_return_value py4j.protocol.Py4JError: An error occurred while calling z:org.apache.spark.sql.functions.ascii. Trace: py4j.Py4JException: Method ascii([class java.lang.String]) does not exist at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:318) at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:339) at py4j.Gateway.invoke(Gateway.java:276) at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132) at py4j.commands.CallCommand.execute(CallCommand.java:79) at py4j.GatewayConnection.run(GatewayConnection.java:238) at java.lang.Thread.run(Thread.java:748) ``` ```python >>> from pyspark.sql.functions import atan2 >>> spark.range(1).select(atan2("id", "id")) Traceback (most recent call last): File "<stdin>", line 1, in <module> File "/.../spark/python/pyspark/sql/functions.py", line 78, in _ jc = getattr(sc._jvm.functions, name)(col1._jc if isinstance(col1, Column) else float(col1), ValueError: could not convert string to float: id ``` **After:** ```python >>> from pyspark.sql.functions import lit, ascii >>> spark.range(1).select(lit('a').alias("value")).select(ascii("value")) DataFrame[ascii(value): int] ``` ```python >>> from pyspark.sql.functions import atan2 >>> spark.range(1).select(atan2("id", "id")) DataFrame[ATAN2(id, id): double] ``` Note that, - This PR causes a slight behaviour changes for math functions. For instance, numbers as strings (e.g., `"1"`) were supported as arguments of binary math functions before. After this PR, it recognises it as column names. - I also intentionally didn't document this behaviour changes since we're going ahead for Spark 3.0 and I don't think numbers as strings make much sense in math functions. - There is another exception `when`, which takes string as literal values as below. This PR doeesn't fix this ambiguity. ```python >>> spark.range(1).select(when(lit(True), col("id"))).show() ``` ``` +--------------------------+ |CASE WHEN true THEN id END| +--------------------------+ | 0| +--------------------------+ ``` ```python >>> spark.range(1).select(when(lit(True), "id")).show() ``` ``` +--------------------------+ |CASE WHEN true THEN id END| +--------------------------+ | id| +--------------------------+ ``` This PR also fixes as below: https://github.com/apache/spark/pull/23882 fixed it to: - Rename `_create_function` to `_create_name_function` - Define new `_create_function` to take strings as column names. This PR, I proposes to: - Revert `_create_name_function` name to `_create_function`. - Define new `_create_function_over_column` to take strings as column names. ## How was this patch tested? Some unit tests were added for binary math / string functions. Closes #24121 from HyukjinKwon/SPARK-26979. Authored-by: Hyukjin Kwon <gurwls223@apache.org> Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-19 19:06:10 -04:00
for _name, _doc in _functions_2_4.items():
globals()[_name] = since(2.4)(_create_function(_name, _doc))
[SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames - The old implicit would convert RDDs directly to DataFrames, and that added too many methods. - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed Python changes: - toDataFrame -> toDF - Dsl -> functions package - addColumn -> withColumn - renameColumn -> withColumnRenamed - add toDF functions to RDD on SQLContext init - add flatMap to DataFrame Author: Reynold Xin <rxin@databricks.com> Author: Davies Liu <davies@databricks.com> Closes #4556 from rxin/SPARK-5752 and squashes the following commits: 5ef9910 [Reynold Xin] More fix 61d3fca [Reynold Xin] Merge branch 'df5' of github.com:davies/spark into SPARK-5752 ff5832c [Reynold Xin] Fix python 749c675 [Reynold Xin] count(*) fixes. 5806df0 [Reynold Xin] Fix build break again. d941f3d [Reynold Xin] Fixed explode compilation break. fe1267a [Davies Liu] flatMap c4afb8e [Reynold Xin] style d9de47f [Davies Liu] add comment b783994 [Davies Liu] add comment for toDF e2154e5 [Davies Liu] schema() -> schema 3a1004f [Davies Liu] Dsl -> functions, toDF() fb256af [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 0dd74eb [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames 97dd47c [Davies Liu] fix mistake 6168f74 [Davies Liu] fix test 1fc0199 [Davies Liu] fix test a075cd5 [Davies Liu] clean up, toPandas 663d314 [Davies Liu] add test for agg('*') 9e214d5 [Reynold Xin] count(*) fixes. 1ed7136 [Reynold Xin] Fix build break again. 921b2e3 [Reynold Xin] Fixed explode compilation break. 14698d4 [Davies Liu] flatMap ba3e12d [Reynold Xin] style d08c92d [Davies Liu] add comment 5c8b524 [Davies Liu] add comment for toDF a4e5e66 [Davies Liu] schema() -> schema d377fc9 [Davies Liu] Dsl -> functions, toDF() 6b3086c [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 807e8b1 [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames
2015-02-14 02:03:22 -05:00
del _name, _doc
@since(2.1)
def approx_count_distinct(col, rsd=None):
"""Aggregate function: returns a new :class:`Column` for approximate distinct count of
column `col`.
:param rsd: maximum estimation error allowed (default = 0.05). For rsd < 0.01, it is more
efficient to use :func:`countDistinct`
>>> df.agg(approx_count_distinct(df.age).alias('distinct_ages')).collect()
[Row(distinct_ages=2)]
"""
sc = SparkContext._active_spark_context
if rsd is None:
jc = sc._jvm.functions.approx_count_distinct(_to_java_column(col))
else:
jc = sc._jvm.functions.approx_count_distinct(_to_java_column(col), rsd)
return Column(jc)
@since(1.6)
def broadcast(df):
"""Marks a DataFrame as small enough for use in broadcast joins."""
sc = SparkContext._active_spark_context
return DataFrame(sc._jvm.functions.broadcast(df._jdf), df.sql_ctx)
@since(1.4)
def coalesce(*cols):
"""Returns the first column that is not null.
>>> cDf = spark.createDataFrame([(None, None), (1, None), (None, 2)], ("a", "b"))
>>> cDf.show()
+----+----+
| a| b|
+----+----+
|null|null|
| 1|null|
|null| 2|
+----+----+
>>> cDf.select(coalesce(cDf["a"], cDf["b"])).show()
+--------------+
|coalesce(a, b)|
+--------------+
| null|
| 1|
| 2|
+--------------+
>>> cDf.select('*', coalesce(cDf["a"], lit(0.0))).show()
+----+----+----------------+
| a| b|coalesce(a, 0.0)|
+----+----+----------------+
|null|null| 0.0|
| 1|null| 1.0|
|null| 2| 0.0|
+----+----+----------------+
"""
sc = SparkContext._active_spark_context
jc = sc._jvm.functions.coalesce(_to_seq(sc, cols, _to_java_column))
return Column(jc)
@since(1.6)
def corr(col1, col2):
"""Returns a new :class:`Column` for the Pearson Correlation Coefficient for ``col1``
and ``col2``.
>>> a = range(20)
>>> b = [2 * x for x in range(20)]
>>> df = spark.createDataFrame(zip(a, b), ["a", "b"])
>>> df.agg(corr("a", "b").alias('c')).collect()
[Row(c=1.0)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.corr(_to_java_column(col1), _to_java_column(col2)))
@since(2.0)
def covar_pop(col1, col2):
"""Returns a new :class:`Column` for the population covariance of ``col1`` and ``col2``.
>>> a = [1] * 10
>>> b = [1] * 10
>>> df = spark.createDataFrame(zip(a, b), ["a", "b"])
>>> df.agg(covar_pop("a", "b").alias('c')).collect()
[Row(c=0.0)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.covar_pop(_to_java_column(col1), _to_java_column(col2)))
@since(2.0)
def covar_samp(col1, col2):
"""Returns a new :class:`Column` for the sample covariance of ``col1`` and ``col2``.
>>> a = [1] * 10
>>> b = [1] * 10
>>> df = spark.createDataFrame(zip(a, b), ["a", "b"])
>>> df.agg(covar_samp("a", "b").alias('c')).collect()
[Row(c=0.0)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.covar_samp(_to_java_column(col1), _to_java_column(col2)))
@since(1.3)
[SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames - The old implicit would convert RDDs directly to DataFrames, and that added too many methods. - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed Python changes: - toDataFrame -> toDF - Dsl -> functions package - addColumn -> withColumn - renameColumn -> withColumnRenamed - add toDF functions to RDD on SQLContext init - add flatMap to DataFrame Author: Reynold Xin <rxin@databricks.com> Author: Davies Liu <davies@databricks.com> Closes #4556 from rxin/SPARK-5752 and squashes the following commits: 5ef9910 [Reynold Xin] More fix 61d3fca [Reynold Xin] Merge branch 'df5' of github.com:davies/spark into SPARK-5752 ff5832c [Reynold Xin] Fix python 749c675 [Reynold Xin] count(*) fixes. 5806df0 [Reynold Xin] Fix build break again. d941f3d [Reynold Xin] Fixed explode compilation break. fe1267a [Davies Liu] flatMap c4afb8e [Reynold Xin] style d9de47f [Davies Liu] add comment b783994 [Davies Liu] add comment for toDF e2154e5 [Davies Liu] schema() -> schema 3a1004f [Davies Liu] Dsl -> functions, toDF() fb256af [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 0dd74eb [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames 97dd47c [Davies Liu] fix mistake 6168f74 [Davies Liu] fix test 1fc0199 [Davies Liu] fix test a075cd5 [Davies Liu] clean up, toPandas 663d314 [Davies Liu] add test for agg('*') 9e214d5 [Reynold Xin] count(*) fixes. 1ed7136 [Reynold Xin] Fix build break again. 921b2e3 [Reynold Xin] Fixed explode compilation break. 14698d4 [Davies Liu] flatMap ba3e12d [Reynold Xin] style d08c92d [Davies Liu] add comment 5c8b524 [Davies Liu] add comment for toDF a4e5e66 [Davies Liu] schema() -> schema d377fc9 [Davies Liu] Dsl -> functions, toDF() 6b3086c [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 807e8b1 [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames
2015-02-14 02:03:22 -05:00
def countDistinct(col, *cols):
"""Returns a new :class:`Column` for distinct count of ``col`` or ``cols``.
[SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames - The old implicit would convert RDDs directly to DataFrames, and that added too many methods. - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed Python changes: - toDataFrame -> toDF - Dsl -> functions package - addColumn -> withColumn - renameColumn -> withColumnRenamed - add toDF functions to RDD on SQLContext init - add flatMap to DataFrame Author: Reynold Xin <rxin@databricks.com> Author: Davies Liu <davies@databricks.com> Closes #4556 from rxin/SPARK-5752 and squashes the following commits: 5ef9910 [Reynold Xin] More fix 61d3fca [Reynold Xin] Merge branch 'df5' of github.com:davies/spark into SPARK-5752 ff5832c [Reynold Xin] Fix python 749c675 [Reynold Xin] count(*) fixes. 5806df0 [Reynold Xin] Fix build break again. d941f3d [Reynold Xin] Fixed explode compilation break. fe1267a [Davies Liu] flatMap c4afb8e [Reynold Xin] style d9de47f [Davies Liu] add comment b783994 [Davies Liu] add comment for toDF e2154e5 [Davies Liu] schema() -> schema 3a1004f [Davies Liu] Dsl -> functions, toDF() fb256af [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 0dd74eb [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames 97dd47c [Davies Liu] fix mistake 6168f74 [Davies Liu] fix test 1fc0199 [Davies Liu] fix test a075cd5 [Davies Liu] clean up, toPandas 663d314 [Davies Liu] add test for agg('*') 9e214d5 [Reynold Xin] count(*) fixes. 1ed7136 [Reynold Xin] Fix build break again. 921b2e3 [Reynold Xin] Fixed explode compilation break. 14698d4 [Davies Liu] flatMap ba3e12d [Reynold Xin] style d08c92d [Davies Liu] add comment 5c8b524 [Davies Liu] add comment for toDF a4e5e66 [Davies Liu] schema() -> schema d377fc9 [Davies Liu] Dsl -> functions, toDF() 6b3086c [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 807e8b1 [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames
2015-02-14 02:03:22 -05:00
>>> df.agg(countDistinct(df.age, df.name).alias('c')).collect()
[Row(c=2)]
>>> df.agg(countDistinct("age", "name").alias('c')).collect()
[Row(c=2)]
"""
sc = SparkContext._active_spark_context
jc = sc._jvm.functions.countDistinct(_to_java_column(col), _to_seq(sc, cols, _to_java_column))
[SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames - The old implicit would convert RDDs directly to DataFrames, and that added too many methods. - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed Python changes: - toDataFrame -> toDF - Dsl -> functions package - addColumn -> withColumn - renameColumn -> withColumnRenamed - add toDF functions to RDD on SQLContext init - add flatMap to DataFrame Author: Reynold Xin <rxin@databricks.com> Author: Davies Liu <davies@databricks.com> Closes #4556 from rxin/SPARK-5752 and squashes the following commits: 5ef9910 [Reynold Xin] More fix 61d3fca [Reynold Xin] Merge branch 'df5' of github.com:davies/spark into SPARK-5752 ff5832c [Reynold Xin] Fix python 749c675 [Reynold Xin] count(*) fixes. 5806df0 [Reynold Xin] Fix build break again. d941f3d [Reynold Xin] Fixed explode compilation break. fe1267a [Davies Liu] flatMap c4afb8e [Reynold Xin] style d9de47f [Davies Liu] add comment b783994 [Davies Liu] add comment for toDF e2154e5 [Davies Liu] schema() -> schema 3a1004f [Davies Liu] Dsl -> functions, toDF() fb256af [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 0dd74eb [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames 97dd47c [Davies Liu] fix mistake 6168f74 [Davies Liu] fix test 1fc0199 [Davies Liu] fix test a075cd5 [Davies Liu] clean up, toPandas 663d314 [Davies Liu] add test for agg('*') 9e214d5 [Reynold Xin] count(*) fixes. 1ed7136 [Reynold Xin] Fix build break again. 921b2e3 [Reynold Xin] Fixed explode compilation break. 14698d4 [Davies Liu] flatMap ba3e12d [Reynold Xin] style d08c92d [Davies Liu] add comment 5c8b524 [Davies Liu] add comment for toDF a4e5e66 [Davies Liu] schema() -> schema d377fc9 [Davies Liu] Dsl -> functions, toDF() 6b3086c [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 807e8b1 [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames
2015-02-14 02:03:22 -05:00
return Column(jc)
@since(1.3)
def first(col, ignorenulls=False):
"""Aggregate function: returns the first value in a group.
The function by default returns the first values it sees. It will return the first non-null
value it sees when ignoreNulls is set to true. If all values are null, then null is returned.
.. note:: The function is non-deterministic because its results depends on order of rows which
may be non-deterministic after a shuffle.
"""
sc = SparkContext._active_spark_context
jc = sc._jvm.functions.first(_to_java_column(col), ignorenulls)
return Column(jc)
@since(2.0)
def grouping(col):
"""
Aggregate function: indicates whether a specified column in a GROUP BY list is aggregated
or not, returns 1 for aggregated or 0 for not aggregated in the result set.
>>> df.cube("name").agg(grouping("name"), sum("age")).orderBy("name").show()
+-----+--------------+--------+
| name|grouping(name)|sum(age)|
+-----+--------------+--------+
| null| 1| 7|
|Alice| 0| 2|
| Bob| 0| 5|
+-----+--------------+--------+
"""
sc = SparkContext._active_spark_context
jc = sc._jvm.functions.grouping(_to_java_column(col))
return Column(jc)
@since(2.0)
def grouping_id(*cols):
"""
Aggregate function: returns the level of grouping, equals to
(grouping(c1) << (n-1)) + (grouping(c2) << (n-2)) + ... + grouping(cn)
.. note:: The list of columns should match with grouping columns exactly, or empty (means all
[MINOR][DOCUMENTATION] Fix some minor descriptions in functions consistently with expressions ## What changes were proposed in this pull request? This PR proposes to improve documentation and fix some descriptions equivalent to several minor fixes identified in https://github.com/apache/spark/pull/15677 Also, this suggests to change `Note:` and `NOTE:` to `.. note::` consistently with the others which marks up pretty. ## How was this patch tested? Jenkins tests and manually. For PySpark, `Note:` and `NOTE:` to `.. note::` make the document as below: **From** ![2016-11-04 6 53 35](https://cloud.githubusercontent.com/assets/6477701/20002648/42989922-a2c5-11e6-8a32-b73eda49e8c3.png) ![2016-11-04 6 53 45](https://cloud.githubusercontent.com/assets/6477701/20002650/429fb310-a2c5-11e6-926b-e030d7eb0185.png) ![2016-11-04 6 54 11](https://cloud.githubusercontent.com/assets/6477701/20002649/429d570a-a2c5-11e6-9e7e-44090f337e32.png) ![2016-11-04 6 53 51](https://cloud.githubusercontent.com/assets/6477701/20002647/4297fc74-a2c5-11e6-801a-b89fbcbfca44.png) ![2016-11-04 6 53 51](https://cloud.githubusercontent.com/assets/6477701/20002697/749f5780-a2c5-11e6-835f-022e1f2f82e3.png) **To** ![2016-11-04 7 03 48](https://cloud.githubusercontent.com/assets/6477701/20002659/4961b504-a2c5-11e6-9ee0-ef0751482f47.png) ![2016-11-04 7 04 03](https://cloud.githubusercontent.com/assets/6477701/20002660/49871d3a-a2c5-11e6-85ea-d9a5d11efeff.png) ![2016-11-04 7 04 28](https://cloud.githubusercontent.com/assets/6477701/20002662/498e0f14-a2c5-11e6-803d-c0c5aeda4153.png) ![2016-11-04 7 33 39](https://cloud.githubusercontent.com/assets/6477701/20002731/a76e30d2-a2c5-11e6-993b-0481b8342d6b.png) ![2016-11-04 7 33 39](https://cloud.githubusercontent.com/assets/6477701/20002731/a76e30d2-a2c5-11e6-993b-0481b8342d6b.png) Author: hyukjinkwon <gurwls223@gmail.com> Closes #15765 from HyukjinKwon/minor-function-doc.
2016-11-06 00:47:33 -04:00
the grouping columns).
>>> df.cube("name").agg(grouping_id(), sum("age")).orderBy("name").show()
+-----+-------------+--------+
| name|grouping_id()|sum(age)|
+-----+-------------+--------+
| null| 1| 7|
|Alice| 0| 2|
| Bob| 0| 5|
+-----+-------------+--------+
"""
sc = SparkContext._active_spark_context
jc = sc._jvm.functions.grouping_id(_to_seq(sc, cols, _to_java_column))
return Column(jc)
@since(1.6)
def input_file_name():
"""Creates a string column for the file name of the current Spark task.
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.input_file_name())
@since(1.6)
def isnan(col):
"""An expression that returns true iff the column is NaN.
>>> df = spark.createDataFrame([(1.0, float('nan')), (float('nan'), 2.0)], ("a", "b"))
>>> df.select(isnan("a").alias("r1"), isnan(df.a).alias("r2")).collect()
[Row(r1=False, r2=False), Row(r1=True, r2=True)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.isnan(_to_java_column(col)))
@since(1.6)
def isnull(col):
"""An expression that returns true iff the column is null.
>>> df = spark.createDataFrame([(1, None), (None, 2)], ("a", "b"))
>>> df.select(isnull("a").alias("r1"), isnull(df.a).alias("r2")).collect()
[Row(r1=False, r2=False), Row(r1=True, r2=True)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.isnull(_to_java_column(col)))
@since(1.3)
def last(col, ignorenulls=False):
"""Aggregate function: returns the last value in a group.
The function by default returns the last values it sees. It will return the last non-null
value it sees when ignoreNulls is set to true. If all values are null, then null is returned.
.. note:: The function is non-deterministic because its results depends on order of rows
which may be non-deterministic after a shuffle.
"""
sc = SparkContext._active_spark_context
jc = sc._jvm.functions.last(_to_java_column(col), ignorenulls)
return Column(jc)
@since(1.6)
def monotonically_increasing_id():
"""A column that generates monotonically increasing 64-bit integers.
The generated ID is guaranteed to be monotonically increasing and unique, but not consecutive.
The current implementation puts the partition ID in the upper 31 bits, and the record number
within each partition in the lower 33 bits. The assumption is that the data frame has
less than 1 billion partitions, and each partition has less than 8 billion records.
.. note:: The function is non-deterministic because its result depends on partition IDs.
As an example, consider a :class:`DataFrame` with two partitions, each with 3 records.
This expression would return the following IDs:
0, 1, 2, 8589934592 (1L << 33), 8589934593, 8589934594.
>>> df0 = sc.parallelize(range(2), 2).mapPartitions(lambda x: [(1,), (2,), (3,)]).toDF(['col1'])
>>> df0.select(monotonically_increasing_id().alias('id')).collect()
[Row(id=0), Row(id=1), Row(id=2), Row(id=8589934592), Row(id=8589934593), Row(id=8589934594)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.monotonically_increasing_id())
@since(1.6)
def nanvl(col1, col2):
"""Returns col1 if it is not NaN, or col2 if col1 is NaN.
Both inputs should be floating point columns (:class:`DoubleType` or :class:`FloatType`).
>>> df = spark.createDataFrame([(1.0, float('nan')), (float('nan'), 2.0)], ("a", "b"))
>>> df.select(nanvl("a", "b").alias("r1"), nanvl(df.a, df.b).alias("r2")).collect()
[Row(r1=1.0, r2=1.0), Row(r1=2.0, r2=2.0)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.nanvl(_to_java_column(col1), _to_java_column(col2)))
@ignore_unicode_prefix
@since(1.4)
def rand(seed=None):
[MINOR][DOCUMENTATION] Fix some minor descriptions in functions consistently with expressions ## What changes were proposed in this pull request? This PR proposes to improve documentation and fix some descriptions equivalent to several minor fixes identified in https://github.com/apache/spark/pull/15677 Also, this suggests to change `Note:` and `NOTE:` to `.. note::` consistently with the others which marks up pretty. ## How was this patch tested? Jenkins tests and manually. For PySpark, `Note:` and `NOTE:` to `.. note::` make the document as below: **From** ![2016-11-04 6 53 35](https://cloud.githubusercontent.com/assets/6477701/20002648/42989922-a2c5-11e6-8a32-b73eda49e8c3.png) ![2016-11-04 6 53 45](https://cloud.githubusercontent.com/assets/6477701/20002650/429fb310-a2c5-11e6-926b-e030d7eb0185.png) ![2016-11-04 6 54 11](https://cloud.githubusercontent.com/assets/6477701/20002649/429d570a-a2c5-11e6-9e7e-44090f337e32.png) ![2016-11-04 6 53 51](https://cloud.githubusercontent.com/assets/6477701/20002647/4297fc74-a2c5-11e6-801a-b89fbcbfca44.png) ![2016-11-04 6 53 51](https://cloud.githubusercontent.com/assets/6477701/20002697/749f5780-a2c5-11e6-835f-022e1f2f82e3.png) **To** ![2016-11-04 7 03 48](https://cloud.githubusercontent.com/assets/6477701/20002659/4961b504-a2c5-11e6-9ee0-ef0751482f47.png) ![2016-11-04 7 04 03](https://cloud.githubusercontent.com/assets/6477701/20002660/49871d3a-a2c5-11e6-85ea-d9a5d11efeff.png) ![2016-11-04 7 04 28](https://cloud.githubusercontent.com/assets/6477701/20002662/498e0f14-a2c5-11e6-803d-c0c5aeda4153.png) ![2016-11-04 7 33 39](https://cloud.githubusercontent.com/assets/6477701/20002731/a76e30d2-a2c5-11e6-993b-0481b8342d6b.png) ![2016-11-04 7 33 39](https://cloud.githubusercontent.com/assets/6477701/20002731/a76e30d2-a2c5-11e6-993b-0481b8342d6b.png) Author: hyukjinkwon <gurwls223@gmail.com> Closes #15765 from HyukjinKwon/minor-function-doc.
2016-11-06 00:47:33 -04:00
"""Generates a random column with independent and identically distributed (i.i.d.) samples
from U[0.0, 1.0].
.. note:: The function is non-deterministic in general case.
>>> df.withColumn('rand', rand(seed=42) * 3).collect()
[Row(age=2, name=u'Alice', rand=2.4052597283576684),
Row(age=5, name=u'Bob', rand=2.3913904055683974)]
"""
sc = SparkContext._active_spark_context
if seed is not None:
jc = sc._jvm.functions.rand(seed)
else:
jc = sc._jvm.functions.rand()
return Column(jc)
@ignore_unicode_prefix
@since(1.4)
def randn(seed=None):
[MINOR][DOCUMENTATION] Fix some minor descriptions in functions consistently with expressions ## What changes were proposed in this pull request? This PR proposes to improve documentation and fix some descriptions equivalent to several minor fixes identified in https://github.com/apache/spark/pull/15677 Also, this suggests to change `Note:` and `NOTE:` to `.. note::` consistently with the others which marks up pretty. ## How was this patch tested? Jenkins tests and manually. For PySpark, `Note:` and `NOTE:` to `.. note::` make the document as below: **From** ![2016-11-04 6 53 35](https://cloud.githubusercontent.com/assets/6477701/20002648/42989922-a2c5-11e6-8a32-b73eda49e8c3.png) ![2016-11-04 6 53 45](https://cloud.githubusercontent.com/assets/6477701/20002650/429fb310-a2c5-11e6-926b-e030d7eb0185.png) ![2016-11-04 6 54 11](https://cloud.githubusercontent.com/assets/6477701/20002649/429d570a-a2c5-11e6-9e7e-44090f337e32.png) ![2016-11-04 6 53 51](https://cloud.githubusercontent.com/assets/6477701/20002647/4297fc74-a2c5-11e6-801a-b89fbcbfca44.png) ![2016-11-04 6 53 51](https://cloud.githubusercontent.com/assets/6477701/20002697/749f5780-a2c5-11e6-835f-022e1f2f82e3.png) **To** ![2016-11-04 7 03 48](https://cloud.githubusercontent.com/assets/6477701/20002659/4961b504-a2c5-11e6-9ee0-ef0751482f47.png) ![2016-11-04 7 04 03](https://cloud.githubusercontent.com/assets/6477701/20002660/49871d3a-a2c5-11e6-85ea-d9a5d11efeff.png) ![2016-11-04 7 04 28](https://cloud.githubusercontent.com/assets/6477701/20002662/498e0f14-a2c5-11e6-803d-c0c5aeda4153.png) ![2016-11-04 7 33 39](https://cloud.githubusercontent.com/assets/6477701/20002731/a76e30d2-a2c5-11e6-993b-0481b8342d6b.png) ![2016-11-04 7 33 39](https://cloud.githubusercontent.com/assets/6477701/20002731/a76e30d2-a2c5-11e6-993b-0481b8342d6b.png) Author: hyukjinkwon <gurwls223@gmail.com> Closes #15765 from HyukjinKwon/minor-function-doc.
2016-11-06 00:47:33 -04:00
"""Generates a column with independent and identically distributed (i.i.d.) samples from
the standard normal distribution.
.. note:: The function is non-deterministic in general case.
>>> df.withColumn('randn', randn(seed=42)).collect()
[Row(age=2, name=u'Alice', randn=1.1027054481455365),
Row(age=5, name=u'Bob', randn=0.7400395449950132)]
"""
sc = SparkContext._active_spark_context
if seed is not None:
jc = sc._jvm.functions.randn(seed)
else:
jc = sc._jvm.functions.randn()
return Column(jc)
@since(1.5)
def round(col, scale=0):
"""
Round the given value to `scale` decimal places using HALF_UP rounding mode if `scale` >= 0
or at integral part when `scale` < 0.
>>> spark.createDataFrame([(2.5,)], ['a']).select(round('a', 0).alias('r')).collect()
[Row(r=3.0)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.round(_to_java_column(col), scale))
@since(2.0)
def bround(col, scale=0):
"""
Round the given value to `scale` decimal places using HALF_EVEN rounding mode if `scale` >= 0
or at integral part when `scale` < 0.
>>> spark.createDataFrame([(2.5,)], ['a']).select(bround('a', 0).alias('r')).collect()
[Row(r=2.0)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.bround(_to_java_column(col), scale))
[SPARK-8223] [SPARK-8224] [SQL] shift left and shift right Jira: https://issues.apache.org/jira/browse/SPARK-8223 https://issues.apache.org/jira/browse/SPARK-8224 ~~I am aware of #7174 and will update this pr, if it's merged.~~ Done I don't know if #7034 can simplify this, but we can have a look on it, if it gets merged rxin In the Jira ticket the function as no second argument. I added a `numBits` argument that allows to specify the number of bits. I guess this improves the usability. I wanted to add `shiftleft(value)` as well, but the `selectExpr` dataframe tests crashes, if I have both. I order to do this, I added the following to the functions.scala `def shiftRight(e: Column): Column = ShiftRight(e.expr, lit(1).expr)`, but as I mentioned this doesn't pass tests like `df.selectExpr("shiftRight(a)", ...` (not enough arguments exception). If we need the bitwise shift in order to be hive compatible, I suggest to add `shiftLeft` and something like `shiftLeftX` Author: Tarek Auel <tarek.auel@googlemail.com> Closes #7178 from tarekauel/8223 and squashes the following commits: 8023bb5 [Tarek Auel] [SPARK-8223][SPARK-8224] fixed test f3f64e6 [Tarek Auel] [SPARK-8223][SPARK-8224] Integer -> Int f628706 [Tarek Auel] [SPARK-8223][SPARK-8224] removed toString; updated function description 3b56f2a [Tarek Auel] Merge remote-tracking branch 'origin/master' into 8223 5189690 [Tarek Auel] [SPARK-8223][SPARK-8224] minor fix and style fix 9434a28 [Tarek Auel] Merge remote-tracking branch 'origin/master' into 8223 44ee324 [Tarek Auel] [SPARK-8223][SPARK-8224] docu fix ac7fe9d [Tarek Auel] [SPARK-8223][SPARK-8224] right and left bit shift
2015-07-02 13:02:19 -04:00
@since(1.5)
def shiftLeft(col, numBits):
"""Shift the given value numBits left.
[SPARK-8223] [SPARK-8224] [SQL] shift left and shift right Jira: https://issues.apache.org/jira/browse/SPARK-8223 https://issues.apache.org/jira/browse/SPARK-8224 ~~I am aware of #7174 and will update this pr, if it's merged.~~ Done I don't know if #7034 can simplify this, but we can have a look on it, if it gets merged rxin In the Jira ticket the function as no second argument. I added a `numBits` argument that allows to specify the number of bits. I guess this improves the usability. I wanted to add `shiftleft(value)` as well, but the `selectExpr` dataframe tests crashes, if I have both. I order to do this, I added the following to the functions.scala `def shiftRight(e: Column): Column = ShiftRight(e.expr, lit(1).expr)`, but as I mentioned this doesn't pass tests like `df.selectExpr("shiftRight(a)", ...` (not enough arguments exception). If we need the bitwise shift in order to be hive compatible, I suggest to add `shiftLeft` and something like `shiftLeftX` Author: Tarek Auel <tarek.auel@googlemail.com> Closes #7178 from tarekauel/8223 and squashes the following commits: 8023bb5 [Tarek Auel] [SPARK-8223][SPARK-8224] fixed test f3f64e6 [Tarek Auel] [SPARK-8223][SPARK-8224] Integer -> Int f628706 [Tarek Auel] [SPARK-8223][SPARK-8224] removed toString; updated function description 3b56f2a [Tarek Auel] Merge remote-tracking branch 'origin/master' into 8223 5189690 [Tarek Auel] [SPARK-8223][SPARK-8224] minor fix and style fix 9434a28 [Tarek Auel] Merge remote-tracking branch 'origin/master' into 8223 44ee324 [Tarek Auel] [SPARK-8223][SPARK-8224] docu fix ac7fe9d [Tarek Auel] [SPARK-8223][SPARK-8224] right and left bit shift
2015-07-02 13:02:19 -04:00
>>> spark.createDataFrame([(21,)], ['a']).select(shiftLeft('a', 1).alias('r')).collect()
[SPARK-8223] [SPARK-8224] [SQL] shift left and shift right Jira: https://issues.apache.org/jira/browse/SPARK-8223 https://issues.apache.org/jira/browse/SPARK-8224 ~~I am aware of #7174 and will update this pr, if it's merged.~~ Done I don't know if #7034 can simplify this, but we can have a look on it, if it gets merged rxin In the Jira ticket the function as no second argument. I added a `numBits` argument that allows to specify the number of bits. I guess this improves the usability. I wanted to add `shiftleft(value)` as well, but the `selectExpr` dataframe tests crashes, if I have both. I order to do this, I added the following to the functions.scala `def shiftRight(e: Column): Column = ShiftRight(e.expr, lit(1).expr)`, but as I mentioned this doesn't pass tests like `df.selectExpr("shiftRight(a)", ...` (not enough arguments exception). If we need the bitwise shift in order to be hive compatible, I suggest to add `shiftLeft` and something like `shiftLeftX` Author: Tarek Auel <tarek.auel@googlemail.com> Closes #7178 from tarekauel/8223 and squashes the following commits: 8023bb5 [Tarek Auel] [SPARK-8223][SPARK-8224] fixed test f3f64e6 [Tarek Auel] [SPARK-8223][SPARK-8224] Integer -> Int f628706 [Tarek Auel] [SPARK-8223][SPARK-8224] removed toString; updated function description 3b56f2a [Tarek Auel] Merge remote-tracking branch 'origin/master' into 8223 5189690 [Tarek Auel] [SPARK-8223][SPARK-8224] minor fix and style fix 9434a28 [Tarek Auel] Merge remote-tracking branch 'origin/master' into 8223 44ee324 [Tarek Auel] [SPARK-8223][SPARK-8224] docu fix ac7fe9d [Tarek Auel] [SPARK-8223][SPARK-8224] right and left bit shift
2015-07-02 13:02:19 -04:00
[Row(r=42)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.shiftLeft(_to_java_column(col), numBits))
[SPARK-8223] [SPARK-8224] [SQL] shift left and shift right Jira: https://issues.apache.org/jira/browse/SPARK-8223 https://issues.apache.org/jira/browse/SPARK-8224 ~~I am aware of #7174 and will update this pr, if it's merged.~~ Done I don't know if #7034 can simplify this, but we can have a look on it, if it gets merged rxin In the Jira ticket the function as no second argument. I added a `numBits` argument that allows to specify the number of bits. I guess this improves the usability. I wanted to add `shiftleft(value)` as well, but the `selectExpr` dataframe tests crashes, if I have both. I order to do this, I added the following to the functions.scala `def shiftRight(e: Column): Column = ShiftRight(e.expr, lit(1).expr)`, but as I mentioned this doesn't pass tests like `df.selectExpr("shiftRight(a)", ...` (not enough arguments exception). If we need the bitwise shift in order to be hive compatible, I suggest to add `shiftLeft` and something like `shiftLeftX` Author: Tarek Auel <tarek.auel@googlemail.com> Closes #7178 from tarekauel/8223 and squashes the following commits: 8023bb5 [Tarek Auel] [SPARK-8223][SPARK-8224] fixed test f3f64e6 [Tarek Auel] [SPARK-8223][SPARK-8224] Integer -> Int f628706 [Tarek Auel] [SPARK-8223][SPARK-8224] removed toString; updated function description 3b56f2a [Tarek Auel] Merge remote-tracking branch 'origin/master' into 8223 5189690 [Tarek Auel] [SPARK-8223][SPARK-8224] minor fix and style fix 9434a28 [Tarek Auel] Merge remote-tracking branch 'origin/master' into 8223 44ee324 [Tarek Auel] [SPARK-8223][SPARK-8224] docu fix ac7fe9d [Tarek Auel] [SPARK-8223][SPARK-8224] right and left bit shift
2015-07-02 13:02:19 -04:00
@since(1.5)
def shiftRight(col, numBits):
[MINOR][DOCUMENTATION] Fix some minor descriptions in functions consistently with expressions ## What changes were proposed in this pull request? This PR proposes to improve documentation and fix some descriptions equivalent to several minor fixes identified in https://github.com/apache/spark/pull/15677 Also, this suggests to change `Note:` and `NOTE:` to `.. note::` consistently with the others which marks up pretty. ## How was this patch tested? Jenkins tests and manually. For PySpark, `Note:` and `NOTE:` to `.. note::` make the document as below: **From** ![2016-11-04 6 53 35](https://cloud.githubusercontent.com/assets/6477701/20002648/42989922-a2c5-11e6-8a32-b73eda49e8c3.png) ![2016-11-04 6 53 45](https://cloud.githubusercontent.com/assets/6477701/20002650/429fb310-a2c5-11e6-926b-e030d7eb0185.png) ![2016-11-04 6 54 11](https://cloud.githubusercontent.com/assets/6477701/20002649/429d570a-a2c5-11e6-9e7e-44090f337e32.png) ![2016-11-04 6 53 51](https://cloud.githubusercontent.com/assets/6477701/20002647/4297fc74-a2c5-11e6-801a-b89fbcbfca44.png) ![2016-11-04 6 53 51](https://cloud.githubusercontent.com/assets/6477701/20002697/749f5780-a2c5-11e6-835f-022e1f2f82e3.png) **To** ![2016-11-04 7 03 48](https://cloud.githubusercontent.com/assets/6477701/20002659/4961b504-a2c5-11e6-9ee0-ef0751482f47.png) ![2016-11-04 7 04 03](https://cloud.githubusercontent.com/assets/6477701/20002660/49871d3a-a2c5-11e6-85ea-d9a5d11efeff.png) ![2016-11-04 7 04 28](https://cloud.githubusercontent.com/assets/6477701/20002662/498e0f14-a2c5-11e6-803d-c0c5aeda4153.png) ![2016-11-04 7 33 39](https://cloud.githubusercontent.com/assets/6477701/20002731/a76e30d2-a2c5-11e6-993b-0481b8342d6b.png) ![2016-11-04 7 33 39](https://cloud.githubusercontent.com/assets/6477701/20002731/a76e30d2-a2c5-11e6-993b-0481b8342d6b.png) Author: hyukjinkwon <gurwls223@gmail.com> Closes #15765 from HyukjinKwon/minor-function-doc.
2016-11-06 00:47:33 -04:00
"""(Signed) shift the given value numBits right.
[SPARK-8223] [SPARK-8224] [SQL] shift left and shift right Jira: https://issues.apache.org/jira/browse/SPARK-8223 https://issues.apache.org/jira/browse/SPARK-8224 ~~I am aware of #7174 and will update this pr, if it's merged.~~ Done I don't know if #7034 can simplify this, but we can have a look on it, if it gets merged rxin In the Jira ticket the function as no second argument. I added a `numBits` argument that allows to specify the number of bits. I guess this improves the usability. I wanted to add `shiftleft(value)` as well, but the `selectExpr` dataframe tests crashes, if I have both. I order to do this, I added the following to the functions.scala `def shiftRight(e: Column): Column = ShiftRight(e.expr, lit(1).expr)`, but as I mentioned this doesn't pass tests like `df.selectExpr("shiftRight(a)", ...` (not enough arguments exception). If we need the bitwise shift in order to be hive compatible, I suggest to add `shiftLeft` and something like `shiftLeftX` Author: Tarek Auel <tarek.auel@googlemail.com> Closes #7178 from tarekauel/8223 and squashes the following commits: 8023bb5 [Tarek Auel] [SPARK-8223][SPARK-8224] fixed test f3f64e6 [Tarek Auel] [SPARK-8223][SPARK-8224] Integer -> Int f628706 [Tarek Auel] [SPARK-8223][SPARK-8224] removed toString; updated function description 3b56f2a [Tarek Auel] Merge remote-tracking branch 'origin/master' into 8223 5189690 [Tarek Auel] [SPARK-8223][SPARK-8224] minor fix and style fix 9434a28 [Tarek Auel] Merge remote-tracking branch 'origin/master' into 8223 44ee324 [Tarek Auel] [SPARK-8223][SPARK-8224] docu fix ac7fe9d [Tarek Auel] [SPARK-8223][SPARK-8224] right and left bit shift
2015-07-02 13:02:19 -04:00
>>> spark.createDataFrame([(42,)], ['a']).select(shiftRight('a', 1).alias('r')).collect()
[SPARK-8223] [SPARK-8224] [SQL] shift left and shift right Jira: https://issues.apache.org/jira/browse/SPARK-8223 https://issues.apache.org/jira/browse/SPARK-8224 ~~I am aware of #7174 and will update this pr, if it's merged.~~ Done I don't know if #7034 can simplify this, but we can have a look on it, if it gets merged rxin In the Jira ticket the function as no second argument. I added a `numBits` argument that allows to specify the number of bits. I guess this improves the usability. I wanted to add `shiftleft(value)` as well, but the `selectExpr` dataframe tests crashes, if I have both. I order to do this, I added the following to the functions.scala `def shiftRight(e: Column): Column = ShiftRight(e.expr, lit(1).expr)`, but as I mentioned this doesn't pass tests like `df.selectExpr("shiftRight(a)", ...` (not enough arguments exception). If we need the bitwise shift in order to be hive compatible, I suggest to add `shiftLeft` and something like `shiftLeftX` Author: Tarek Auel <tarek.auel@googlemail.com> Closes #7178 from tarekauel/8223 and squashes the following commits: 8023bb5 [Tarek Auel] [SPARK-8223][SPARK-8224] fixed test f3f64e6 [Tarek Auel] [SPARK-8223][SPARK-8224] Integer -> Int f628706 [Tarek Auel] [SPARK-8223][SPARK-8224] removed toString; updated function description 3b56f2a [Tarek Auel] Merge remote-tracking branch 'origin/master' into 8223 5189690 [Tarek Auel] [SPARK-8223][SPARK-8224] minor fix and style fix 9434a28 [Tarek Auel] Merge remote-tracking branch 'origin/master' into 8223 44ee324 [Tarek Auel] [SPARK-8223][SPARK-8224] docu fix ac7fe9d [Tarek Auel] [SPARK-8223][SPARK-8224] right and left bit shift
2015-07-02 13:02:19 -04:00
[Row(r=21)]
"""
sc = SparkContext._active_spark_context
jc = sc._jvm.functions.shiftRight(_to_java_column(col), numBits)
return Column(jc)
@since(1.5)
def shiftRightUnsigned(col, numBits):
"""Unsigned shift the given value numBits right.
>>> df = spark.createDataFrame([(-42,)], ['a'])
>>> df.select(shiftRightUnsigned('a', 1).alias('r')).collect()
[Row(r=9223372036854775787)]
"""
sc = SparkContext._active_spark_context
jc = sc._jvm.functions.shiftRightUnsigned(_to_java_column(col), numBits)
return Column(jc)
@since(1.6)
def spark_partition_id():
"""A column for partition ID.
[SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames - The old implicit would convert RDDs directly to DataFrames, and that added too many methods. - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed Python changes: - toDataFrame -> toDF - Dsl -> functions package - addColumn -> withColumn - renameColumn -> withColumnRenamed - add toDF functions to RDD on SQLContext init - add flatMap to DataFrame Author: Reynold Xin <rxin@databricks.com> Author: Davies Liu <davies@databricks.com> Closes #4556 from rxin/SPARK-5752 and squashes the following commits: 5ef9910 [Reynold Xin] More fix 61d3fca [Reynold Xin] Merge branch 'df5' of github.com:davies/spark into SPARK-5752 ff5832c [Reynold Xin] Fix python 749c675 [Reynold Xin] count(*) fixes. 5806df0 [Reynold Xin] Fix build break again. d941f3d [Reynold Xin] Fixed explode compilation break. fe1267a [Davies Liu] flatMap c4afb8e [Reynold Xin] style d9de47f [Davies Liu] add comment b783994 [Davies Liu] add comment for toDF e2154e5 [Davies Liu] schema() -> schema 3a1004f [Davies Liu] Dsl -> functions, toDF() fb256af [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 0dd74eb [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames 97dd47c [Davies Liu] fix mistake 6168f74 [Davies Liu] fix test 1fc0199 [Davies Liu] fix test a075cd5 [Davies Liu] clean up, toPandas 663d314 [Davies Liu] add test for agg('*') 9e214d5 [Reynold Xin] count(*) fixes. 1ed7136 [Reynold Xin] Fix build break again. 921b2e3 [Reynold Xin] Fixed explode compilation break. 14698d4 [Davies Liu] flatMap ba3e12d [Reynold Xin] style d08c92d [Davies Liu] add comment 5c8b524 [Davies Liu] add comment for toDF a4e5e66 [Davies Liu] schema() -> schema d377fc9 [Davies Liu] Dsl -> functions, toDF() 6b3086c [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 807e8b1 [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames
2015-02-14 02:03:22 -05:00
.. note:: This is indeterministic because it depends on data partitioning and task scheduling.
>>> df.repartition(1).select(spark_partition_id().alias("pid")).collect()
[Row(pid=0), Row(pid=0)]
[SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames - The old implicit would convert RDDs directly to DataFrames, and that added too many methods. - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed Python changes: - toDataFrame -> toDF - Dsl -> functions package - addColumn -> withColumn - renameColumn -> withColumnRenamed - add toDF functions to RDD on SQLContext init - add flatMap to DataFrame Author: Reynold Xin <rxin@databricks.com> Author: Davies Liu <davies@databricks.com> Closes #4556 from rxin/SPARK-5752 and squashes the following commits: 5ef9910 [Reynold Xin] More fix 61d3fca [Reynold Xin] Merge branch 'df5' of github.com:davies/spark into SPARK-5752 ff5832c [Reynold Xin] Fix python 749c675 [Reynold Xin] count(*) fixes. 5806df0 [Reynold Xin] Fix build break again. d941f3d [Reynold Xin] Fixed explode compilation break. fe1267a [Davies Liu] flatMap c4afb8e [Reynold Xin] style d9de47f [Davies Liu] add comment b783994 [Davies Liu] add comment for toDF e2154e5 [Davies Liu] schema() -> schema 3a1004f [Davies Liu] Dsl -> functions, toDF() fb256af [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 0dd74eb [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames 97dd47c [Davies Liu] fix mistake 6168f74 [Davies Liu] fix test 1fc0199 [Davies Liu] fix test a075cd5 [Davies Liu] clean up, toPandas 663d314 [Davies Liu] add test for agg('*') 9e214d5 [Reynold Xin] count(*) fixes. 1ed7136 [Reynold Xin] Fix build break again. 921b2e3 [Reynold Xin] Fixed explode compilation break. 14698d4 [Davies Liu] flatMap ba3e12d [Reynold Xin] style d08c92d [Davies Liu] add comment 5c8b524 [Davies Liu] add comment for toDF a4e5e66 [Davies Liu] schema() -> schema d377fc9 [Davies Liu] Dsl -> functions, toDF() 6b3086c [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 807e8b1 [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames
2015-02-14 02:03:22 -05:00
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.spark_partition_id())
[SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames - The old implicit would convert RDDs directly to DataFrames, and that added too many methods. - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed Python changes: - toDataFrame -> toDF - Dsl -> functions package - addColumn -> withColumn - renameColumn -> withColumnRenamed - add toDF functions to RDD on SQLContext init - add flatMap to DataFrame Author: Reynold Xin <rxin@databricks.com> Author: Davies Liu <davies@databricks.com> Closes #4556 from rxin/SPARK-5752 and squashes the following commits: 5ef9910 [Reynold Xin] More fix 61d3fca [Reynold Xin] Merge branch 'df5' of github.com:davies/spark into SPARK-5752 ff5832c [Reynold Xin] Fix python 749c675 [Reynold Xin] count(*) fixes. 5806df0 [Reynold Xin] Fix build break again. d941f3d [Reynold Xin] Fixed explode compilation break. fe1267a [Davies Liu] flatMap c4afb8e [Reynold Xin] style d9de47f [Davies Liu] add comment b783994 [Davies Liu] add comment for toDF e2154e5 [Davies Liu] schema() -> schema 3a1004f [Davies Liu] Dsl -> functions, toDF() fb256af [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 0dd74eb [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames 97dd47c [Davies Liu] fix mistake 6168f74 [Davies Liu] fix test 1fc0199 [Davies Liu] fix test a075cd5 [Davies Liu] clean up, toPandas 663d314 [Davies Liu] add test for agg('*') 9e214d5 [Reynold Xin] count(*) fixes. 1ed7136 [Reynold Xin] Fix build break again. 921b2e3 [Reynold Xin] Fixed explode compilation break. 14698d4 [Davies Liu] flatMap ba3e12d [Reynold Xin] style d08c92d [Davies Liu] add comment 5c8b524 [Davies Liu] add comment for toDF a4e5e66 [Davies Liu] schema() -> schema d377fc9 [Davies Liu] Dsl -> functions, toDF() 6b3086c [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 807e8b1 [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames
2015-02-14 02:03:22 -05:00
@since(1.5)
def expr(str):
"""Parses the expression string into the column that it represents
>>> df.select(expr("length(name)")).collect()
[Row(length(name)=5), Row(length(name)=3)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.expr(str))
@ignore_unicode_prefix
@since(1.4)
def struct(*cols):
"""Creates a new struct column.
:param cols: list of column names (string) or list of :class:`Column` expressions
>>> df.select(struct('age', 'name').alias("struct")).collect()
[Row(struct=Row(age=2, name=u'Alice')), Row(struct=Row(age=5, name=u'Bob'))]
>>> df.select(struct([df.age, df.name]).alias("struct")).collect()
[Row(struct=Row(age=2, name=u'Alice')), Row(struct=Row(age=5, name=u'Bob'))]
"""
sc = SparkContext._active_spark_context
if len(cols) == 1 and isinstance(cols[0], (list, set)):
cols = cols[0]
jc = sc._jvm.functions.struct(_to_seq(sc, cols, _to_java_column))
return Column(jc)
@since(1.5)
def greatest(*cols):
"""
Returns the greatest value of the list of column names, skipping null values.
This function takes at least 2 parameters. It will return null iff all parameters are null.
>>> df = spark.createDataFrame([(1, 4, 3)], ['a', 'b', 'c'])
>>> df.select(greatest(df.a, df.b, df.c).alias("greatest")).collect()
[Row(greatest=4)]
"""
if len(cols) < 2:
raise ValueError("greatest should take at least two columns")
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.greatest(_to_seq(sc, cols, _to_java_column)))
@since(1.5)
def least(*cols):
"""
Returns the least value of the list of column names, skipping null values.
This function takes at least 2 parameters. It will return null iff all parameters are null.
>>> df = spark.createDataFrame([(1, 4, 3)], ['a', 'b', 'c'])
>>> df.select(least(df.a, df.b, df.c).alias("least")).collect()
[Row(least=1)]
"""
if len(cols) < 2:
raise ValueError("least should take at least two columns")
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.least(_to_seq(sc, cols, _to_java_column)))
@since(1.4)
def when(condition, value):
"""Evaluates a list of conditions and returns one of multiple possible result expressions.
If :func:`Column.otherwise` is not invoked, None is returned for unmatched conditions.
:param condition: a boolean :class:`Column` expression.
:param value: a literal value, or a :class:`Column` expression.
>>> df.select(when(df['age'] == 2, 3).otherwise(4).alias("age")).collect()
[Row(age=3), Row(age=4)]
>>> df.select(when(df.age == 2, df.age + 1).alias("age")).collect()
[Row(age=3), Row(age=None)]
"""
sc = SparkContext._active_spark_context
if not isinstance(condition, Column):
raise TypeError("condition should be a Column")
v = value._jc if isinstance(value, Column) else value
jc = sc._jvm.functions.when(condition._jc, v)
return Column(jc)
@since(1.5)
def log(arg1, arg2=None):
[SPARK-8218][SQL] Add binary log math function JIRA: https://issues.apache.org/jira/browse/SPARK-8218 Because there is already `log` unary function defined, the binary log function is called `logarithm` for now. Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #6725 from viirya/expr_binary_log and squashes the following commits: bf96bd9 [Liang-Chi Hsieh] Compare log result in string. 102070d [Liang-Chi Hsieh] Round log result to better comparing in python test. fd01863 [Liang-Chi Hsieh] For comments. beed631 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_binary_log 6089d11 [Liang-Chi Hsieh] Remove unnecessary override. 8cf37b7 [Liang-Chi Hsieh] For comments. bc89597 [Liang-Chi Hsieh] For comments. db7dc38 [Liang-Chi Hsieh] Use ctor instead of companion object. 0634ef7 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_binary_log 1750034 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_binary_log 3d75bfc [Liang-Chi Hsieh] Fix scala style. 5b39c02 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_binary_log 23c54a3 [Liang-Chi Hsieh] Fix scala style. ebc9929 [Liang-Chi Hsieh] Let Logarithm accept one parameter too. 605574d [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_binary_log 21c3bfd [Liang-Chi Hsieh] Fix scala style. c6c187f [Liang-Chi Hsieh] For comments. c795342 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_binary_log f373bac [Liang-Chi Hsieh] Add binary log expression.
2015-06-18 02:31:30 -04:00
"""Returns the first argument-based logarithm of the second argument.
If there is only one argument, then this takes the natural logarithm of the argument.
>>> df.select(log(10.0, df.age).alias('ten')).rdd.map(lambda l: str(l.ten)[:7]).collect()
[SPARK-8218][SQL] Add binary log math function JIRA: https://issues.apache.org/jira/browse/SPARK-8218 Because there is already `log` unary function defined, the binary log function is called `logarithm` for now. Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #6725 from viirya/expr_binary_log and squashes the following commits: bf96bd9 [Liang-Chi Hsieh] Compare log result in string. 102070d [Liang-Chi Hsieh] Round log result to better comparing in python test. fd01863 [Liang-Chi Hsieh] For comments. beed631 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_binary_log 6089d11 [Liang-Chi Hsieh] Remove unnecessary override. 8cf37b7 [Liang-Chi Hsieh] For comments. bc89597 [Liang-Chi Hsieh] For comments. db7dc38 [Liang-Chi Hsieh] Use ctor instead of companion object. 0634ef7 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_binary_log 1750034 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_binary_log 3d75bfc [Liang-Chi Hsieh] Fix scala style. 5b39c02 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_binary_log 23c54a3 [Liang-Chi Hsieh] Fix scala style. ebc9929 [Liang-Chi Hsieh] Let Logarithm accept one parameter too. 605574d [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_binary_log 21c3bfd [Liang-Chi Hsieh] Fix scala style. c6c187f [Liang-Chi Hsieh] For comments. c795342 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_binary_log f373bac [Liang-Chi Hsieh] Add binary log expression.
2015-06-18 02:31:30 -04:00
['0.30102', '0.69897']
>>> df.select(log(df.age).alias('e')).rdd.map(lambda l: str(l.e)[:7]).collect()
[SPARK-8218][SQL] Add binary log math function JIRA: https://issues.apache.org/jira/browse/SPARK-8218 Because there is already `log` unary function defined, the binary log function is called `logarithm` for now. Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #6725 from viirya/expr_binary_log and squashes the following commits: bf96bd9 [Liang-Chi Hsieh] Compare log result in string. 102070d [Liang-Chi Hsieh] Round log result to better comparing in python test. fd01863 [Liang-Chi Hsieh] For comments. beed631 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_binary_log 6089d11 [Liang-Chi Hsieh] Remove unnecessary override. 8cf37b7 [Liang-Chi Hsieh] For comments. bc89597 [Liang-Chi Hsieh] For comments. db7dc38 [Liang-Chi Hsieh] Use ctor instead of companion object. 0634ef7 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_binary_log 1750034 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_binary_log 3d75bfc [Liang-Chi Hsieh] Fix scala style. 5b39c02 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_binary_log 23c54a3 [Liang-Chi Hsieh] Fix scala style. ebc9929 [Liang-Chi Hsieh] Let Logarithm accept one parameter too. 605574d [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_binary_log 21c3bfd [Liang-Chi Hsieh] Fix scala style. c6c187f [Liang-Chi Hsieh] For comments. c795342 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_binary_log f373bac [Liang-Chi Hsieh] Add binary log expression.
2015-06-18 02:31:30 -04:00
['0.69314', '1.60943']
"""
sc = SparkContext._active_spark_context
if arg2 is None:
jc = sc._jvm.functions.log(_to_java_column(arg1))
else:
jc = sc._jvm.functions.log(arg1, _to_java_column(arg2))
[SPARK-8218][SQL] Add binary log math function JIRA: https://issues.apache.org/jira/browse/SPARK-8218 Because there is already `log` unary function defined, the binary log function is called `logarithm` for now. Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #6725 from viirya/expr_binary_log and squashes the following commits: bf96bd9 [Liang-Chi Hsieh] Compare log result in string. 102070d [Liang-Chi Hsieh] Round log result to better comparing in python test. fd01863 [Liang-Chi Hsieh] For comments. beed631 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_binary_log 6089d11 [Liang-Chi Hsieh] Remove unnecessary override. 8cf37b7 [Liang-Chi Hsieh] For comments. bc89597 [Liang-Chi Hsieh] For comments. db7dc38 [Liang-Chi Hsieh] Use ctor instead of companion object. 0634ef7 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_binary_log 1750034 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_binary_log 3d75bfc [Liang-Chi Hsieh] Fix scala style. 5b39c02 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_binary_log 23c54a3 [Liang-Chi Hsieh] Fix scala style. ebc9929 [Liang-Chi Hsieh] Let Logarithm accept one parameter too. 605574d [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_binary_log 21c3bfd [Liang-Chi Hsieh] Fix scala style. c6c187f [Liang-Chi Hsieh] For comments. c795342 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into expr_binary_log f373bac [Liang-Chi Hsieh] Add binary log expression.
2015-06-18 02:31:30 -04:00
return Column(jc)
@since(1.5)
def log2(col):
"""Returns the base-2 logarithm of the argument.
>>> spark.createDataFrame([(4,)], ['a']).select(log2('a').alias('log2')).collect()
[Row(log2=2.0)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.log2(_to_java_column(col)))
@since(1.5)
@ignore_unicode_prefix
def conv(col, fromBase, toBase):
"""
Convert a number in a string column from one base to another.
>>> df = spark.createDataFrame([("010101",)], ['n'])
>>> df.select(conv(df.n, 2, 16).alias('hex')).collect()
[Row(hex=u'15')]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.conv(_to_java_column(col), fromBase, toBase))
@since(1.5)
def factorial(col):
"""
Computes the factorial of the given value.
>>> df = spark.createDataFrame([(5,)], ['n'])
>>> df.select(factorial(df.n).alias('f')).collect()
[Row(f=120)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.factorial(_to_java_column(col)))
# --------------- Window functions ------------------------
@since(1.4)
def lag(col, offset=1, default=None):
"""
Window function: returns the value that is `offset` rows before the current row, and
`defaultValue` if there is less than `offset` rows before the current row. For example,
an `offset` of one will return the previous row at any given point in the window partition.
This is equivalent to the LAG function in SQL.
:param col: name of column or expression
:param offset: number of row to extend
:param default: default value
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.lag(_to_java_column(col), offset, default))
@since(1.4)
def lead(col, offset=1, default=None):
"""
Window function: returns the value that is `offset` rows after the current row, and
`defaultValue` if there is less than `offset` rows after the current row. For example,
an `offset` of one will return the next row at any given point in the window partition.
This is equivalent to the LEAD function in SQL.
:param col: name of column or expression
:param offset: number of row to extend
:param default: default value
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.lead(_to_java_column(col), offset, default))
@since(1.4)
def ntile(n):
"""
Window function: returns the ntile group id (from 1 to `n` inclusive)
in an ordered window partition. For example, if `n` is 4, the first
quarter of the rows will get value 1, the second quarter will get 2,
the third quarter will get 3, and the last quarter will get 4.
This is equivalent to the NTILE function in SQL.
:param n: an integer
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.ntile(int(n)))
# ---------------------- Date/Timestamp functions ------------------------------
@since(1.5)
def current_date():
"""
Returns the current date as a :class:`DateType` column.
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.current_date())
def current_timestamp():
"""
Returns the current timestamp as a :class:`TimestampType` column.
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.current_timestamp())
[SPARK-8199][SPARK-8184][SPARK-8183][SPARK-8182][SPARK-8181][SPARK-8180][SPARK-8179][SPARK-8177][SPARK-8178][SPARK-9115][SQL] date functions Jira: https://issues.apache.org/jira/browse/SPARK-8199 https://issues.apache.org/jira/browse/SPARK-8184 https://issues.apache.org/jira/browse/SPARK-8183 https://issues.apache.org/jira/browse/SPARK-8182 https://issues.apache.org/jira/browse/SPARK-8181 https://issues.apache.org/jira/browse/SPARK-8180 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-8177 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-9115 Regarding `day`and `dayofmonth` are both necessary? ~~I am going to add `Quarter` to this PR as well.~~ Done. ~~As soon as the Scala coding is reviewed and discussed, I'll add the python api.~~ Done Author: Tarek Auel <tarek.auel@googlemail.com> Author: Tarek Auel <tarek.auel@gmail.com> Closes #6981 from tarekauel/SPARK-8199 and squashes the following commits: f7b4c8c [Tarek Auel] [SPARK-8199] fixed bug in tests bb567b6 [Tarek Auel] [SPARK-8199] fixed test 3e095ba [Tarek Auel] [SPARK-8199] style and timezone fix 256c357 [Tarek Auel] [SPARK-8199] code cleanup 5983dcc [Tarek Auel] [SPARK-8199] whitespace fix 6e0c78f [Tarek Auel] [SPARK-8199] removed setTimeZone in tests, according to cloud-fans comment in #7488 4afc09c [Tarek Auel] [SPARK-8199] concise leap year handling ea6c110 [Tarek Auel] [SPARK-8199] fix after merging master 70238e0 [Tarek Auel] Merge branch 'master' into SPARK-8199 3c6ae2e [Tarek Auel] [SPARK-8199] removed binary search fb98ba0 [Tarek Auel] [SPARK-8199] python docstring fix cdfae27 [Tarek Auel] [SPARK-8199] cleanup & python docstring fix 746b80a [Tarek Auel] [SPARK-8199] build fix 0ad6db8 [Tarek Auel] [SPARK-8199] minor fix 523542d [Tarek Auel] [SPARK-8199] address comments 2259299 [Tarek Auel] [SPARK-8199] day_of_month alias d01b977 [Tarek Auel] [SPARK-8199] python underscore 56c4a92 [Tarek Auel] [SPARK-8199] update python docu e223bc0 [Tarek Auel] [SPARK-8199] refactoring d6aa14e [Tarek Auel] [SPARK-8199] fixed Hive compatibility b382267 [Tarek Auel] [SPARK-8199] fixed bug in day calculation; removed set TimeZone in HiveCompatibilitySuite for test purposes; removed Hive tests for second and minute, because we can cast '2015-03-18' to a timestamp and extract a minute/second from it 1b2e540 [Tarek Auel] [SPARK-8119] style fix 0852655 [Tarek Auel] [SPARK-8119] changed from ExpectsInputTypes to implicit casts ec87c69 [Tarek Auel] [SPARK-8119] bug fixing and refactoring 1358cdc [Tarek Auel] Merge remote-tracking branch 'origin/master' into SPARK-8199 740af0e [Tarek Auel] implement date function using a calculation based on days 4fb66da [Tarek Auel] WIP: date functions on calculation only 1a436c9 [Tarek Auel] wip f775f39 [Tarek Auel] fixed return type ad17e96 [Tarek Auel] improved implementation c42b444 [Tarek Auel] Removed merge conflict file ccb723c [Tarek Auel] [SPARK-8199] style and fixed merge issues 10e4ad1 [Tarek Auel] Merge branch 'master' into date-functions-fast 7d9f0eb [Tarek Auel] [SPARK-8199] git renaming issue f3e7a9f [Tarek Auel] [SPARK-8199] revert change in DataFrameFunctionsSuite 6f5d95c [Tarek Auel] [SPARK-8199] fixed year interval d9f8ac3 [Tarek Auel] [SPARK-8199] implement fast track 7bc9d93 [Tarek Auel] Merge branch 'master' into SPARK-8199 5a105d9 [Tarek Auel] [SPARK-8199] rebase after #6985 got merged eb6760d [Tarek Auel] Merge branch 'master' into SPARK-8199 f120415 [Tarek Auel] improved runtime a8edebd [Tarek Auel] use Calendar instead of SimpleDateFormat 5fe74e1 [Tarek Auel] fixed python style 3bfac90 [Tarek Auel] fixed style 356df78 [Tarek Auel] rely on cast mechanism of Spark. Simplified implementation 02efc5d [Tarek Auel] removed doubled code a5ea120 [Tarek Auel] added python api; changed test to be more meaningful b680db6 [Tarek Auel] added codegeneration to all functions c739788 [Tarek Auel] added support for quarter SPARK-8178 849fb41 [Tarek Auel] fixed stupid test 638596f [Tarek Auel] improved codegen 4d8049b [Tarek Auel] fixed tests and added type check 5ebb235 [Tarek Auel] resolved naming conflict d0e2f99 [Tarek Auel] date functions
2015-07-19 01:48:05 -04:00
@ignore_unicode_prefix
@since(1.5)
def date_format(date, format):
[SPARK-8199][SPARK-8184][SPARK-8183][SPARK-8182][SPARK-8181][SPARK-8180][SPARK-8179][SPARK-8177][SPARK-8178][SPARK-9115][SQL] date functions Jira: https://issues.apache.org/jira/browse/SPARK-8199 https://issues.apache.org/jira/browse/SPARK-8184 https://issues.apache.org/jira/browse/SPARK-8183 https://issues.apache.org/jira/browse/SPARK-8182 https://issues.apache.org/jira/browse/SPARK-8181 https://issues.apache.org/jira/browse/SPARK-8180 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-8177 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-9115 Regarding `day`and `dayofmonth` are both necessary? ~~I am going to add `Quarter` to this PR as well.~~ Done. ~~As soon as the Scala coding is reviewed and discussed, I'll add the python api.~~ Done Author: Tarek Auel <tarek.auel@googlemail.com> Author: Tarek Auel <tarek.auel@gmail.com> Closes #6981 from tarekauel/SPARK-8199 and squashes the following commits: f7b4c8c [Tarek Auel] [SPARK-8199] fixed bug in tests bb567b6 [Tarek Auel] [SPARK-8199] fixed test 3e095ba [Tarek Auel] [SPARK-8199] style and timezone fix 256c357 [Tarek Auel] [SPARK-8199] code cleanup 5983dcc [Tarek Auel] [SPARK-8199] whitespace fix 6e0c78f [Tarek Auel] [SPARK-8199] removed setTimeZone in tests, according to cloud-fans comment in #7488 4afc09c [Tarek Auel] [SPARK-8199] concise leap year handling ea6c110 [Tarek Auel] [SPARK-8199] fix after merging master 70238e0 [Tarek Auel] Merge branch 'master' into SPARK-8199 3c6ae2e [Tarek Auel] [SPARK-8199] removed binary search fb98ba0 [Tarek Auel] [SPARK-8199] python docstring fix cdfae27 [Tarek Auel] [SPARK-8199] cleanup & python docstring fix 746b80a [Tarek Auel] [SPARK-8199] build fix 0ad6db8 [Tarek Auel] [SPARK-8199] minor fix 523542d [Tarek Auel] [SPARK-8199] address comments 2259299 [Tarek Auel] [SPARK-8199] day_of_month alias d01b977 [Tarek Auel] [SPARK-8199] python underscore 56c4a92 [Tarek Auel] [SPARK-8199] update python docu e223bc0 [Tarek Auel] [SPARK-8199] refactoring d6aa14e [Tarek Auel] [SPARK-8199] fixed Hive compatibility b382267 [Tarek Auel] [SPARK-8199] fixed bug in day calculation; removed set TimeZone in HiveCompatibilitySuite for test purposes; removed Hive tests for second and minute, because we can cast '2015-03-18' to a timestamp and extract a minute/second from it 1b2e540 [Tarek Auel] [SPARK-8119] style fix 0852655 [Tarek Auel] [SPARK-8119] changed from ExpectsInputTypes to implicit casts ec87c69 [Tarek Auel] [SPARK-8119] bug fixing and refactoring 1358cdc [Tarek Auel] Merge remote-tracking branch 'origin/master' into SPARK-8199 740af0e [Tarek Auel] implement date function using a calculation based on days 4fb66da [Tarek Auel] WIP: date functions on calculation only 1a436c9 [Tarek Auel] wip f775f39 [Tarek Auel] fixed return type ad17e96 [Tarek Auel] improved implementation c42b444 [Tarek Auel] Removed merge conflict file ccb723c [Tarek Auel] [SPARK-8199] style and fixed merge issues 10e4ad1 [Tarek Auel] Merge branch 'master' into date-functions-fast 7d9f0eb [Tarek Auel] [SPARK-8199] git renaming issue f3e7a9f [Tarek Auel] [SPARK-8199] revert change in DataFrameFunctionsSuite 6f5d95c [Tarek Auel] [SPARK-8199] fixed year interval d9f8ac3 [Tarek Auel] [SPARK-8199] implement fast track 7bc9d93 [Tarek Auel] Merge branch 'master' into SPARK-8199 5a105d9 [Tarek Auel] [SPARK-8199] rebase after #6985 got merged eb6760d [Tarek Auel] Merge branch 'master' into SPARK-8199 f120415 [Tarek Auel] improved runtime a8edebd [Tarek Auel] use Calendar instead of SimpleDateFormat 5fe74e1 [Tarek Auel] fixed python style 3bfac90 [Tarek Auel] fixed style 356df78 [Tarek Auel] rely on cast mechanism of Spark. Simplified implementation 02efc5d [Tarek Auel] removed doubled code a5ea120 [Tarek Auel] added python api; changed test to be more meaningful b680db6 [Tarek Auel] added codegeneration to all functions c739788 [Tarek Auel] added support for quarter SPARK-8178 849fb41 [Tarek Auel] fixed stupid test 638596f [Tarek Auel] improved codegen 4d8049b [Tarek Auel] fixed tests and added type check 5ebb235 [Tarek Auel] resolved naming conflict d0e2f99 [Tarek Auel] date functions
2015-07-19 01:48:05 -04:00
"""
Converts a date/timestamp/string to a value of string in the format specified by the date
format given by the second argument.
A pattern could be for instance `dd.MM.yyyy` and could return a string like '18.03.1993'. All
pattern letters of the Java class `java.time.format.DateTimeFormatter` can be used.
[SPARK-8199][SPARK-8184][SPARK-8183][SPARK-8182][SPARK-8181][SPARK-8180][SPARK-8179][SPARK-8177][SPARK-8178][SPARK-9115][SQL] date functions Jira: https://issues.apache.org/jira/browse/SPARK-8199 https://issues.apache.org/jira/browse/SPARK-8184 https://issues.apache.org/jira/browse/SPARK-8183 https://issues.apache.org/jira/browse/SPARK-8182 https://issues.apache.org/jira/browse/SPARK-8181 https://issues.apache.org/jira/browse/SPARK-8180 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-8177 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-9115 Regarding `day`and `dayofmonth` are both necessary? ~~I am going to add `Quarter` to this PR as well.~~ Done. ~~As soon as the Scala coding is reviewed and discussed, I'll add the python api.~~ Done Author: Tarek Auel <tarek.auel@googlemail.com> Author: Tarek Auel <tarek.auel@gmail.com> Closes #6981 from tarekauel/SPARK-8199 and squashes the following commits: f7b4c8c [Tarek Auel] [SPARK-8199] fixed bug in tests bb567b6 [Tarek Auel] [SPARK-8199] fixed test 3e095ba [Tarek Auel] [SPARK-8199] style and timezone fix 256c357 [Tarek Auel] [SPARK-8199] code cleanup 5983dcc [Tarek Auel] [SPARK-8199] whitespace fix 6e0c78f [Tarek Auel] [SPARK-8199] removed setTimeZone in tests, according to cloud-fans comment in #7488 4afc09c [Tarek Auel] [SPARK-8199] concise leap year handling ea6c110 [Tarek Auel] [SPARK-8199] fix after merging master 70238e0 [Tarek Auel] Merge branch 'master' into SPARK-8199 3c6ae2e [Tarek Auel] [SPARK-8199] removed binary search fb98ba0 [Tarek Auel] [SPARK-8199] python docstring fix cdfae27 [Tarek Auel] [SPARK-8199] cleanup & python docstring fix 746b80a [Tarek Auel] [SPARK-8199] build fix 0ad6db8 [Tarek Auel] [SPARK-8199] minor fix 523542d [Tarek Auel] [SPARK-8199] address comments 2259299 [Tarek Auel] [SPARK-8199] day_of_month alias d01b977 [Tarek Auel] [SPARK-8199] python underscore 56c4a92 [Tarek Auel] [SPARK-8199] update python docu e223bc0 [Tarek Auel] [SPARK-8199] refactoring d6aa14e [Tarek Auel] [SPARK-8199] fixed Hive compatibility b382267 [Tarek Auel] [SPARK-8199] fixed bug in day calculation; removed set TimeZone in HiveCompatibilitySuite for test purposes; removed Hive tests for second and minute, because we can cast '2015-03-18' to a timestamp and extract a minute/second from it 1b2e540 [Tarek Auel] [SPARK-8119] style fix 0852655 [Tarek Auel] [SPARK-8119] changed from ExpectsInputTypes to implicit casts ec87c69 [Tarek Auel] [SPARK-8119] bug fixing and refactoring 1358cdc [Tarek Auel] Merge remote-tracking branch 'origin/master' into SPARK-8199 740af0e [Tarek Auel] implement date function using a calculation based on days 4fb66da [Tarek Auel] WIP: date functions on calculation only 1a436c9 [Tarek Auel] wip f775f39 [Tarek Auel] fixed return type ad17e96 [Tarek Auel] improved implementation c42b444 [Tarek Auel] Removed merge conflict file ccb723c [Tarek Auel] [SPARK-8199] style and fixed merge issues 10e4ad1 [Tarek Auel] Merge branch 'master' into date-functions-fast 7d9f0eb [Tarek Auel] [SPARK-8199] git renaming issue f3e7a9f [Tarek Auel] [SPARK-8199] revert change in DataFrameFunctionsSuite 6f5d95c [Tarek Auel] [SPARK-8199] fixed year interval d9f8ac3 [Tarek Auel] [SPARK-8199] implement fast track 7bc9d93 [Tarek Auel] Merge branch 'master' into SPARK-8199 5a105d9 [Tarek Auel] [SPARK-8199] rebase after #6985 got merged eb6760d [Tarek Auel] Merge branch 'master' into SPARK-8199 f120415 [Tarek Auel] improved runtime a8edebd [Tarek Auel] use Calendar instead of SimpleDateFormat 5fe74e1 [Tarek Auel] fixed python style 3bfac90 [Tarek Auel] fixed style 356df78 [Tarek Auel] rely on cast mechanism of Spark. Simplified implementation 02efc5d [Tarek Auel] removed doubled code a5ea120 [Tarek Auel] added python api; changed test to be more meaningful b680db6 [Tarek Auel] added codegeneration to all functions c739788 [Tarek Auel] added support for quarter SPARK-8178 849fb41 [Tarek Auel] fixed stupid test 638596f [Tarek Auel] improved codegen 4d8049b [Tarek Auel] fixed tests and added type check 5ebb235 [Tarek Auel] resolved naming conflict d0e2f99 [Tarek Auel] date functions
2015-07-19 01:48:05 -04:00
[MINOR][DOCUMENTATION] Fix some minor descriptions in functions consistently with expressions ## What changes were proposed in this pull request? This PR proposes to improve documentation and fix some descriptions equivalent to several minor fixes identified in https://github.com/apache/spark/pull/15677 Also, this suggests to change `Note:` and `NOTE:` to `.. note::` consistently with the others which marks up pretty. ## How was this patch tested? Jenkins tests and manually. For PySpark, `Note:` and `NOTE:` to `.. note::` make the document as below: **From** ![2016-11-04 6 53 35](https://cloud.githubusercontent.com/assets/6477701/20002648/42989922-a2c5-11e6-8a32-b73eda49e8c3.png) ![2016-11-04 6 53 45](https://cloud.githubusercontent.com/assets/6477701/20002650/429fb310-a2c5-11e6-926b-e030d7eb0185.png) ![2016-11-04 6 54 11](https://cloud.githubusercontent.com/assets/6477701/20002649/429d570a-a2c5-11e6-9e7e-44090f337e32.png) ![2016-11-04 6 53 51](https://cloud.githubusercontent.com/assets/6477701/20002647/4297fc74-a2c5-11e6-801a-b89fbcbfca44.png) ![2016-11-04 6 53 51](https://cloud.githubusercontent.com/assets/6477701/20002697/749f5780-a2c5-11e6-835f-022e1f2f82e3.png) **To** ![2016-11-04 7 03 48](https://cloud.githubusercontent.com/assets/6477701/20002659/4961b504-a2c5-11e6-9ee0-ef0751482f47.png) ![2016-11-04 7 04 03](https://cloud.githubusercontent.com/assets/6477701/20002660/49871d3a-a2c5-11e6-85ea-d9a5d11efeff.png) ![2016-11-04 7 04 28](https://cloud.githubusercontent.com/assets/6477701/20002662/498e0f14-a2c5-11e6-803d-c0c5aeda4153.png) ![2016-11-04 7 33 39](https://cloud.githubusercontent.com/assets/6477701/20002731/a76e30d2-a2c5-11e6-993b-0481b8342d6b.png) ![2016-11-04 7 33 39](https://cloud.githubusercontent.com/assets/6477701/20002731/a76e30d2-a2c5-11e6-993b-0481b8342d6b.png) Author: hyukjinkwon <gurwls223@gmail.com> Closes #15765 from HyukjinKwon/minor-function-doc.
2016-11-06 00:47:33 -04:00
.. note:: Use when ever possible specialized functions like `year`. These benefit from a
specialized implementation.
[SPARK-8199][SPARK-8184][SPARK-8183][SPARK-8182][SPARK-8181][SPARK-8180][SPARK-8179][SPARK-8177][SPARK-8178][SPARK-9115][SQL] date functions Jira: https://issues.apache.org/jira/browse/SPARK-8199 https://issues.apache.org/jira/browse/SPARK-8184 https://issues.apache.org/jira/browse/SPARK-8183 https://issues.apache.org/jira/browse/SPARK-8182 https://issues.apache.org/jira/browse/SPARK-8181 https://issues.apache.org/jira/browse/SPARK-8180 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-8177 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-9115 Regarding `day`and `dayofmonth` are both necessary? ~~I am going to add `Quarter` to this PR as well.~~ Done. ~~As soon as the Scala coding is reviewed and discussed, I'll add the python api.~~ Done Author: Tarek Auel <tarek.auel@googlemail.com> Author: Tarek Auel <tarek.auel@gmail.com> Closes #6981 from tarekauel/SPARK-8199 and squashes the following commits: f7b4c8c [Tarek Auel] [SPARK-8199] fixed bug in tests bb567b6 [Tarek Auel] [SPARK-8199] fixed test 3e095ba [Tarek Auel] [SPARK-8199] style and timezone fix 256c357 [Tarek Auel] [SPARK-8199] code cleanup 5983dcc [Tarek Auel] [SPARK-8199] whitespace fix 6e0c78f [Tarek Auel] [SPARK-8199] removed setTimeZone in tests, according to cloud-fans comment in #7488 4afc09c [Tarek Auel] [SPARK-8199] concise leap year handling ea6c110 [Tarek Auel] [SPARK-8199] fix after merging master 70238e0 [Tarek Auel] Merge branch 'master' into SPARK-8199 3c6ae2e [Tarek Auel] [SPARK-8199] removed binary search fb98ba0 [Tarek Auel] [SPARK-8199] python docstring fix cdfae27 [Tarek Auel] [SPARK-8199] cleanup & python docstring fix 746b80a [Tarek Auel] [SPARK-8199] build fix 0ad6db8 [Tarek Auel] [SPARK-8199] minor fix 523542d [Tarek Auel] [SPARK-8199] address comments 2259299 [Tarek Auel] [SPARK-8199] day_of_month alias d01b977 [Tarek Auel] [SPARK-8199] python underscore 56c4a92 [Tarek Auel] [SPARK-8199] update python docu e223bc0 [Tarek Auel] [SPARK-8199] refactoring d6aa14e [Tarek Auel] [SPARK-8199] fixed Hive compatibility b382267 [Tarek Auel] [SPARK-8199] fixed bug in day calculation; removed set TimeZone in HiveCompatibilitySuite for test purposes; removed Hive tests for second and minute, because we can cast '2015-03-18' to a timestamp and extract a minute/second from it 1b2e540 [Tarek Auel] [SPARK-8119] style fix 0852655 [Tarek Auel] [SPARK-8119] changed from ExpectsInputTypes to implicit casts ec87c69 [Tarek Auel] [SPARK-8119] bug fixing and refactoring 1358cdc [Tarek Auel] Merge remote-tracking branch 'origin/master' into SPARK-8199 740af0e [Tarek Auel] implement date function using a calculation based on days 4fb66da [Tarek Auel] WIP: date functions on calculation only 1a436c9 [Tarek Auel] wip f775f39 [Tarek Auel] fixed return type ad17e96 [Tarek Auel] improved implementation c42b444 [Tarek Auel] Removed merge conflict file ccb723c [Tarek Auel] [SPARK-8199] style and fixed merge issues 10e4ad1 [Tarek Auel] Merge branch 'master' into date-functions-fast 7d9f0eb [Tarek Auel] [SPARK-8199] git renaming issue f3e7a9f [Tarek Auel] [SPARK-8199] revert change in DataFrameFunctionsSuite 6f5d95c [Tarek Auel] [SPARK-8199] fixed year interval d9f8ac3 [Tarek Auel] [SPARK-8199] implement fast track 7bc9d93 [Tarek Auel] Merge branch 'master' into SPARK-8199 5a105d9 [Tarek Auel] [SPARK-8199] rebase after #6985 got merged eb6760d [Tarek Auel] Merge branch 'master' into SPARK-8199 f120415 [Tarek Auel] improved runtime a8edebd [Tarek Auel] use Calendar instead of SimpleDateFormat 5fe74e1 [Tarek Auel] fixed python style 3bfac90 [Tarek Auel] fixed style 356df78 [Tarek Auel] rely on cast mechanism of Spark. Simplified implementation 02efc5d [Tarek Auel] removed doubled code a5ea120 [Tarek Auel] added python api; changed test to be more meaningful b680db6 [Tarek Auel] added codegeneration to all functions c739788 [Tarek Auel] added support for quarter SPARK-8178 849fb41 [Tarek Auel] fixed stupid test 638596f [Tarek Auel] improved codegen 4d8049b [Tarek Auel] fixed tests and added type check 5ebb235 [Tarek Auel] resolved naming conflict d0e2f99 [Tarek Auel] date functions
2015-07-19 01:48:05 -04:00
>>> df = spark.createDataFrame([('2015-04-08',)], ['dt'])
>>> df.select(date_format('dt', 'MM/dd/yyy').alias('date')).collect()
[SPARK-8199][SPARK-8184][SPARK-8183][SPARK-8182][SPARK-8181][SPARK-8180][SPARK-8179][SPARK-8177][SPARK-8178][SPARK-9115][SQL] date functions Jira: https://issues.apache.org/jira/browse/SPARK-8199 https://issues.apache.org/jira/browse/SPARK-8184 https://issues.apache.org/jira/browse/SPARK-8183 https://issues.apache.org/jira/browse/SPARK-8182 https://issues.apache.org/jira/browse/SPARK-8181 https://issues.apache.org/jira/browse/SPARK-8180 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-8177 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-9115 Regarding `day`and `dayofmonth` are both necessary? ~~I am going to add `Quarter` to this PR as well.~~ Done. ~~As soon as the Scala coding is reviewed and discussed, I'll add the python api.~~ Done Author: Tarek Auel <tarek.auel@googlemail.com> Author: Tarek Auel <tarek.auel@gmail.com> Closes #6981 from tarekauel/SPARK-8199 and squashes the following commits: f7b4c8c [Tarek Auel] [SPARK-8199] fixed bug in tests bb567b6 [Tarek Auel] [SPARK-8199] fixed test 3e095ba [Tarek Auel] [SPARK-8199] style and timezone fix 256c357 [Tarek Auel] [SPARK-8199] code cleanup 5983dcc [Tarek Auel] [SPARK-8199] whitespace fix 6e0c78f [Tarek Auel] [SPARK-8199] removed setTimeZone in tests, according to cloud-fans comment in #7488 4afc09c [Tarek Auel] [SPARK-8199] concise leap year handling ea6c110 [Tarek Auel] [SPARK-8199] fix after merging master 70238e0 [Tarek Auel] Merge branch 'master' into SPARK-8199 3c6ae2e [Tarek Auel] [SPARK-8199] removed binary search fb98ba0 [Tarek Auel] [SPARK-8199] python docstring fix cdfae27 [Tarek Auel] [SPARK-8199] cleanup & python docstring fix 746b80a [Tarek Auel] [SPARK-8199] build fix 0ad6db8 [Tarek Auel] [SPARK-8199] minor fix 523542d [Tarek Auel] [SPARK-8199] address comments 2259299 [Tarek Auel] [SPARK-8199] day_of_month alias d01b977 [Tarek Auel] [SPARK-8199] python underscore 56c4a92 [Tarek Auel] [SPARK-8199] update python docu e223bc0 [Tarek Auel] [SPARK-8199] refactoring d6aa14e [Tarek Auel] [SPARK-8199] fixed Hive compatibility b382267 [Tarek Auel] [SPARK-8199] fixed bug in day calculation; removed set TimeZone in HiveCompatibilitySuite for test purposes; removed Hive tests for second and minute, because we can cast '2015-03-18' to a timestamp and extract a minute/second from it 1b2e540 [Tarek Auel] [SPARK-8119] style fix 0852655 [Tarek Auel] [SPARK-8119] changed from ExpectsInputTypes to implicit casts ec87c69 [Tarek Auel] [SPARK-8119] bug fixing and refactoring 1358cdc [Tarek Auel] Merge remote-tracking branch 'origin/master' into SPARK-8199 740af0e [Tarek Auel] implement date function using a calculation based on days 4fb66da [Tarek Auel] WIP: date functions on calculation only 1a436c9 [Tarek Auel] wip f775f39 [Tarek Auel] fixed return type ad17e96 [Tarek Auel] improved implementation c42b444 [Tarek Auel] Removed merge conflict file ccb723c [Tarek Auel] [SPARK-8199] style and fixed merge issues 10e4ad1 [Tarek Auel] Merge branch 'master' into date-functions-fast 7d9f0eb [Tarek Auel] [SPARK-8199] git renaming issue f3e7a9f [Tarek Auel] [SPARK-8199] revert change in DataFrameFunctionsSuite 6f5d95c [Tarek Auel] [SPARK-8199] fixed year interval d9f8ac3 [Tarek Auel] [SPARK-8199] implement fast track 7bc9d93 [Tarek Auel] Merge branch 'master' into SPARK-8199 5a105d9 [Tarek Auel] [SPARK-8199] rebase after #6985 got merged eb6760d [Tarek Auel] Merge branch 'master' into SPARK-8199 f120415 [Tarek Auel] improved runtime a8edebd [Tarek Auel] use Calendar instead of SimpleDateFormat 5fe74e1 [Tarek Auel] fixed python style 3bfac90 [Tarek Auel] fixed style 356df78 [Tarek Auel] rely on cast mechanism of Spark. Simplified implementation 02efc5d [Tarek Auel] removed doubled code a5ea120 [Tarek Auel] added python api; changed test to be more meaningful b680db6 [Tarek Auel] added codegeneration to all functions c739788 [Tarek Auel] added support for quarter SPARK-8178 849fb41 [Tarek Auel] fixed stupid test 638596f [Tarek Auel] improved codegen 4d8049b [Tarek Auel] fixed tests and added type check 5ebb235 [Tarek Auel] resolved naming conflict d0e2f99 [Tarek Auel] date functions
2015-07-19 01:48:05 -04:00
[Row(date=u'04/08/2015')]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.date_format(_to_java_column(date), format))
[SPARK-8199][SPARK-8184][SPARK-8183][SPARK-8182][SPARK-8181][SPARK-8180][SPARK-8179][SPARK-8177][SPARK-8178][SPARK-9115][SQL] date functions Jira: https://issues.apache.org/jira/browse/SPARK-8199 https://issues.apache.org/jira/browse/SPARK-8184 https://issues.apache.org/jira/browse/SPARK-8183 https://issues.apache.org/jira/browse/SPARK-8182 https://issues.apache.org/jira/browse/SPARK-8181 https://issues.apache.org/jira/browse/SPARK-8180 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-8177 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-9115 Regarding `day`and `dayofmonth` are both necessary? ~~I am going to add `Quarter` to this PR as well.~~ Done. ~~As soon as the Scala coding is reviewed and discussed, I'll add the python api.~~ Done Author: Tarek Auel <tarek.auel@googlemail.com> Author: Tarek Auel <tarek.auel@gmail.com> Closes #6981 from tarekauel/SPARK-8199 and squashes the following commits: f7b4c8c [Tarek Auel] [SPARK-8199] fixed bug in tests bb567b6 [Tarek Auel] [SPARK-8199] fixed test 3e095ba [Tarek Auel] [SPARK-8199] style and timezone fix 256c357 [Tarek Auel] [SPARK-8199] code cleanup 5983dcc [Tarek Auel] [SPARK-8199] whitespace fix 6e0c78f [Tarek Auel] [SPARK-8199] removed setTimeZone in tests, according to cloud-fans comment in #7488 4afc09c [Tarek Auel] [SPARK-8199] concise leap year handling ea6c110 [Tarek Auel] [SPARK-8199] fix after merging master 70238e0 [Tarek Auel] Merge branch 'master' into SPARK-8199 3c6ae2e [Tarek Auel] [SPARK-8199] removed binary search fb98ba0 [Tarek Auel] [SPARK-8199] python docstring fix cdfae27 [Tarek Auel] [SPARK-8199] cleanup & python docstring fix 746b80a [Tarek Auel] [SPARK-8199] build fix 0ad6db8 [Tarek Auel] [SPARK-8199] minor fix 523542d [Tarek Auel] [SPARK-8199] address comments 2259299 [Tarek Auel] [SPARK-8199] day_of_month alias d01b977 [Tarek Auel] [SPARK-8199] python underscore 56c4a92 [Tarek Auel] [SPARK-8199] update python docu e223bc0 [Tarek Auel] [SPARK-8199] refactoring d6aa14e [Tarek Auel] [SPARK-8199] fixed Hive compatibility b382267 [Tarek Auel] [SPARK-8199] fixed bug in day calculation; removed set TimeZone in HiveCompatibilitySuite for test purposes; removed Hive tests for second and minute, because we can cast '2015-03-18' to a timestamp and extract a minute/second from it 1b2e540 [Tarek Auel] [SPARK-8119] style fix 0852655 [Tarek Auel] [SPARK-8119] changed from ExpectsInputTypes to implicit casts ec87c69 [Tarek Auel] [SPARK-8119] bug fixing and refactoring 1358cdc [Tarek Auel] Merge remote-tracking branch 'origin/master' into SPARK-8199 740af0e [Tarek Auel] implement date function using a calculation based on days 4fb66da [Tarek Auel] WIP: date functions on calculation only 1a436c9 [Tarek Auel] wip f775f39 [Tarek Auel] fixed return type ad17e96 [Tarek Auel] improved implementation c42b444 [Tarek Auel] Removed merge conflict file ccb723c [Tarek Auel] [SPARK-8199] style and fixed merge issues 10e4ad1 [Tarek Auel] Merge branch 'master' into date-functions-fast 7d9f0eb [Tarek Auel] [SPARK-8199] git renaming issue f3e7a9f [Tarek Auel] [SPARK-8199] revert change in DataFrameFunctionsSuite 6f5d95c [Tarek Auel] [SPARK-8199] fixed year interval d9f8ac3 [Tarek Auel] [SPARK-8199] implement fast track 7bc9d93 [Tarek Auel] Merge branch 'master' into SPARK-8199 5a105d9 [Tarek Auel] [SPARK-8199] rebase after #6985 got merged eb6760d [Tarek Auel] Merge branch 'master' into SPARK-8199 f120415 [Tarek Auel] improved runtime a8edebd [Tarek Auel] use Calendar instead of SimpleDateFormat 5fe74e1 [Tarek Auel] fixed python style 3bfac90 [Tarek Auel] fixed style 356df78 [Tarek Auel] rely on cast mechanism of Spark. Simplified implementation 02efc5d [Tarek Auel] removed doubled code a5ea120 [Tarek Auel] added python api; changed test to be more meaningful b680db6 [Tarek Auel] added codegeneration to all functions c739788 [Tarek Auel] added support for quarter SPARK-8178 849fb41 [Tarek Auel] fixed stupid test 638596f [Tarek Auel] improved codegen 4d8049b [Tarek Auel] fixed tests and added type check 5ebb235 [Tarek Auel] resolved naming conflict d0e2f99 [Tarek Auel] date functions
2015-07-19 01:48:05 -04:00
@since(1.5)
def year(col):
"""
Extract the year of a given date as integer.
>>> df = spark.createDataFrame([('2015-04-08',)], ['dt'])
>>> df.select(year('dt').alias('year')).collect()
[SPARK-8199][SPARK-8184][SPARK-8183][SPARK-8182][SPARK-8181][SPARK-8180][SPARK-8179][SPARK-8177][SPARK-8178][SPARK-9115][SQL] date functions Jira: https://issues.apache.org/jira/browse/SPARK-8199 https://issues.apache.org/jira/browse/SPARK-8184 https://issues.apache.org/jira/browse/SPARK-8183 https://issues.apache.org/jira/browse/SPARK-8182 https://issues.apache.org/jira/browse/SPARK-8181 https://issues.apache.org/jira/browse/SPARK-8180 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-8177 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-9115 Regarding `day`and `dayofmonth` are both necessary? ~~I am going to add `Quarter` to this PR as well.~~ Done. ~~As soon as the Scala coding is reviewed and discussed, I'll add the python api.~~ Done Author: Tarek Auel <tarek.auel@googlemail.com> Author: Tarek Auel <tarek.auel@gmail.com> Closes #6981 from tarekauel/SPARK-8199 and squashes the following commits: f7b4c8c [Tarek Auel] [SPARK-8199] fixed bug in tests bb567b6 [Tarek Auel] [SPARK-8199] fixed test 3e095ba [Tarek Auel] [SPARK-8199] style and timezone fix 256c357 [Tarek Auel] [SPARK-8199] code cleanup 5983dcc [Tarek Auel] [SPARK-8199] whitespace fix 6e0c78f [Tarek Auel] [SPARK-8199] removed setTimeZone in tests, according to cloud-fans comment in #7488 4afc09c [Tarek Auel] [SPARK-8199] concise leap year handling ea6c110 [Tarek Auel] [SPARK-8199] fix after merging master 70238e0 [Tarek Auel] Merge branch 'master' into SPARK-8199 3c6ae2e [Tarek Auel] [SPARK-8199] removed binary search fb98ba0 [Tarek Auel] [SPARK-8199] python docstring fix cdfae27 [Tarek Auel] [SPARK-8199] cleanup & python docstring fix 746b80a [Tarek Auel] [SPARK-8199] build fix 0ad6db8 [Tarek Auel] [SPARK-8199] minor fix 523542d [Tarek Auel] [SPARK-8199] address comments 2259299 [Tarek Auel] [SPARK-8199] day_of_month alias d01b977 [Tarek Auel] [SPARK-8199] python underscore 56c4a92 [Tarek Auel] [SPARK-8199] update python docu e223bc0 [Tarek Auel] [SPARK-8199] refactoring d6aa14e [Tarek Auel] [SPARK-8199] fixed Hive compatibility b382267 [Tarek Auel] [SPARK-8199] fixed bug in day calculation; removed set TimeZone in HiveCompatibilitySuite for test purposes; removed Hive tests for second and minute, because we can cast '2015-03-18' to a timestamp and extract a minute/second from it 1b2e540 [Tarek Auel] [SPARK-8119] style fix 0852655 [Tarek Auel] [SPARK-8119] changed from ExpectsInputTypes to implicit casts ec87c69 [Tarek Auel] [SPARK-8119] bug fixing and refactoring 1358cdc [Tarek Auel] Merge remote-tracking branch 'origin/master' into SPARK-8199 740af0e [Tarek Auel] implement date function using a calculation based on days 4fb66da [Tarek Auel] WIP: date functions on calculation only 1a436c9 [Tarek Auel] wip f775f39 [Tarek Auel] fixed return type ad17e96 [Tarek Auel] improved implementation c42b444 [Tarek Auel] Removed merge conflict file ccb723c [Tarek Auel] [SPARK-8199] style and fixed merge issues 10e4ad1 [Tarek Auel] Merge branch 'master' into date-functions-fast 7d9f0eb [Tarek Auel] [SPARK-8199] git renaming issue f3e7a9f [Tarek Auel] [SPARK-8199] revert change in DataFrameFunctionsSuite 6f5d95c [Tarek Auel] [SPARK-8199] fixed year interval d9f8ac3 [Tarek Auel] [SPARK-8199] implement fast track 7bc9d93 [Tarek Auel] Merge branch 'master' into SPARK-8199 5a105d9 [Tarek Auel] [SPARK-8199] rebase after #6985 got merged eb6760d [Tarek Auel] Merge branch 'master' into SPARK-8199 f120415 [Tarek Auel] improved runtime a8edebd [Tarek Auel] use Calendar instead of SimpleDateFormat 5fe74e1 [Tarek Auel] fixed python style 3bfac90 [Tarek Auel] fixed style 356df78 [Tarek Auel] rely on cast mechanism of Spark. Simplified implementation 02efc5d [Tarek Auel] removed doubled code a5ea120 [Tarek Auel] added python api; changed test to be more meaningful b680db6 [Tarek Auel] added codegeneration to all functions c739788 [Tarek Auel] added support for quarter SPARK-8178 849fb41 [Tarek Auel] fixed stupid test 638596f [Tarek Auel] improved codegen 4d8049b [Tarek Auel] fixed tests and added type check 5ebb235 [Tarek Auel] resolved naming conflict d0e2f99 [Tarek Auel] date functions
2015-07-19 01:48:05 -04:00
[Row(year=2015)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.year(_to_java_column(col)))
[SPARK-8199][SPARK-8184][SPARK-8183][SPARK-8182][SPARK-8181][SPARK-8180][SPARK-8179][SPARK-8177][SPARK-8178][SPARK-9115][SQL] date functions Jira: https://issues.apache.org/jira/browse/SPARK-8199 https://issues.apache.org/jira/browse/SPARK-8184 https://issues.apache.org/jira/browse/SPARK-8183 https://issues.apache.org/jira/browse/SPARK-8182 https://issues.apache.org/jira/browse/SPARK-8181 https://issues.apache.org/jira/browse/SPARK-8180 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-8177 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-9115 Regarding `day`and `dayofmonth` are both necessary? ~~I am going to add `Quarter` to this PR as well.~~ Done. ~~As soon as the Scala coding is reviewed and discussed, I'll add the python api.~~ Done Author: Tarek Auel <tarek.auel@googlemail.com> Author: Tarek Auel <tarek.auel@gmail.com> Closes #6981 from tarekauel/SPARK-8199 and squashes the following commits: f7b4c8c [Tarek Auel] [SPARK-8199] fixed bug in tests bb567b6 [Tarek Auel] [SPARK-8199] fixed test 3e095ba [Tarek Auel] [SPARK-8199] style and timezone fix 256c357 [Tarek Auel] [SPARK-8199] code cleanup 5983dcc [Tarek Auel] [SPARK-8199] whitespace fix 6e0c78f [Tarek Auel] [SPARK-8199] removed setTimeZone in tests, according to cloud-fans comment in #7488 4afc09c [Tarek Auel] [SPARK-8199] concise leap year handling ea6c110 [Tarek Auel] [SPARK-8199] fix after merging master 70238e0 [Tarek Auel] Merge branch 'master' into SPARK-8199 3c6ae2e [Tarek Auel] [SPARK-8199] removed binary search fb98ba0 [Tarek Auel] [SPARK-8199] python docstring fix cdfae27 [Tarek Auel] [SPARK-8199] cleanup & python docstring fix 746b80a [Tarek Auel] [SPARK-8199] build fix 0ad6db8 [Tarek Auel] [SPARK-8199] minor fix 523542d [Tarek Auel] [SPARK-8199] address comments 2259299 [Tarek Auel] [SPARK-8199] day_of_month alias d01b977 [Tarek Auel] [SPARK-8199] python underscore 56c4a92 [Tarek Auel] [SPARK-8199] update python docu e223bc0 [Tarek Auel] [SPARK-8199] refactoring d6aa14e [Tarek Auel] [SPARK-8199] fixed Hive compatibility b382267 [Tarek Auel] [SPARK-8199] fixed bug in day calculation; removed set TimeZone in HiveCompatibilitySuite for test purposes; removed Hive tests for second and minute, because we can cast '2015-03-18' to a timestamp and extract a minute/second from it 1b2e540 [Tarek Auel] [SPARK-8119] style fix 0852655 [Tarek Auel] [SPARK-8119] changed from ExpectsInputTypes to implicit casts ec87c69 [Tarek Auel] [SPARK-8119] bug fixing and refactoring 1358cdc [Tarek Auel] Merge remote-tracking branch 'origin/master' into SPARK-8199 740af0e [Tarek Auel] implement date function using a calculation based on days 4fb66da [Tarek Auel] WIP: date functions on calculation only 1a436c9 [Tarek Auel] wip f775f39 [Tarek Auel] fixed return type ad17e96 [Tarek Auel] improved implementation c42b444 [Tarek Auel] Removed merge conflict file ccb723c [Tarek Auel] [SPARK-8199] style and fixed merge issues 10e4ad1 [Tarek Auel] Merge branch 'master' into date-functions-fast 7d9f0eb [Tarek Auel] [SPARK-8199] git renaming issue f3e7a9f [Tarek Auel] [SPARK-8199] revert change in DataFrameFunctionsSuite 6f5d95c [Tarek Auel] [SPARK-8199] fixed year interval d9f8ac3 [Tarek Auel] [SPARK-8199] implement fast track 7bc9d93 [Tarek Auel] Merge branch 'master' into SPARK-8199 5a105d9 [Tarek Auel] [SPARK-8199] rebase after #6985 got merged eb6760d [Tarek Auel] Merge branch 'master' into SPARK-8199 f120415 [Tarek Auel] improved runtime a8edebd [Tarek Auel] use Calendar instead of SimpleDateFormat 5fe74e1 [Tarek Auel] fixed python style 3bfac90 [Tarek Auel] fixed style 356df78 [Tarek Auel] rely on cast mechanism of Spark. Simplified implementation 02efc5d [Tarek Auel] removed doubled code a5ea120 [Tarek Auel] added python api; changed test to be more meaningful b680db6 [Tarek Auel] added codegeneration to all functions c739788 [Tarek Auel] added support for quarter SPARK-8178 849fb41 [Tarek Auel] fixed stupid test 638596f [Tarek Auel] improved codegen 4d8049b [Tarek Auel] fixed tests and added type check 5ebb235 [Tarek Auel] resolved naming conflict d0e2f99 [Tarek Auel] date functions
2015-07-19 01:48:05 -04:00
@since(1.5)
def quarter(col):
"""
Extract the quarter of a given date as integer.
>>> df = spark.createDataFrame([('2015-04-08',)], ['dt'])
>>> df.select(quarter('dt').alias('quarter')).collect()
[SPARK-8199][SPARK-8184][SPARK-8183][SPARK-8182][SPARK-8181][SPARK-8180][SPARK-8179][SPARK-8177][SPARK-8178][SPARK-9115][SQL] date functions Jira: https://issues.apache.org/jira/browse/SPARK-8199 https://issues.apache.org/jira/browse/SPARK-8184 https://issues.apache.org/jira/browse/SPARK-8183 https://issues.apache.org/jira/browse/SPARK-8182 https://issues.apache.org/jira/browse/SPARK-8181 https://issues.apache.org/jira/browse/SPARK-8180 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-8177 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-9115 Regarding `day`and `dayofmonth` are both necessary? ~~I am going to add `Quarter` to this PR as well.~~ Done. ~~As soon as the Scala coding is reviewed and discussed, I'll add the python api.~~ Done Author: Tarek Auel <tarek.auel@googlemail.com> Author: Tarek Auel <tarek.auel@gmail.com> Closes #6981 from tarekauel/SPARK-8199 and squashes the following commits: f7b4c8c [Tarek Auel] [SPARK-8199] fixed bug in tests bb567b6 [Tarek Auel] [SPARK-8199] fixed test 3e095ba [Tarek Auel] [SPARK-8199] style and timezone fix 256c357 [Tarek Auel] [SPARK-8199] code cleanup 5983dcc [Tarek Auel] [SPARK-8199] whitespace fix 6e0c78f [Tarek Auel] [SPARK-8199] removed setTimeZone in tests, according to cloud-fans comment in #7488 4afc09c [Tarek Auel] [SPARK-8199] concise leap year handling ea6c110 [Tarek Auel] [SPARK-8199] fix after merging master 70238e0 [Tarek Auel] Merge branch 'master' into SPARK-8199 3c6ae2e [Tarek Auel] [SPARK-8199] removed binary search fb98ba0 [Tarek Auel] [SPARK-8199] python docstring fix cdfae27 [Tarek Auel] [SPARK-8199] cleanup & python docstring fix 746b80a [Tarek Auel] [SPARK-8199] build fix 0ad6db8 [Tarek Auel] [SPARK-8199] minor fix 523542d [Tarek Auel] [SPARK-8199] address comments 2259299 [Tarek Auel] [SPARK-8199] day_of_month alias d01b977 [Tarek Auel] [SPARK-8199] python underscore 56c4a92 [Tarek Auel] [SPARK-8199] update python docu e223bc0 [Tarek Auel] [SPARK-8199] refactoring d6aa14e [Tarek Auel] [SPARK-8199] fixed Hive compatibility b382267 [Tarek Auel] [SPARK-8199] fixed bug in day calculation; removed set TimeZone in HiveCompatibilitySuite for test purposes; removed Hive tests for second and minute, because we can cast '2015-03-18' to a timestamp and extract a minute/second from it 1b2e540 [Tarek Auel] [SPARK-8119] style fix 0852655 [Tarek Auel] [SPARK-8119] changed from ExpectsInputTypes to implicit casts ec87c69 [Tarek Auel] [SPARK-8119] bug fixing and refactoring 1358cdc [Tarek Auel] Merge remote-tracking branch 'origin/master' into SPARK-8199 740af0e [Tarek Auel] implement date function using a calculation based on days 4fb66da [Tarek Auel] WIP: date functions on calculation only 1a436c9 [Tarek Auel] wip f775f39 [Tarek Auel] fixed return type ad17e96 [Tarek Auel] improved implementation c42b444 [Tarek Auel] Removed merge conflict file ccb723c [Tarek Auel] [SPARK-8199] style and fixed merge issues 10e4ad1 [Tarek Auel] Merge branch 'master' into date-functions-fast 7d9f0eb [Tarek Auel] [SPARK-8199] git renaming issue f3e7a9f [Tarek Auel] [SPARK-8199] revert change in DataFrameFunctionsSuite 6f5d95c [Tarek Auel] [SPARK-8199] fixed year interval d9f8ac3 [Tarek Auel] [SPARK-8199] implement fast track 7bc9d93 [Tarek Auel] Merge branch 'master' into SPARK-8199 5a105d9 [Tarek Auel] [SPARK-8199] rebase after #6985 got merged eb6760d [Tarek Auel] Merge branch 'master' into SPARK-8199 f120415 [Tarek Auel] improved runtime a8edebd [Tarek Auel] use Calendar instead of SimpleDateFormat 5fe74e1 [Tarek Auel] fixed python style 3bfac90 [Tarek Auel] fixed style 356df78 [Tarek Auel] rely on cast mechanism of Spark. Simplified implementation 02efc5d [Tarek Auel] removed doubled code a5ea120 [Tarek Auel] added python api; changed test to be more meaningful b680db6 [Tarek Auel] added codegeneration to all functions c739788 [Tarek Auel] added support for quarter SPARK-8178 849fb41 [Tarek Auel] fixed stupid test 638596f [Tarek Auel] improved codegen 4d8049b [Tarek Auel] fixed tests and added type check 5ebb235 [Tarek Auel] resolved naming conflict d0e2f99 [Tarek Auel] date functions
2015-07-19 01:48:05 -04:00
[Row(quarter=2)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.quarter(_to_java_column(col)))
[SPARK-8199][SPARK-8184][SPARK-8183][SPARK-8182][SPARK-8181][SPARK-8180][SPARK-8179][SPARK-8177][SPARK-8178][SPARK-9115][SQL] date functions Jira: https://issues.apache.org/jira/browse/SPARK-8199 https://issues.apache.org/jira/browse/SPARK-8184 https://issues.apache.org/jira/browse/SPARK-8183 https://issues.apache.org/jira/browse/SPARK-8182 https://issues.apache.org/jira/browse/SPARK-8181 https://issues.apache.org/jira/browse/SPARK-8180 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-8177 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-9115 Regarding `day`and `dayofmonth` are both necessary? ~~I am going to add `Quarter` to this PR as well.~~ Done. ~~As soon as the Scala coding is reviewed and discussed, I'll add the python api.~~ Done Author: Tarek Auel <tarek.auel@googlemail.com> Author: Tarek Auel <tarek.auel@gmail.com> Closes #6981 from tarekauel/SPARK-8199 and squashes the following commits: f7b4c8c [Tarek Auel] [SPARK-8199] fixed bug in tests bb567b6 [Tarek Auel] [SPARK-8199] fixed test 3e095ba [Tarek Auel] [SPARK-8199] style and timezone fix 256c357 [Tarek Auel] [SPARK-8199] code cleanup 5983dcc [Tarek Auel] [SPARK-8199] whitespace fix 6e0c78f [Tarek Auel] [SPARK-8199] removed setTimeZone in tests, according to cloud-fans comment in #7488 4afc09c [Tarek Auel] [SPARK-8199] concise leap year handling ea6c110 [Tarek Auel] [SPARK-8199] fix after merging master 70238e0 [Tarek Auel] Merge branch 'master' into SPARK-8199 3c6ae2e [Tarek Auel] [SPARK-8199] removed binary search fb98ba0 [Tarek Auel] [SPARK-8199] python docstring fix cdfae27 [Tarek Auel] [SPARK-8199] cleanup & python docstring fix 746b80a [Tarek Auel] [SPARK-8199] build fix 0ad6db8 [Tarek Auel] [SPARK-8199] minor fix 523542d [Tarek Auel] [SPARK-8199] address comments 2259299 [Tarek Auel] [SPARK-8199] day_of_month alias d01b977 [Tarek Auel] [SPARK-8199] python underscore 56c4a92 [Tarek Auel] [SPARK-8199] update python docu e223bc0 [Tarek Auel] [SPARK-8199] refactoring d6aa14e [Tarek Auel] [SPARK-8199] fixed Hive compatibility b382267 [Tarek Auel] [SPARK-8199] fixed bug in day calculation; removed set TimeZone in HiveCompatibilitySuite for test purposes; removed Hive tests for second and minute, because we can cast '2015-03-18' to a timestamp and extract a minute/second from it 1b2e540 [Tarek Auel] [SPARK-8119] style fix 0852655 [Tarek Auel] [SPARK-8119] changed from ExpectsInputTypes to implicit casts ec87c69 [Tarek Auel] [SPARK-8119] bug fixing and refactoring 1358cdc [Tarek Auel] Merge remote-tracking branch 'origin/master' into SPARK-8199 740af0e [Tarek Auel] implement date function using a calculation based on days 4fb66da [Tarek Auel] WIP: date functions on calculation only 1a436c9 [Tarek Auel] wip f775f39 [Tarek Auel] fixed return type ad17e96 [Tarek Auel] improved implementation c42b444 [Tarek Auel] Removed merge conflict file ccb723c [Tarek Auel] [SPARK-8199] style and fixed merge issues 10e4ad1 [Tarek Auel] Merge branch 'master' into date-functions-fast 7d9f0eb [Tarek Auel] [SPARK-8199] git renaming issue f3e7a9f [Tarek Auel] [SPARK-8199] revert change in DataFrameFunctionsSuite 6f5d95c [Tarek Auel] [SPARK-8199] fixed year interval d9f8ac3 [Tarek Auel] [SPARK-8199] implement fast track 7bc9d93 [Tarek Auel] Merge branch 'master' into SPARK-8199 5a105d9 [Tarek Auel] [SPARK-8199] rebase after #6985 got merged eb6760d [Tarek Auel] Merge branch 'master' into SPARK-8199 f120415 [Tarek Auel] improved runtime a8edebd [Tarek Auel] use Calendar instead of SimpleDateFormat 5fe74e1 [Tarek Auel] fixed python style 3bfac90 [Tarek Auel] fixed style 356df78 [Tarek Auel] rely on cast mechanism of Spark. Simplified implementation 02efc5d [Tarek Auel] removed doubled code a5ea120 [Tarek Auel] added python api; changed test to be more meaningful b680db6 [Tarek Auel] added codegeneration to all functions c739788 [Tarek Auel] added support for quarter SPARK-8178 849fb41 [Tarek Auel] fixed stupid test 638596f [Tarek Auel] improved codegen 4d8049b [Tarek Auel] fixed tests and added type check 5ebb235 [Tarek Auel] resolved naming conflict d0e2f99 [Tarek Auel] date functions
2015-07-19 01:48:05 -04:00
@since(1.5)
def month(col):
"""
Extract the month of a given date as integer.
>>> df = spark.createDataFrame([('2015-04-08',)], ['dt'])
>>> df.select(month('dt').alias('month')).collect()
[SPARK-8199][SPARK-8184][SPARK-8183][SPARK-8182][SPARK-8181][SPARK-8180][SPARK-8179][SPARK-8177][SPARK-8178][SPARK-9115][SQL] date functions Jira: https://issues.apache.org/jira/browse/SPARK-8199 https://issues.apache.org/jira/browse/SPARK-8184 https://issues.apache.org/jira/browse/SPARK-8183 https://issues.apache.org/jira/browse/SPARK-8182 https://issues.apache.org/jira/browse/SPARK-8181 https://issues.apache.org/jira/browse/SPARK-8180 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-8177 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-9115 Regarding `day`and `dayofmonth` are both necessary? ~~I am going to add `Quarter` to this PR as well.~~ Done. ~~As soon as the Scala coding is reviewed and discussed, I'll add the python api.~~ Done Author: Tarek Auel <tarek.auel@googlemail.com> Author: Tarek Auel <tarek.auel@gmail.com> Closes #6981 from tarekauel/SPARK-8199 and squashes the following commits: f7b4c8c [Tarek Auel] [SPARK-8199] fixed bug in tests bb567b6 [Tarek Auel] [SPARK-8199] fixed test 3e095ba [Tarek Auel] [SPARK-8199] style and timezone fix 256c357 [Tarek Auel] [SPARK-8199] code cleanup 5983dcc [Tarek Auel] [SPARK-8199] whitespace fix 6e0c78f [Tarek Auel] [SPARK-8199] removed setTimeZone in tests, according to cloud-fans comment in #7488 4afc09c [Tarek Auel] [SPARK-8199] concise leap year handling ea6c110 [Tarek Auel] [SPARK-8199] fix after merging master 70238e0 [Tarek Auel] Merge branch 'master' into SPARK-8199 3c6ae2e [Tarek Auel] [SPARK-8199] removed binary search fb98ba0 [Tarek Auel] [SPARK-8199] python docstring fix cdfae27 [Tarek Auel] [SPARK-8199] cleanup & python docstring fix 746b80a [Tarek Auel] [SPARK-8199] build fix 0ad6db8 [Tarek Auel] [SPARK-8199] minor fix 523542d [Tarek Auel] [SPARK-8199] address comments 2259299 [Tarek Auel] [SPARK-8199] day_of_month alias d01b977 [Tarek Auel] [SPARK-8199] python underscore 56c4a92 [Tarek Auel] [SPARK-8199] update python docu e223bc0 [Tarek Auel] [SPARK-8199] refactoring d6aa14e [Tarek Auel] [SPARK-8199] fixed Hive compatibility b382267 [Tarek Auel] [SPARK-8199] fixed bug in day calculation; removed set TimeZone in HiveCompatibilitySuite for test purposes; removed Hive tests for second and minute, because we can cast '2015-03-18' to a timestamp and extract a minute/second from it 1b2e540 [Tarek Auel] [SPARK-8119] style fix 0852655 [Tarek Auel] [SPARK-8119] changed from ExpectsInputTypes to implicit casts ec87c69 [Tarek Auel] [SPARK-8119] bug fixing and refactoring 1358cdc [Tarek Auel] Merge remote-tracking branch 'origin/master' into SPARK-8199 740af0e [Tarek Auel] implement date function using a calculation based on days 4fb66da [Tarek Auel] WIP: date functions on calculation only 1a436c9 [Tarek Auel] wip f775f39 [Tarek Auel] fixed return type ad17e96 [Tarek Auel] improved implementation c42b444 [Tarek Auel] Removed merge conflict file ccb723c [Tarek Auel] [SPARK-8199] style and fixed merge issues 10e4ad1 [Tarek Auel] Merge branch 'master' into date-functions-fast 7d9f0eb [Tarek Auel] [SPARK-8199] git renaming issue f3e7a9f [Tarek Auel] [SPARK-8199] revert change in DataFrameFunctionsSuite 6f5d95c [Tarek Auel] [SPARK-8199] fixed year interval d9f8ac3 [Tarek Auel] [SPARK-8199] implement fast track 7bc9d93 [Tarek Auel] Merge branch 'master' into SPARK-8199 5a105d9 [Tarek Auel] [SPARK-8199] rebase after #6985 got merged eb6760d [Tarek Auel] Merge branch 'master' into SPARK-8199 f120415 [Tarek Auel] improved runtime a8edebd [Tarek Auel] use Calendar instead of SimpleDateFormat 5fe74e1 [Tarek Auel] fixed python style 3bfac90 [Tarek Auel] fixed style 356df78 [Tarek Auel] rely on cast mechanism of Spark. Simplified implementation 02efc5d [Tarek Auel] removed doubled code a5ea120 [Tarek Auel] added python api; changed test to be more meaningful b680db6 [Tarek Auel] added codegeneration to all functions c739788 [Tarek Auel] added support for quarter SPARK-8178 849fb41 [Tarek Auel] fixed stupid test 638596f [Tarek Auel] improved codegen 4d8049b [Tarek Auel] fixed tests and added type check 5ebb235 [Tarek Auel] resolved naming conflict d0e2f99 [Tarek Auel] date functions
2015-07-19 01:48:05 -04:00
[Row(month=4)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.month(_to_java_column(col)))
[SPARK-8199][SPARK-8184][SPARK-8183][SPARK-8182][SPARK-8181][SPARK-8180][SPARK-8179][SPARK-8177][SPARK-8178][SPARK-9115][SQL] date functions Jira: https://issues.apache.org/jira/browse/SPARK-8199 https://issues.apache.org/jira/browse/SPARK-8184 https://issues.apache.org/jira/browse/SPARK-8183 https://issues.apache.org/jira/browse/SPARK-8182 https://issues.apache.org/jira/browse/SPARK-8181 https://issues.apache.org/jira/browse/SPARK-8180 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-8177 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-9115 Regarding `day`and `dayofmonth` are both necessary? ~~I am going to add `Quarter` to this PR as well.~~ Done. ~~As soon as the Scala coding is reviewed and discussed, I'll add the python api.~~ Done Author: Tarek Auel <tarek.auel@googlemail.com> Author: Tarek Auel <tarek.auel@gmail.com> Closes #6981 from tarekauel/SPARK-8199 and squashes the following commits: f7b4c8c [Tarek Auel] [SPARK-8199] fixed bug in tests bb567b6 [Tarek Auel] [SPARK-8199] fixed test 3e095ba [Tarek Auel] [SPARK-8199] style and timezone fix 256c357 [Tarek Auel] [SPARK-8199] code cleanup 5983dcc [Tarek Auel] [SPARK-8199] whitespace fix 6e0c78f [Tarek Auel] [SPARK-8199] removed setTimeZone in tests, according to cloud-fans comment in #7488 4afc09c [Tarek Auel] [SPARK-8199] concise leap year handling ea6c110 [Tarek Auel] [SPARK-8199] fix after merging master 70238e0 [Tarek Auel] Merge branch 'master' into SPARK-8199 3c6ae2e [Tarek Auel] [SPARK-8199] removed binary search fb98ba0 [Tarek Auel] [SPARK-8199] python docstring fix cdfae27 [Tarek Auel] [SPARK-8199] cleanup & python docstring fix 746b80a [Tarek Auel] [SPARK-8199] build fix 0ad6db8 [Tarek Auel] [SPARK-8199] minor fix 523542d [Tarek Auel] [SPARK-8199] address comments 2259299 [Tarek Auel] [SPARK-8199] day_of_month alias d01b977 [Tarek Auel] [SPARK-8199] python underscore 56c4a92 [Tarek Auel] [SPARK-8199] update python docu e223bc0 [Tarek Auel] [SPARK-8199] refactoring d6aa14e [Tarek Auel] [SPARK-8199] fixed Hive compatibility b382267 [Tarek Auel] [SPARK-8199] fixed bug in day calculation; removed set TimeZone in HiveCompatibilitySuite for test purposes; removed Hive tests for second and minute, because we can cast '2015-03-18' to a timestamp and extract a minute/second from it 1b2e540 [Tarek Auel] [SPARK-8119] style fix 0852655 [Tarek Auel] [SPARK-8119] changed from ExpectsInputTypes to implicit casts ec87c69 [Tarek Auel] [SPARK-8119] bug fixing and refactoring 1358cdc [Tarek Auel] Merge remote-tracking branch 'origin/master' into SPARK-8199 740af0e [Tarek Auel] implement date function using a calculation based on days 4fb66da [Tarek Auel] WIP: date functions on calculation only 1a436c9 [Tarek Auel] wip f775f39 [Tarek Auel] fixed return type ad17e96 [Tarek Auel] improved implementation c42b444 [Tarek Auel] Removed merge conflict file ccb723c [Tarek Auel] [SPARK-8199] style and fixed merge issues 10e4ad1 [Tarek Auel] Merge branch 'master' into date-functions-fast 7d9f0eb [Tarek Auel] [SPARK-8199] git renaming issue f3e7a9f [Tarek Auel] [SPARK-8199] revert change in DataFrameFunctionsSuite 6f5d95c [Tarek Auel] [SPARK-8199] fixed year interval d9f8ac3 [Tarek Auel] [SPARK-8199] implement fast track 7bc9d93 [Tarek Auel] Merge branch 'master' into SPARK-8199 5a105d9 [Tarek Auel] [SPARK-8199] rebase after #6985 got merged eb6760d [Tarek Auel] Merge branch 'master' into SPARK-8199 f120415 [Tarek Auel] improved runtime a8edebd [Tarek Auel] use Calendar instead of SimpleDateFormat 5fe74e1 [Tarek Auel] fixed python style 3bfac90 [Tarek Auel] fixed style 356df78 [Tarek Auel] rely on cast mechanism of Spark. Simplified implementation 02efc5d [Tarek Auel] removed doubled code a5ea120 [Tarek Auel] added python api; changed test to be more meaningful b680db6 [Tarek Auel] added codegeneration to all functions c739788 [Tarek Auel] added support for quarter SPARK-8178 849fb41 [Tarek Auel] fixed stupid test 638596f [Tarek Auel] improved codegen 4d8049b [Tarek Auel] fixed tests and added type check 5ebb235 [Tarek Auel] resolved naming conflict d0e2f99 [Tarek Auel] date functions
2015-07-19 01:48:05 -04:00
@since(2.3)
def dayofweek(col):
"""
Extract the day of the week of a given date as integer.
>>> df = spark.createDataFrame([('2015-04-08',)], ['dt'])
>>> df.select(dayofweek('dt').alias('day')).collect()
[Row(day=4)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.dayofweek(_to_java_column(col)))
[SPARK-8199][SPARK-8184][SPARK-8183][SPARK-8182][SPARK-8181][SPARK-8180][SPARK-8179][SPARK-8177][SPARK-8178][SPARK-9115][SQL] date functions Jira: https://issues.apache.org/jira/browse/SPARK-8199 https://issues.apache.org/jira/browse/SPARK-8184 https://issues.apache.org/jira/browse/SPARK-8183 https://issues.apache.org/jira/browse/SPARK-8182 https://issues.apache.org/jira/browse/SPARK-8181 https://issues.apache.org/jira/browse/SPARK-8180 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-8177 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-9115 Regarding `day`and `dayofmonth` are both necessary? ~~I am going to add `Quarter` to this PR as well.~~ Done. ~~As soon as the Scala coding is reviewed and discussed, I'll add the python api.~~ Done Author: Tarek Auel <tarek.auel@googlemail.com> Author: Tarek Auel <tarek.auel@gmail.com> Closes #6981 from tarekauel/SPARK-8199 and squashes the following commits: f7b4c8c [Tarek Auel] [SPARK-8199] fixed bug in tests bb567b6 [Tarek Auel] [SPARK-8199] fixed test 3e095ba [Tarek Auel] [SPARK-8199] style and timezone fix 256c357 [Tarek Auel] [SPARK-8199] code cleanup 5983dcc [Tarek Auel] [SPARK-8199] whitespace fix 6e0c78f [Tarek Auel] [SPARK-8199] removed setTimeZone in tests, according to cloud-fans comment in #7488 4afc09c [Tarek Auel] [SPARK-8199] concise leap year handling ea6c110 [Tarek Auel] [SPARK-8199] fix after merging master 70238e0 [Tarek Auel] Merge branch 'master' into SPARK-8199 3c6ae2e [Tarek Auel] [SPARK-8199] removed binary search fb98ba0 [Tarek Auel] [SPARK-8199] python docstring fix cdfae27 [Tarek Auel] [SPARK-8199] cleanup & python docstring fix 746b80a [Tarek Auel] [SPARK-8199] build fix 0ad6db8 [Tarek Auel] [SPARK-8199] minor fix 523542d [Tarek Auel] [SPARK-8199] address comments 2259299 [Tarek Auel] [SPARK-8199] day_of_month alias d01b977 [Tarek Auel] [SPARK-8199] python underscore 56c4a92 [Tarek Auel] [SPARK-8199] update python docu e223bc0 [Tarek Auel] [SPARK-8199] refactoring d6aa14e [Tarek Auel] [SPARK-8199] fixed Hive compatibility b382267 [Tarek Auel] [SPARK-8199] fixed bug in day calculation; removed set TimeZone in HiveCompatibilitySuite for test purposes; removed Hive tests for second and minute, because we can cast '2015-03-18' to a timestamp and extract a minute/second from it 1b2e540 [Tarek Auel] [SPARK-8119] style fix 0852655 [Tarek Auel] [SPARK-8119] changed from ExpectsInputTypes to implicit casts ec87c69 [Tarek Auel] [SPARK-8119] bug fixing and refactoring 1358cdc [Tarek Auel] Merge remote-tracking branch 'origin/master' into SPARK-8199 740af0e [Tarek Auel] implement date function using a calculation based on days 4fb66da [Tarek Auel] WIP: date functions on calculation only 1a436c9 [Tarek Auel] wip f775f39 [Tarek Auel] fixed return type ad17e96 [Tarek Auel] improved implementation c42b444 [Tarek Auel] Removed merge conflict file ccb723c [Tarek Auel] [SPARK-8199] style and fixed merge issues 10e4ad1 [Tarek Auel] Merge branch 'master' into date-functions-fast 7d9f0eb [Tarek Auel] [SPARK-8199] git renaming issue f3e7a9f [Tarek Auel] [SPARK-8199] revert change in DataFrameFunctionsSuite 6f5d95c [Tarek Auel] [SPARK-8199] fixed year interval d9f8ac3 [Tarek Auel] [SPARK-8199] implement fast track 7bc9d93 [Tarek Auel] Merge branch 'master' into SPARK-8199 5a105d9 [Tarek Auel] [SPARK-8199] rebase after #6985 got merged eb6760d [Tarek Auel] Merge branch 'master' into SPARK-8199 f120415 [Tarek Auel] improved runtime a8edebd [Tarek Auel] use Calendar instead of SimpleDateFormat 5fe74e1 [Tarek Auel] fixed python style 3bfac90 [Tarek Auel] fixed style 356df78 [Tarek Auel] rely on cast mechanism of Spark. Simplified implementation 02efc5d [Tarek Auel] removed doubled code a5ea120 [Tarek Auel] added python api; changed test to be more meaningful b680db6 [Tarek Auel] added codegeneration to all functions c739788 [Tarek Auel] added support for quarter SPARK-8178 849fb41 [Tarek Auel] fixed stupid test 638596f [Tarek Auel] improved codegen 4d8049b [Tarek Auel] fixed tests and added type check 5ebb235 [Tarek Auel] resolved naming conflict d0e2f99 [Tarek Auel] date functions
2015-07-19 01:48:05 -04:00
@since(1.5)
def dayofmonth(col):
[SPARK-8199][SPARK-8184][SPARK-8183][SPARK-8182][SPARK-8181][SPARK-8180][SPARK-8179][SPARK-8177][SPARK-8178][SPARK-9115][SQL] date functions Jira: https://issues.apache.org/jira/browse/SPARK-8199 https://issues.apache.org/jira/browse/SPARK-8184 https://issues.apache.org/jira/browse/SPARK-8183 https://issues.apache.org/jira/browse/SPARK-8182 https://issues.apache.org/jira/browse/SPARK-8181 https://issues.apache.org/jira/browse/SPARK-8180 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-8177 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-9115 Regarding `day`and `dayofmonth` are both necessary? ~~I am going to add `Quarter` to this PR as well.~~ Done. ~~As soon as the Scala coding is reviewed and discussed, I'll add the python api.~~ Done Author: Tarek Auel <tarek.auel@googlemail.com> Author: Tarek Auel <tarek.auel@gmail.com> Closes #6981 from tarekauel/SPARK-8199 and squashes the following commits: f7b4c8c [Tarek Auel] [SPARK-8199] fixed bug in tests bb567b6 [Tarek Auel] [SPARK-8199] fixed test 3e095ba [Tarek Auel] [SPARK-8199] style and timezone fix 256c357 [Tarek Auel] [SPARK-8199] code cleanup 5983dcc [Tarek Auel] [SPARK-8199] whitespace fix 6e0c78f [Tarek Auel] [SPARK-8199] removed setTimeZone in tests, according to cloud-fans comment in #7488 4afc09c [Tarek Auel] [SPARK-8199] concise leap year handling ea6c110 [Tarek Auel] [SPARK-8199] fix after merging master 70238e0 [Tarek Auel] Merge branch 'master' into SPARK-8199 3c6ae2e [Tarek Auel] [SPARK-8199] removed binary search fb98ba0 [Tarek Auel] [SPARK-8199] python docstring fix cdfae27 [Tarek Auel] [SPARK-8199] cleanup & python docstring fix 746b80a [Tarek Auel] [SPARK-8199] build fix 0ad6db8 [Tarek Auel] [SPARK-8199] minor fix 523542d [Tarek Auel] [SPARK-8199] address comments 2259299 [Tarek Auel] [SPARK-8199] day_of_month alias d01b977 [Tarek Auel] [SPARK-8199] python underscore 56c4a92 [Tarek Auel] [SPARK-8199] update python docu e223bc0 [Tarek Auel] [SPARK-8199] refactoring d6aa14e [Tarek Auel] [SPARK-8199] fixed Hive compatibility b382267 [Tarek Auel] [SPARK-8199] fixed bug in day calculation; removed set TimeZone in HiveCompatibilitySuite for test purposes; removed Hive tests for second and minute, because we can cast '2015-03-18' to a timestamp and extract a minute/second from it 1b2e540 [Tarek Auel] [SPARK-8119] style fix 0852655 [Tarek Auel] [SPARK-8119] changed from ExpectsInputTypes to implicit casts ec87c69 [Tarek Auel] [SPARK-8119] bug fixing and refactoring 1358cdc [Tarek Auel] Merge remote-tracking branch 'origin/master' into SPARK-8199 740af0e [Tarek Auel] implement date function using a calculation based on days 4fb66da [Tarek Auel] WIP: date functions on calculation only 1a436c9 [Tarek Auel] wip f775f39 [Tarek Auel] fixed return type ad17e96 [Tarek Auel] improved implementation c42b444 [Tarek Auel] Removed merge conflict file ccb723c [Tarek Auel] [SPARK-8199] style and fixed merge issues 10e4ad1 [Tarek Auel] Merge branch 'master' into date-functions-fast 7d9f0eb [Tarek Auel] [SPARK-8199] git renaming issue f3e7a9f [Tarek Auel] [SPARK-8199] revert change in DataFrameFunctionsSuite 6f5d95c [Tarek Auel] [SPARK-8199] fixed year interval d9f8ac3 [Tarek Auel] [SPARK-8199] implement fast track 7bc9d93 [Tarek Auel] Merge branch 'master' into SPARK-8199 5a105d9 [Tarek Auel] [SPARK-8199] rebase after #6985 got merged eb6760d [Tarek Auel] Merge branch 'master' into SPARK-8199 f120415 [Tarek Auel] improved runtime a8edebd [Tarek Auel] use Calendar instead of SimpleDateFormat 5fe74e1 [Tarek Auel] fixed python style 3bfac90 [Tarek Auel] fixed style 356df78 [Tarek Auel] rely on cast mechanism of Spark. Simplified implementation 02efc5d [Tarek Auel] removed doubled code a5ea120 [Tarek Auel] added python api; changed test to be more meaningful b680db6 [Tarek Auel] added codegeneration to all functions c739788 [Tarek Auel] added support for quarter SPARK-8178 849fb41 [Tarek Auel] fixed stupid test 638596f [Tarek Auel] improved codegen 4d8049b [Tarek Auel] fixed tests and added type check 5ebb235 [Tarek Auel] resolved naming conflict d0e2f99 [Tarek Auel] date functions
2015-07-19 01:48:05 -04:00
"""
Extract the day of the month of a given date as integer.
>>> df = spark.createDataFrame([('2015-04-08',)], ['dt'])
>>> df.select(dayofmonth('dt').alias('day')).collect()
[SPARK-8199][SPARK-8184][SPARK-8183][SPARK-8182][SPARK-8181][SPARK-8180][SPARK-8179][SPARK-8177][SPARK-8178][SPARK-9115][SQL] date functions Jira: https://issues.apache.org/jira/browse/SPARK-8199 https://issues.apache.org/jira/browse/SPARK-8184 https://issues.apache.org/jira/browse/SPARK-8183 https://issues.apache.org/jira/browse/SPARK-8182 https://issues.apache.org/jira/browse/SPARK-8181 https://issues.apache.org/jira/browse/SPARK-8180 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-8177 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-9115 Regarding `day`and `dayofmonth` are both necessary? ~~I am going to add `Quarter` to this PR as well.~~ Done. ~~As soon as the Scala coding is reviewed and discussed, I'll add the python api.~~ Done Author: Tarek Auel <tarek.auel@googlemail.com> Author: Tarek Auel <tarek.auel@gmail.com> Closes #6981 from tarekauel/SPARK-8199 and squashes the following commits: f7b4c8c [Tarek Auel] [SPARK-8199] fixed bug in tests bb567b6 [Tarek Auel] [SPARK-8199] fixed test 3e095ba [Tarek Auel] [SPARK-8199] style and timezone fix 256c357 [Tarek Auel] [SPARK-8199] code cleanup 5983dcc [Tarek Auel] [SPARK-8199] whitespace fix 6e0c78f [Tarek Auel] [SPARK-8199] removed setTimeZone in tests, according to cloud-fans comment in #7488 4afc09c [Tarek Auel] [SPARK-8199] concise leap year handling ea6c110 [Tarek Auel] [SPARK-8199] fix after merging master 70238e0 [Tarek Auel] Merge branch 'master' into SPARK-8199 3c6ae2e [Tarek Auel] [SPARK-8199] removed binary search fb98ba0 [Tarek Auel] [SPARK-8199] python docstring fix cdfae27 [Tarek Auel] [SPARK-8199] cleanup & python docstring fix 746b80a [Tarek Auel] [SPARK-8199] build fix 0ad6db8 [Tarek Auel] [SPARK-8199] minor fix 523542d [Tarek Auel] [SPARK-8199] address comments 2259299 [Tarek Auel] [SPARK-8199] day_of_month alias d01b977 [Tarek Auel] [SPARK-8199] python underscore 56c4a92 [Tarek Auel] [SPARK-8199] update python docu e223bc0 [Tarek Auel] [SPARK-8199] refactoring d6aa14e [Tarek Auel] [SPARK-8199] fixed Hive compatibility b382267 [Tarek Auel] [SPARK-8199] fixed bug in day calculation; removed set TimeZone in HiveCompatibilitySuite for test purposes; removed Hive tests for second and minute, because we can cast '2015-03-18' to a timestamp and extract a minute/second from it 1b2e540 [Tarek Auel] [SPARK-8119] style fix 0852655 [Tarek Auel] [SPARK-8119] changed from ExpectsInputTypes to implicit casts ec87c69 [Tarek Auel] [SPARK-8119] bug fixing and refactoring 1358cdc [Tarek Auel] Merge remote-tracking branch 'origin/master' into SPARK-8199 740af0e [Tarek Auel] implement date function using a calculation based on days 4fb66da [Tarek Auel] WIP: date functions on calculation only 1a436c9 [Tarek Auel] wip f775f39 [Tarek Auel] fixed return type ad17e96 [Tarek Auel] improved implementation c42b444 [Tarek Auel] Removed merge conflict file ccb723c [Tarek Auel] [SPARK-8199] style and fixed merge issues 10e4ad1 [Tarek Auel] Merge branch 'master' into date-functions-fast 7d9f0eb [Tarek Auel] [SPARK-8199] git renaming issue f3e7a9f [Tarek Auel] [SPARK-8199] revert change in DataFrameFunctionsSuite 6f5d95c [Tarek Auel] [SPARK-8199] fixed year interval d9f8ac3 [Tarek Auel] [SPARK-8199] implement fast track 7bc9d93 [Tarek Auel] Merge branch 'master' into SPARK-8199 5a105d9 [Tarek Auel] [SPARK-8199] rebase after #6985 got merged eb6760d [Tarek Auel] Merge branch 'master' into SPARK-8199 f120415 [Tarek Auel] improved runtime a8edebd [Tarek Auel] use Calendar instead of SimpleDateFormat 5fe74e1 [Tarek Auel] fixed python style 3bfac90 [Tarek Auel] fixed style 356df78 [Tarek Auel] rely on cast mechanism of Spark. Simplified implementation 02efc5d [Tarek Auel] removed doubled code a5ea120 [Tarek Auel] added python api; changed test to be more meaningful b680db6 [Tarek Auel] added codegeneration to all functions c739788 [Tarek Auel] added support for quarter SPARK-8178 849fb41 [Tarek Auel] fixed stupid test 638596f [Tarek Auel] improved codegen 4d8049b [Tarek Auel] fixed tests and added type check 5ebb235 [Tarek Auel] resolved naming conflict d0e2f99 [Tarek Auel] date functions
2015-07-19 01:48:05 -04:00
[Row(day=8)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.dayofmonth(_to_java_column(col)))
[SPARK-8199][SPARK-8184][SPARK-8183][SPARK-8182][SPARK-8181][SPARK-8180][SPARK-8179][SPARK-8177][SPARK-8178][SPARK-9115][SQL] date functions Jira: https://issues.apache.org/jira/browse/SPARK-8199 https://issues.apache.org/jira/browse/SPARK-8184 https://issues.apache.org/jira/browse/SPARK-8183 https://issues.apache.org/jira/browse/SPARK-8182 https://issues.apache.org/jira/browse/SPARK-8181 https://issues.apache.org/jira/browse/SPARK-8180 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-8177 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-9115 Regarding `day`and `dayofmonth` are both necessary? ~~I am going to add `Quarter` to this PR as well.~~ Done. ~~As soon as the Scala coding is reviewed and discussed, I'll add the python api.~~ Done Author: Tarek Auel <tarek.auel@googlemail.com> Author: Tarek Auel <tarek.auel@gmail.com> Closes #6981 from tarekauel/SPARK-8199 and squashes the following commits: f7b4c8c [Tarek Auel] [SPARK-8199] fixed bug in tests bb567b6 [Tarek Auel] [SPARK-8199] fixed test 3e095ba [Tarek Auel] [SPARK-8199] style and timezone fix 256c357 [Tarek Auel] [SPARK-8199] code cleanup 5983dcc [Tarek Auel] [SPARK-8199] whitespace fix 6e0c78f [Tarek Auel] [SPARK-8199] removed setTimeZone in tests, according to cloud-fans comment in #7488 4afc09c [Tarek Auel] [SPARK-8199] concise leap year handling ea6c110 [Tarek Auel] [SPARK-8199] fix after merging master 70238e0 [Tarek Auel] Merge branch 'master' into SPARK-8199 3c6ae2e [Tarek Auel] [SPARK-8199] removed binary search fb98ba0 [Tarek Auel] [SPARK-8199] python docstring fix cdfae27 [Tarek Auel] [SPARK-8199] cleanup & python docstring fix 746b80a [Tarek Auel] [SPARK-8199] build fix 0ad6db8 [Tarek Auel] [SPARK-8199] minor fix 523542d [Tarek Auel] [SPARK-8199] address comments 2259299 [Tarek Auel] [SPARK-8199] day_of_month alias d01b977 [Tarek Auel] [SPARK-8199] python underscore 56c4a92 [Tarek Auel] [SPARK-8199] update python docu e223bc0 [Tarek Auel] [SPARK-8199] refactoring d6aa14e [Tarek Auel] [SPARK-8199] fixed Hive compatibility b382267 [Tarek Auel] [SPARK-8199] fixed bug in day calculation; removed set TimeZone in HiveCompatibilitySuite for test purposes; removed Hive tests for second and minute, because we can cast '2015-03-18' to a timestamp and extract a minute/second from it 1b2e540 [Tarek Auel] [SPARK-8119] style fix 0852655 [Tarek Auel] [SPARK-8119] changed from ExpectsInputTypes to implicit casts ec87c69 [Tarek Auel] [SPARK-8119] bug fixing and refactoring 1358cdc [Tarek Auel] Merge remote-tracking branch 'origin/master' into SPARK-8199 740af0e [Tarek Auel] implement date function using a calculation based on days 4fb66da [Tarek Auel] WIP: date functions on calculation only 1a436c9 [Tarek Auel] wip f775f39 [Tarek Auel] fixed return type ad17e96 [Tarek Auel] improved implementation c42b444 [Tarek Auel] Removed merge conflict file ccb723c [Tarek Auel] [SPARK-8199] style and fixed merge issues 10e4ad1 [Tarek Auel] Merge branch 'master' into date-functions-fast 7d9f0eb [Tarek Auel] [SPARK-8199] git renaming issue f3e7a9f [Tarek Auel] [SPARK-8199] revert change in DataFrameFunctionsSuite 6f5d95c [Tarek Auel] [SPARK-8199] fixed year interval d9f8ac3 [Tarek Auel] [SPARK-8199] implement fast track 7bc9d93 [Tarek Auel] Merge branch 'master' into SPARK-8199 5a105d9 [Tarek Auel] [SPARK-8199] rebase after #6985 got merged eb6760d [Tarek Auel] Merge branch 'master' into SPARK-8199 f120415 [Tarek Auel] improved runtime a8edebd [Tarek Auel] use Calendar instead of SimpleDateFormat 5fe74e1 [Tarek Auel] fixed python style 3bfac90 [Tarek Auel] fixed style 356df78 [Tarek Auel] rely on cast mechanism of Spark. Simplified implementation 02efc5d [Tarek Auel] removed doubled code a5ea120 [Tarek Auel] added python api; changed test to be more meaningful b680db6 [Tarek Auel] added codegeneration to all functions c739788 [Tarek Auel] added support for quarter SPARK-8178 849fb41 [Tarek Auel] fixed stupid test 638596f [Tarek Auel] improved codegen 4d8049b [Tarek Auel] fixed tests and added type check 5ebb235 [Tarek Auel] resolved naming conflict d0e2f99 [Tarek Auel] date functions
2015-07-19 01:48:05 -04:00
@since(1.5)
def dayofyear(col):
[SPARK-8199][SPARK-8184][SPARK-8183][SPARK-8182][SPARK-8181][SPARK-8180][SPARK-8179][SPARK-8177][SPARK-8178][SPARK-9115][SQL] date functions Jira: https://issues.apache.org/jira/browse/SPARK-8199 https://issues.apache.org/jira/browse/SPARK-8184 https://issues.apache.org/jira/browse/SPARK-8183 https://issues.apache.org/jira/browse/SPARK-8182 https://issues.apache.org/jira/browse/SPARK-8181 https://issues.apache.org/jira/browse/SPARK-8180 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-8177 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-9115 Regarding `day`and `dayofmonth` are both necessary? ~~I am going to add `Quarter` to this PR as well.~~ Done. ~~As soon as the Scala coding is reviewed and discussed, I'll add the python api.~~ Done Author: Tarek Auel <tarek.auel@googlemail.com> Author: Tarek Auel <tarek.auel@gmail.com> Closes #6981 from tarekauel/SPARK-8199 and squashes the following commits: f7b4c8c [Tarek Auel] [SPARK-8199] fixed bug in tests bb567b6 [Tarek Auel] [SPARK-8199] fixed test 3e095ba [Tarek Auel] [SPARK-8199] style and timezone fix 256c357 [Tarek Auel] [SPARK-8199] code cleanup 5983dcc [Tarek Auel] [SPARK-8199] whitespace fix 6e0c78f [Tarek Auel] [SPARK-8199] removed setTimeZone in tests, according to cloud-fans comment in #7488 4afc09c [Tarek Auel] [SPARK-8199] concise leap year handling ea6c110 [Tarek Auel] [SPARK-8199] fix after merging master 70238e0 [Tarek Auel] Merge branch 'master' into SPARK-8199 3c6ae2e [Tarek Auel] [SPARK-8199] removed binary search fb98ba0 [Tarek Auel] [SPARK-8199] python docstring fix cdfae27 [Tarek Auel] [SPARK-8199] cleanup & python docstring fix 746b80a [Tarek Auel] [SPARK-8199] build fix 0ad6db8 [Tarek Auel] [SPARK-8199] minor fix 523542d [Tarek Auel] [SPARK-8199] address comments 2259299 [Tarek Auel] [SPARK-8199] day_of_month alias d01b977 [Tarek Auel] [SPARK-8199] python underscore 56c4a92 [Tarek Auel] [SPARK-8199] update python docu e223bc0 [Tarek Auel] [SPARK-8199] refactoring d6aa14e [Tarek Auel] [SPARK-8199] fixed Hive compatibility b382267 [Tarek Auel] [SPARK-8199] fixed bug in day calculation; removed set TimeZone in HiveCompatibilitySuite for test purposes; removed Hive tests for second and minute, because we can cast '2015-03-18' to a timestamp and extract a minute/second from it 1b2e540 [Tarek Auel] [SPARK-8119] style fix 0852655 [Tarek Auel] [SPARK-8119] changed from ExpectsInputTypes to implicit casts ec87c69 [Tarek Auel] [SPARK-8119] bug fixing and refactoring 1358cdc [Tarek Auel] Merge remote-tracking branch 'origin/master' into SPARK-8199 740af0e [Tarek Auel] implement date function using a calculation based on days 4fb66da [Tarek Auel] WIP: date functions on calculation only 1a436c9 [Tarek Auel] wip f775f39 [Tarek Auel] fixed return type ad17e96 [Tarek Auel] improved implementation c42b444 [Tarek Auel] Removed merge conflict file ccb723c [Tarek Auel] [SPARK-8199] style and fixed merge issues 10e4ad1 [Tarek Auel] Merge branch 'master' into date-functions-fast 7d9f0eb [Tarek Auel] [SPARK-8199] git renaming issue f3e7a9f [Tarek Auel] [SPARK-8199] revert change in DataFrameFunctionsSuite 6f5d95c [Tarek Auel] [SPARK-8199] fixed year interval d9f8ac3 [Tarek Auel] [SPARK-8199] implement fast track 7bc9d93 [Tarek Auel] Merge branch 'master' into SPARK-8199 5a105d9 [Tarek Auel] [SPARK-8199] rebase after #6985 got merged eb6760d [Tarek Auel] Merge branch 'master' into SPARK-8199 f120415 [Tarek Auel] improved runtime a8edebd [Tarek Auel] use Calendar instead of SimpleDateFormat 5fe74e1 [Tarek Auel] fixed python style 3bfac90 [Tarek Auel] fixed style 356df78 [Tarek Auel] rely on cast mechanism of Spark. Simplified implementation 02efc5d [Tarek Auel] removed doubled code a5ea120 [Tarek Auel] added python api; changed test to be more meaningful b680db6 [Tarek Auel] added codegeneration to all functions c739788 [Tarek Auel] added support for quarter SPARK-8178 849fb41 [Tarek Auel] fixed stupid test 638596f [Tarek Auel] improved codegen 4d8049b [Tarek Auel] fixed tests and added type check 5ebb235 [Tarek Auel] resolved naming conflict d0e2f99 [Tarek Auel] date functions
2015-07-19 01:48:05 -04:00
"""
Extract the day of the year of a given date as integer.
>>> df = spark.createDataFrame([('2015-04-08',)], ['dt'])
>>> df.select(dayofyear('dt').alias('day')).collect()
[SPARK-8199][SPARK-8184][SPARK-8183][SPARK-8182][SPARK-8181][SPARK-8180][SPARK-8179][SPARK-8177][SPARK-8178][SPARK-9115][SQL] date functions Jira: https://issues.apache.org/jira/browse/SPARK-8199 https://issues.apache.org/jira/browse/SPARK-8184 https://issues.apache.org/jira/browse/SPARK-8183 https://issues.apache.org/jira/browse/SPARK-8182 https://issues.apache.org/jira/browse/SPARK-8181 https://issues.apache.org/jira/browse/SPARK-8180 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-8177 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-9115 Regarding `day`and `dayofmonth` are both necessary? ~~I am going to add `Quarter` to this PR as well.~~ Done. ~~As soon as the Scala coding is reviewed and discussed, I'll add the python api.~~ Done Author: Tarek Auel <tarek.auel@googlemail.com> Author: Tarek Auel <tarek.auel@gmail.com> Closes #6981 from tarekauel/SPARK-8199 and squashes the following commits: f7b4c8c [Tarek Auel] [SPARK-8199] fixed bug in tests bb567b6 [Tarek Auel] [SPARK-8199] fixed test 3e095ba [Tarek Auel] [SPARK-8199] style and timezone fix 256c357 [Tarek Auel] [SPARK-8199] code cleanup 5983dcc [Tarek Auel] [SPARK-8199] whitespace fix 6e0c78f [Tarek Auel] [SPARK-8199] removed setTimeZone in tests, according to cloud-fans comment in #7488 4afc09c [Tarek Auel] [SPARK-8199] concise leap year handling ea6c110 [Tarek Auel] [SPARK-8199] fix after merging master 70238e0 [Tarek Auel] Merge branch 'master' into SPARK-8199 3c6ae2e [Tarek Auel] [SPARK-8199] removed binary search fb98ba0 [Tarek Auel] [SPARK-8199] python docstring fix cdfae27 [Tarek Auel] [SPARK-8199] cleanup & python docstring fix 746b80a [Tarek Auel] [SPARK-8199] build fix 0ad6db8 [Tarek Auel] [SPARK-8199] minor fix 523542d [Tarek Auel] [SPARK-8199] address comments 2259299 [Tarek Auel] [SPARK-8199] day_of_month alias d01b977 [Tarek Auel] [SPARK-8199] python underscore 56c4a92 [Tarek Auel] [SPARK-8199] update python docu e223bc0 [Tarek Auel] [SPARK-8199] refactoring d6aa14e [Tarek Auel] [SPARK-8199] fixed Hive compatibility b382267 [Tarek Auel] [SPARK-8199] fixed bug in day calculation; removed set TimeZone in HiveCompatibilitySuite for test purposes; removed Hive tests for second and minute, because we can cast '2015-03-18' to a timestamp and extract a minute/second from it 1b2e540 [Tarek Auel] [SPARK-8119] style fix 0852655 [Tarek Auel] [SPARK-8119] changed from ExpectsInputTypes to implicit casts ec87c69 [Tarek Auel] [SPARK-8119] bug fixing and refactoring 1358cdc [Tarek Auel] Merge remote-tracking branch 'origin/master' into SPARK-8199 740af0e [Tarek Auel] implement date function using a calculation based on days 4fb66da [Tarek Auel] WIP: date functions on calculation only 1a436c9 [Tarek Auel] wip f775f39 [Tarek Auel] fixed return type ad17e96 [Tarek Auel] improved implementation c42b444 [Tarek Auel] Removed merge conflict file ccb723c [Tarek Auel] [SPARK-8199] style and fixed merge issues 10e4ad1 [Tarek Auel] Merge branch 'master' into date-functions-fast 7d9f0eb [Tarek Auel] [SPARK-8199] git renaming issue f3e7a9f [Tarek Auel] [SPARK-8199] revert change in DataFrameFunctionsSuite 6f5d95c [Tarek Auel] [SPARK-8199] fixed year interval d9f8ac3 [Tarek Auel] [SPARK-8199] implement fast track 7bc9d93 [Tarek Auel] Merge branch 'master' into SPARK-8199 5a105d9 [Tarek Auel] [SPARK-8199] rebase after #6985 got merged eb6760d [Tarek Auel] Merge branch 'master' into SPARK-8199 f120415 [Tarek Auel] improved runtime a8edebd [Tarek Auel] use Calendar instead of SimpleDateFormat 5fe74e1 [Tarek Auel] fixed python style 3bfac90 [Tarek Auel] fixed style 356df78 [Tarek Auel] rely on cast mechanism of Spark. Simplified implementation 02efc5d [Tarek Auel] removed doubled code a5ea120 [Tarek Auel] added python api; changed test to be more meaningful b680db6 [Tarek Auel] added codegeneration to all functions c739788 [Tarek Auel] added support for quarter SPARK-8178 849fb41 [Tarek Auel] fixed stupid test 638596f [Tarek Auel] improved codegen 4d8049b [Tarek Auel] fixed tests and added type check 5ebb235 [Tarek Auel] resolved naming conflict d0e2f99 [Tarek Auel] date functions
2015-07-19 01:48:05 -04:00
[Row(day=98)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.dayofyear(_to_java_column(col)))
[SPARK-8199][SPARK-8184][SPARK-8183][SPARK-8182][SPARK-8181][SPARK-8180][SPARK-8179][SPARK-8177][SPARK-8178][SPARK-9115][SQL] date functions Jira: https://issues.apache.org/jira/browse/SPARK-8199 https://issues.apache.org/jira/browse/SPARK-8184 https://issues.apache.org/jira/browse/SPARK-8183 https://issues.apache.org/jira/browse/SPARK-8182 https://issues.apache.org/jira/browse/SPARK-8181 https://issues.apache.org/jira/browse/SPARK-8180 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-8177 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-9115 Regarding `day`and `dayofmonth` are both necessary? ~~I am going to add `Quarter` to this PR as well.~~ Done. ~~As soon as the Scala coding is reviewed and discussed, I'll add the python api.~~ Done Author: Tarek Auel <tarek.auel@googlemail.com> Author: Tarek Auel <tarek.auel@gmail.com> Closes #6981 from tarekauel/SPARK-8199 and squashes the following commits: f7b4c8c [Tarek Auel] [SPARK-8199] fixed bug in tests bb567b6 [Tarek Auel] [SPARK-8199] fixed test 3e095ba [Tarek Auel] [SPARK-8199] style and timezone fix 256c357 [Tarek Auel] [SPARK-8199] code cleanup 5983dcc [Tarek Auel] [SPARK-8199] whitespace fix 6e0c78f [Tarek Auel] [SPARK-8199] removed setTimeZone in tests, according to cloud-fans comment in #7488 4afc09c [Tarek Auel] [SPARK-8199] concise leap year handling ea6c110 [Tarek Auel] [SPARK-8199] fix after merging master 70238e0 [Tarek Auel] Merge branch 'master' into SPARK-8199 3c6ae2e [Tarek Auel] [SPARK-8199] removed binary search fb98ba0 [Tarek Auel] [SPARK-8199] python docstring fix cdfae27 [Tarek Auel] [SPARK-8199] cleanup & python docstring fix 746b80a [Tarek Auel] [SPARK-8199] build fix 0ad6db8 [Tarek Auel] [SPARK-8199] minor fix 523542d [Tarek Auel] [SPARK-8199] address comments 2259299 [Tarek Auel] [SPARK-8199] day_of_month alias d01b977 [Tarek Auel] [SPARK-8199] python underscore 56c4a92 [Tarek Auel] [SPARK-8199] update python docu e223bc0 [Tarek Auel] [SPARK-8199] refactoring d6aa14e [Tarek Auel] [SPARK-8199] fixed Hive compatibility b382267 [Tarek Auel] [SPARK-8199] fixed bug in day calculation; removed set TimeZone in HiveCompatibilitySuite for test purposes; removed Hive tests for second and minute, because we can cast '2015-03-18' to a timestamp and extract a minute/second from it 1b2e540 [Tarek Auel] [SPARK-8119] style fix 0852655 [Tarek Auel] [SPARK-8119] changed from ExpectsInputTypes to implicit casts ec87c69 [Tarek Auel] [SPARK-8119] bug fixing and refactoring 1358cdc [Tarek Auel] Merge remote-tracking branch 'origin/master' into SPARK-8199 740af0e [Tarek Auel] implement date function using a calculation based on days 4fb66da [Tarek Auel] WIP: date functions on calculation only 1a436c9 [Tarek Auel] wip f775f39 [Tarek Auel] fixed return type ad17e96 [Tarek Auel] improved implementation c42b444 [Tarek Auel] Removed merge conflict file ccb723c [Tarek Auel] [SPARK-8199] style and fixed merge issues 10e4ad1 [Tarek Auel] Merge branch 'master' into date-functions-fast 7d9f0eb [Tarek Auel] [SPARK-8199] git renaming issue f3e7a9f [Tarek Auel] [SPARK-8199] revert change in DataFrameFunctionsSuite 6f5d95c [Tarek Auel] [SPARK-8199] fixed year interval d9f8ac3 [Tarek Auel] [SPARK-8199] implement fast track 7bc9d93 [Tarek Auel] Merge branch 'master' into SPARK-8199 5a105d9 [Tarek Auel] [SPARK-8199] rebase after #6985 got merged eb6760d [Tarek Auel] Merge branch 'master' into SPARK-8199 f120415 [Tarek Auel] improved runtime a8edebd [Tarek Auel] use Calendar instead of SimpleDateFormat 5fe74e1 [Tarek Auel] fixed python style 3bfac90 [Tarek Auel] fixed style 356df78 [Tarek Auel] rely on cast mechanism of Spark. Simplified implementation 02efc5d [Tarek Auel] removed doubled code a5ea120 [Tarek Auel] added python api; changed test to be more meaningful b680db6 [Tarek Auel] added codegeneration to all functions c739788 [Tarek Auel] added support for quarter SPARK-8178 849fb41 [Tarek Auel] fixed stupid test 638596f [Tarek Auel] improved codegen 4d8049b [Tarek Auel] fixed tests and added type check 5ebb235 [Tarek Auel] resolved naming conflict d0e2f99 [Tarek Auel] date functions
2015-07-19 01:48:05 -04:00
@since(1.5)
def hour(col):
"""
Extract the hours of a given date as integer.
>>> df = spark.createDataFrame([('2015-04-08 13:08:15',)], ['ts'])
>>> df.select(hour('ts').alias('hour')).collect()
[SPARK-8199][SPARK-8184][SPARK-8183][SPARK-8182][SPARK-8181][SPARK-8180][SPARK-8179][SPARK-8177][SPARK-8178][SPARK-9115][SQL] date functions Jira: https://issues.apache.org/jira/browse/SPARK-8199 https://issues.apache.org/jira/browse/SPARK-8184 https://issues.apache.org/jira/browse/SPARK-8183 https://issues.apache.org/jira/browse/SPARK-8182 https://issues.apache.org/jira/browse/SPARK-8181 https://issues.apache.org/jira/browse/SPARK-8180 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-8177 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-9115 Regarding `day`and `dayofmonth` are both necessary? ~~I am going to add `Quarter` to this PR as well.~~ Done. ~~As soon as the Scala coding is reviewed and discussed, I'll add the python api.~~ Done Author: Tarek Auel <tarek.auel@googlemail.com> Author: Tarek Auel <tarek.auel@gmail.com> Closes #6981 from tarekauel/SPARK-8199 and squashes the following commits: f7b4c8c [Tarek Auel] [SPARK-8199] fixed bug in tests bb567b6 [Tarek Auel] [SPARK-8199] fixed test 3e095ba [Tarek Auel] [SPARK-8199] style and timezone fix 256c357 [Tarek Auel] [SPARK-8199] code cleanup 5983dcc [Tarek Auel] [SPARK-8199] whitespace fix 6e0c78f [Tarek Auel] [SPARK-8199] removed setTimeZone in tests, according to cloud-fans comment in #7488 4afc09c [Tarek Auel] [SPARK-8199] concise leap year handling ea6c110 [Tarek Auel] [SPARK-8199] fix after merging master 70238e0 [Tarek Auel] Merge branch 'master' into SPARK-8199 3c6ae2e [Tarek Auel] [SPARK-8199] removed binary search fb98ba0 [Tarek Auel] [SPARK-8199] python docstring fix cdfae27 [Tarek Auel] [SPARK-8199] cleanup & python docstring fix 746b80a [Tarek Auel] [SPARK-8199] build fix 0ad6db8 [Tarek Auel] [SPARK-8199] minor fix 523542d [Tarek Auel] [SPARK-8199] address comments 2259299 [Tarek Auel] [SPARK-8199] day_of_month alias d01b977 [Tarek Auel] [SPARK-8199] python underscore 56c4a92 [Tarek Auel] [SPARK-8199] update python docu e223bc0 [Tarek Auel] [SPARK-8199] refactoring d6aa14e [Tarek Auel] [SPARK-8199] fixed Hive compatibility b382267 [Tarek Auel] [SPARK-8199] fixed bug in day calculation; removed set TimeZone in HiveCompatibilitySuite for test purposes; removed Hive tests for second and minute, because we can cast '2015-03-18' to a timestamp and extract a minute/second from it 1b2e540 [Tarek Auel] [SPARK-8119] style fix 0852655 [Tarek Auel] [SPARK-8119] changed from ExpectsInputTypes to implicit casts ec87c69 [Tarek Auel] [SPARK-8119] bug fixing and refactoring 1358cdc [Tarek Auel] Merge remote-tracking branch 'origin/master' into SPARK-8199 740af0e [Tarek Auel] implement date function using a calculation based on days 4fb66da [Tarek Auel] WIP: date functions on calculation only 1a436c9 [Tarek Auel] wip f775f39 [Tarek Auel] fixed return type ad17e96 [Tarek Auel] improved implementation c42b444 [Tarek Auel] Removed merge conflict file ccb723c [Tarek Auel] [SPARK-8199] style and fixed merge issues 10e4ad1 [Tarek Auel] Merge branch 'master' into date-functions-fast 7d9f0eb [Tarek Auel] [SPARK-8199] git renaming issue f3e7a9f [Tarek Auel] [SPARK-8199] revert change in DataFrameFunctionsSuite 6f5d95c [Tarek Auel] [SPARK-8199] fixed year interval d9f8ac3 [Tarek Auel] [SPARK-8199] implement fast track 7bc9d93 [Tarek Auel] Merge branch 'master' into SPARK-8199 5a105d9 [Tarek Auel] [SPARK-8199] rebase after #6985 got merged eb6760d [Tarek Auel] Merge branch 'master' into SPARK-8199 f120415 [Tarek Auel] improved runtime a8edebd [Tarek Auel] use Calendar instead of SimpleDateFormat 5fe74e1 [Tarek Auel] fixed python style 3bfac90 [Tarek Auel] fixed style 356df78 [Tarek Auel] rely on cast mechanism of Spark. Simplified implementation 02efc5d [Tarek Auel] removed doubled code a5ea120 [Tarek Auel] added python api; changed test to be more meaningful b680db6 [Tarek Auel] added codegeneration to all functions c739788 [Tarek Auel] added support for quarter SPARK-8178 849fb41 [Tarek Auel] fixed stupid test 638596f [Tarek Auel] improved codegen 4d8049b [Tarek Auel] fixed tests and added type check 5ebb235 [Tarek Auel] resolved naming conflict d0e2f99 [Tarek Auel] date functions
2015-07-19 01:48:05 -04:00
[Row(hour=13)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.hour(_to_java_column(col)))
[SPARK-8199][SPARK-8184][SPARK-8183][SPARK-8182][SPARK-8181][SPARK-8180][SPARK-8179][SPARK-8177][SPARK-8178][SPARK-9115][SQL] date functions Jira: https://issues.apache.org/jira/browse/SPARK-8199 https://issues.apache.org/jira/browse/SPARK-8184 https://issues.apache.org/jira/browse/SPARK-8183 https://issues.apache.org/jira/browse/SPARK-8182 https://issues.apache.org/jira/browse/SPARK-8181 https://issues.apache.org/jira/browse/SPARK-8180 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-8177 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-9115 Regarding `day`and `dayofmonth` are both necessary? ~~I am going to add `Quarter` to this PR as well.~~ Done. ~~As soon as the Scala coding is reviewed and discussed, I'll add the python api.~~ Done Author: Tarek Auel <tarek.auel@googlemail.com> Author: Tarek Auel <tarek.auel@gmail.com> Closes #6981 from tarekauel/SPARK-8199 and squashes the following commits: f7b4c8c [Tarek Auel] [SPARK-8199] fixed bug in tests bb567b6 [Tarek Auel] [SPARK-8199] fixed test 3e095ba [Tarek Auel] [SPARK-8199] style and timezone fix 256c357 [Tarek Auel] [SPARK-8199] code cleanup 5983dcc [Tarek Auel] [SPARK-8199] whitespace fix 6e0c78f [Tarek Auel] [SPARK-8199] removed setTimeZone in tests, according to cloud-fans comment in #7488 4afc09c [Tarek Auel] [SPARK-8199] concise leap year handling ea6c110 [Tarek Auel] [SPARK-8199] fix after merging master 70238e0 [Tarek Auel] Merge branch 'master' into SPARK-8199 3c6ae2e [Tarek Auel] [SPARK-8199] removed binary search fb98ba0 [Tarek Auel] [SPARK-8199] python docstring fix cdfae27 [Tarek Auel] [SPARK-8199] cleanup & python docstring fix 746b80a [Tarek Auel] [SPARK-8199] build fix 0ad6db8 [Tarek Auel] [SPARK-8199] minor fix 523542d [Tarek Auel] [SPARK-8199] address comments 2259299 [Tarek Auel] [SPARK-8199] day_of_month alias d01b977 [Tarek Auel] [SPARK-8199] python underscore 56c4a92 [Tarek Auel] [SPARK-8199] update python docu e223bc0 [Tarek Auel] [SPARK-8199] refactoring d6aa14e [Tarek Auel] [SPARK-8199] fixed Hive compatibility b382267 [Tarek Auel] [SPARK-8199] fixed bug in day calculation; removed set TimeZone in HiveCompatibilitySuite for test purposes; removed Hive tests for second and minute, because we can cast '2015-03-18' to a timestamp and extract a minute/second from it 1b2e540 [Tarek Auel] [SPARK-8119] style fix 0852655 [Tarek Auel] [SPARK-8119] changed from ExpectsInputTypes to implicit casts ec87c69 [Tarek Auel] [SPARK-8119] bug fixing and refactoring 1358cdc [Tarek Auel] Merge remote-tracking branch 'origin/master' into SPARK-8199 740af0e [Tarek Auel] implement date function using a calculation based on days 4fb66da [Tarek Auel] WIP: date functions on calculation only 1a436c9 [Tarek Auel] wip f775f39 [Tarek Auel] fixed return type ad17e96 [Tarek Auel] improved implementation c42b444 [Tarek Auel] Removed merge conflict file ccb723c [Tarek Auel] [SPARK-8199] style and fixed merge issues 10e4ad1 [Tarek Auel] Merge branch 'master' into date-functions-fast 7d9f0eb [Tarek Auel] [SPARK-8199] git renaming issue f3e7a9f [Tarek Auel] [SPARK-8199] revert change in DataFrameFunctionsSuite 6f5d95c [Tarek Auel] [SPARK-8199] fixed year interval d9f8ac3 [Tarek Auel] [SPARK-8199] implement fast track 7bc9d93 [Tarek Auel] Merge branch 'master' into SPARK-8199 5a105d9 [Tarek Auel] [SPARK-8199] rebase after #6985 got merged eb6760d [Tarek Auel] Merge branch 'master' into SPARK-8199 f120415 [Tarek Auel] improved runtime a8edebd [Tarek Auel] use Calendar instead of SimpleDateFormat 5fe74e1 [Tarek Auel] fixed python style 3bfac90 [Tarek Auel] fixed style 356df78 [Tarek Auel] rely on cast mechanism of Spark. Simplified implementation 02efc5d [Tarek Auel] removed doubled code a5ea120 [Tarek Auel] added python api; changed test to be more meaningful b680db6 [Tarek Auel] added codegeneration to all functions c739788 [Tarek Auel] added support for quarter SPARK-8178 849fb41 [Tarek Auel] fixed stupid test 638596f [Tarek Auel] improved codegen 4d8049b [Tarek Auel] fixed tests and added type check 5ebb235 [Tarek Auel] resolved naming conflict d0e2f99 [Tarek Auel] date functions
2015-07-19 01:48:05 -04:00
@since(1.5)
def minute(col):
"""
Extract the minutes of a given date as integer.
>>> df = spark.createDataFrame([('2015-04-08 13:08:15',)], ['ts'])
>>> df.select(minute('ts').alias('minute')).collect()
[SPARK-8199][SPARK-8184][SPARK-8183][SPARK-8182][SPARK-8181][SPARK-8180][SPARK-8179][SPARK-8177][SPARK-8178][SPARK-9115][SQL] date functions Jira: https://issues.apache.org/jira/browse/SPARK-8199 https://issues.apache.org/jira/browse/SPARK-8184 https://issues.apache.org/jira/browse/SPARK-8183 https://issues.apache.org/jira/browse/SPARK-8182 https://issues.apache.org/jira/browse/SPARK-8181 https://issues.apache.org/jira/browse/SPARK-8180 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-8177 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-9115 Regarding `day`and `dayofmonth` are both necessary? ~~I am going to add `Quarter` to this PR as well.~~ Done. ~~As soon as the Scala coding is reviewed and discussed, I'll add the python api.~~ Done Author: Tarek Auel <tarek.auel@googlemail.com> Author: Tarek Auel <tarek.auel@gmail.com> Closes #6981 from tarekauel/SPARK-8199 and squashes the following commits: f7b4c8c [Tarek Auel] [SPARK-8199] fixed bug in tests bb567b6 [Tarek Auel] [SPARK-8199] fixed test 3e095ba [Tarek Auel] [SPARK-8199] style and timezone fix 256c357 [Tarek Auel] [SPARK-8199] code cleanup 5983dcc [Tarek Auel] [SPARK-8199] whitespace fix 6e0c78f [Tarek Auel] [SPARK-8199] removed setTimeZone in tests, according to cloud-fans comment in #7488 4afc09c [Tarek Auel] [SPARK-8199] concise leap year handling ea6c110 [Tarek Auel] [SPARK-8199] fix after merging master 70238e0 [Tarek Auel] Merge branch 'master' into SPARK-8199 3c6ae2e [Tarek Auel] [SPARK-8199] removed binary search fb98ba0 [Tarek Auel] [SPARK-8199] python docstring fix cdfae27 [Tarek Auel] [SPARK-8199] cleanup & python docstring fix 746b80a [Tarek Auel] [SPARK-8199] build fix 0ad6db8 [Tarek Auel] [SPARK-8199] minor fix 523542d [Tarek Auel] [SPARK-8199] address comments 2259299 [Tarek Auel] [SPARK-8199] day_of_month alias d01b977 [Tarek Auel] [SPARK-8199] python underscore 56c4a92 [Tarek Auel] [SPARK-8199] update python docu e223bc0 [Tarek Auel] [SPARK-8199] refactoring d6aa14e [Tarek Auel] [SPARK-8199] fixed Hive compatibility b382267 [Tarek Auel] [SPARK-8199] fixed bug in day calculation; removed set TimeZone in HiveCompatibilitySuite for test purposes; removed Hive tests for second and minute, because we can cast '2015-03-18' to a timestamp and extract a minute/second from it 1b2e540 [Tarek Auel] [SPARK-8119] style fix 0852655 [Tarek Auel] [SPARK-8119] changed from ExpectsInputTypes to implicit casts ec87c69 [Tarek Auel] [SPARK-8119] bug fixing and refactoring 1358cdc [Tarek Auel] Merge remote-tracking branch 'origin/master' into SPARK-8199 740af0e [Tarek Auel] implement date function using a calculation based on days 4fb66da [Tarek Auel] WIP: date functions on calculation only 1a436c9 [Tarek Auel] wip f775f39 [Tarek Auel] fixed return type ad17e96 [Tarek Auel] improved implementation c42b444 [Tarek Auel] Removed merge conflict file ccb723c [Tarek Auel] [SPARK-8199] style and fixed merge issues 10e4ad1 [Tarek Auel] Merge branch 'master' into date-functions-fast 7d9f0eb [Tarek Auel] [SPARK-8199] git renaming issue f3e7a9f [Tarek Auel] [SPARK-8199] revert change in DataFrameFunctionsSuite 6f5d95c [Tarek Auel] [SPARK-8199] fixed year interval d9f8ac3 [Tarek Auel] [SPARK-8199] implement fast track 7bc9d93 [Tarek Auel] Merge branch 'master' into SPARK-8199 5a105d9 [Tarek Auel] [SPARK-8199] rebase after #6985 got merged eb6760d [Tarek Auel] Merge branch 'master' into SPARK-8199 f120415 [Tarek Auel] improved runtime a8edebd [Tarek Auel] use Calendar instead of SimpleDateFormat 5fe74e1 [Tarek Auel] fixed python style 3bfac90 [Tarek Auel] fixed style 356df78 [Tarek Auel] rely on cast mechanism of Spark. Simplified implementation 02efc5d [Tarek Auel] removed doubled code a5ea120 [Tarek Auel] added python api; changed test to be more meaningful b680db6 [Tarek Auel] added codegeneration to all functions c739788 [Tarek Auel] added support for quarter SPARK-8178 849fb41 [Tarek Auel] fixed stupid test 638596f [Tarek Auel] improved codegen 4d8049b [Tarek Auel] fixed tests and added type check 5ebb235 [Tarek Auel] resolved naming conflict d0e2f99 [Tarek Auel] date functions
2015-07-19 01:48:05 -04:00
[Row(minute=8)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.minute(_to_java_column(col)))
[SPARK-8199][SPARK-8184][SPARK-8183][SPARK-8182][SPARK-8181][SPARK-8180][SPARK-8179][SPARK-8177][SPARK-8178][SPARK-9115][SQL] date functions Jira: https://issues.apache.org/jira/browse/SPARK-8199 https://issues.apache.org/jira/browse/SPARK-8184 https://issues.apache.org/jira/browse/SPARK-8183 https://issues.apache.org/jira/browse/SPARK-8182 https://issues.apache.org/jira/browse/SPARK-8181 https://issues.apache.org/jira/browse/SPARK-8180 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-8177 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-9115 Regarding `day`and `dayofmonth` are both necessary? ~~I am going to add `Quarter` to this PR as well.~~ Done. ~~As soon as the Scala coding is reviewed and discussed, I'll add the python api.~~ Done Author: Tarek Auel <tarek.auel@googlemail.com> Author: Tarek Auel <tarek.auel@gmail.com> Closes #6981 from tarekauel/SPARK-8199 and squashes the following commits: f7b4c8c [Tarek Auel] [SPARK-8199] fixed bug in tests bb567b6 [Tarek Auel] [SPARK-8199] fixed test 3e095ba [Tarek Auel] [SPARK-8199] style and timezone fix 256c357 [Tarek Auel] [SPARK-8199] code cleanup 5983dcc [Tarek Auel] [SPARK-8199] whitespace fix 6e0c78f [Tarek Auel] [SPARK-8199] removed setTimeZone in tests, according to cloud-fans comment in #7488 4afc09c [Tarek Auel] [SPARK-8199] concise leap year handling ea6c110 [Tarek Auel] [SPARK-8199] fix after merging master 70238e0 [Tarek Auel] Merge branch 'master' into SPARK-8199 3c6ae2e [Tarek Auel] [SPARK-8199] removed binary search fb98ba0 [Tarek Auel] [SPARK-8199] python docstring fix cdfae27 [Tarek Auel] [SPARK-8199] cleanup & python docstring fix 746b80a [Tarek Auel] [SPARK-8199] build fix 0ad6db8 [Tarek Auel] [SPARK-8199] minor fix 523542d [Tarek Auel] [SPARK-8199] address comments 2259299 [Tarek Auel] [SPARK-8199] day_of_month alias d01b977 [Tarek Auel] [SPARK-8199] python underscore 56c4a92 [Tarek Auel] [SPARK-8199] update python docu e223bc0 [Tarek Auel] [SPARK-8199] refactoring d6aa14e [Tarek Auel] [SPARK-8199] fixed Hive compatibility b382267 [Tarek Auel] [SPARK-8199] fixed bug in day calculation; removed set TimeZone in HiveCompatibilitySuite for test purposes; removed Hive tests for second and minute, because we can cast '2015-03-18' to a timestamp and extract a minute/second from it 1b2e540 [Tarek Auel] [SPARK-8119] style fix 0852655 [Tarek Auel] [SPARK-8119] changed from ExpectsInputTypes to implicit casts ec87c69 [Tarek Auel] [SPARK-8119] bug fixing and refactoring 1358cdc [Tarek Auel] Merge remote-tracking branch 'origin/master' into SPARK-8199 740af0e [Tarek Auel] implement date function using a calculation based on days 4fb66da [Tarek Auel] WIP: date functions on calculation only 1a436c9 [Tarek Auel] wip f775f39 [Tarek Auel] fixed return type ad17e96 [Tarek Auel] improved implementation c42b444 [Tarek Auel] Removed merge conflict file ccb723c [Tarek Auel] [SPARK-8199] style and fixed merge issues 10e4ad1 [Tarek Auel] Merge branch 'master' into date-functions-fast 7d9f0eb [Tarek Auel] [SPARK-8199] git renaming issue f3e7a9f [Tarek Auel] [SPARK-8199] revert change in DataFrameFunctionsSuite 6f5d95c [Tarek Auel] [SPARK-8199] fixed year interval d9f8ac3 [Tarek Auel] [SPARK-8199] implement fast track 7bc9d93 [Tarek Auel] Merge branch 'master' into SPARK-8199 5a105d9 [Tarek Auel] [SPARK-8199] rebase after #6985 got merged eb6760d [Tarek Auel] Merge branch 'master' into SPARK-8199 f120415 [Tarek Auel] improved runtime a8edebd [Tarek Auel] use Calendar instead of SimpleDateFormat 5fe74e1 [Tarek Auel] fixed python style 3bfac90 [Tarek Auel] fixed style 356df78 [Tarek Auel] rely on cast mechanism of Spark. Simplified implementation 02efc5d [Tarek Auel] removed doubled code a5ea120 [Tarek Auel] added python api; changed test to be more meaningful b680db6 [Tarek Auel] added codegeneration to all functions c739788 [Tarek Auel] added support for quarter SPARK-8178 849fb41 [Tarek Auel] fixed stupid test 638596f [Tarek Auel] improved codegen 4d8049b [Tarek Auel] fixed tests and added type check 5ebb235 [Tarek Auel] resolved naming conflict d0e2f99 [Tarek Auel] date functions
2015-07-19 01:48:05 -04:00
@since(1.5)
def second(col):
"""
Extract the seconds of a given date as integer.
>>> df = spark.createDataFrame([('2015-04-08 13:08:15',)], ['ts'])
>>> df.select(second('ts').alias('second')).collect()
[SPARK-8199][SPARK-8184][SPARK-8183][SPARK-8182][SPARK-8181][SPARK-8180][SPARK-8179][SPARK-8177][SPARK-8178][SPARK-9115][SQL] date functions Jira: https://issues.apache.org/jira/browse/SPARK-8199 https://issues.apache.org/jira/browse/SPARK-8184 https://issues.apache.org/jira/browse/SPARK-8183 https://issues.apache.org/jira/browse/SPARK-8182 https://issues.apache.org/jira/browse/SPARK-8181 https://issues.apache.org/jira/browse/SPARK-8180 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-8177 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-9115 Regarding `day`and `dayofmonth` are both necessary? ~~I am going to add `Quarter` to this PR as well.~~ Done. ~~As soon as the Scala coding is reviewed and discussed, I'll add the python api.~~ Done Author: Tarek Auel <tarek.auel@googlemail.com> Author: Tarek Auel <tarek.auel@gmail.com> Closes #6981 from tarekauel/SPARK-8199 and squashes the following commits: f7b4c8c [Tarek Auel] [SPARK-8199] fixed bug in tests bb567b6 [Tarek Auel] [SPARK-8199] fixed test 3e095ba [Tarek Auel] [SPARK-8199] style and timezone fix 256c357 [Tarek Auel] [SPARK-8199] code cleanup 5983dcc [Tarek Auel] [SPARK-8199] whitespace fix 6e0c78f [Tarek Auel] [SPARK-8199] removed setTimeZone in tests, according to cloud-fans comment in #7488 4afc09c [Tarek Auel] [SPARK-8199] concise leap year handling ea6c110 [Tarek Auel] [SPARK-8199] fix after merging master 70238e0 [Tarek Auel] Merge branch 'master' into SPARK-8199 3c6ae2e [Tarek Auel] [SPARK-8199] removed binary search fb98ba0 [Tarek Auel] [SPARK-8199] python docstring fix cdfae27 [Tarek Auel] [SPARK-8199] cleanup & python docstring fix 746b80a [Tarek Auel] [SPARK-8199] build fix 0ad6db8 [Tarek Auel] [SPARK-8199] minor fix 523542d [Tarek Auel] [SPARK-8199] address comments 2259299 [Tarek Auel] [SPARK-8199] day_of_month alias d01b977 [Tarek Auel] [SPARK-8199] python underscore 56c4a92 [Tarek Auel] [SPARK-8199] update python docu e223bc0 [Tarek Auel] [SPARK-8199] refactoring d6aa14e [Tarek Auel] [SPARK-8199] fixed Hive compatibility b382267 [Tarek Auel] [SPARK-8199] fixed bug in day calculation; removed set TimeZone in HiveCompatibilitySuite for test purposes; removed Hive tests for second and minute, because we can cast '2015-03-18' to a timestamp and extract a minute/second from it 1b2e540 [Tarek Auel] [SPARK-8119] style fix 0852655 [Tarek Auel] [SPARK-8119] changed from ExpectsInputTypes to implicit casts ec87c69 [Tarek Auel] [SPARK-8119] bug fixing and refactoring 1358cdc [Tarek Auel] Merge remote-tracking branch 'origin/master' into SPARK-8199 740af0e [Tarek Auel] implement date function using a calculation based on days 4fb66da [Tarek Auel] WIP: date functions on calculation only 1a436c9 [Tarek Auel] wip f775f39 [Tarek Auel] fixed return type ad17e96 [Tarek Auel] improved implementation c42b444 [Tarek Auel] Removed merge conflict file ccb723c [Tarek Auel] [SPARK-8199] style and fixed merge issues 10e4ad1 [Tarek Auel] Merge branch 'master' into date-functions-fast 7d9f0eb [Tarek Auel] [SPARK-8199] git renaming issue f3e7a9f [Tarek Auel] [SPARK-8199] revert change in DataFrameFunctionsSuite 6f5d95c [Tarek Auel] [SPARK-8199] fixed year interval d9f8ac3 [Tarek Auel] [SPARK-8199] implement fast track 7bc9d93 [Tarek Auel] Merge branch 'master' into SPARK-8199 5a105d9 [Tarek Auel] [SPARK-8199] rebase after #6985 got merged eb6760d [Tarek Auel] Merge branch 'master' into SPARK-8199 f120415 [Tarek Auel] improved runtime a8edebd [Tarek Auel] use Calendar instead of SimpleDateFormat 5fe74e1 [Tarek Auel] fixed python style 3bfac90 [Tarek Auel] fixed style 356df78 [Tarek Auel] rely on cast mechanism of Spark. Simplified implementation 02efc5d [Tarek Auel] removed doubled code a5ea120 [Tarek Auel] added python api; changed test to be more meaningful b680db6 [Tarek Auel] added codegeneration to all functions c739788 [Tarek Auel] added support for quarter SPARK-8178 849fb41 [Tarek Auel] fixed stupid test 638596f [Tarek Auel] improved codegen 4d8049b [Tarek Auel] fixed tests and added type check 5ebb235 [Tarek Auel] resolved naming conflict d0e2f99 [Tarek Auel] date functions
2015-07-19 01:48:05 -04:00
[Row(second=15)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.second(_to_java_column(col)))
[SPARK-8199][SPARK-8184][SPARK-8183][SPARK-8182][SPARK-8181][SPARK-8180][SPARK-8179][SPARK-8177][SPARK-8178][SPARK-9115][SQL] date functions Jira: https://issues.apache.org/jira/browse/SPARK-8199 https://issues.apache.org/jira/browse/SPARK-8184 https://issues.apache.org/jira/browse/SPARK-8183 https://issues.apache.org/jira/browse/SPARK-8182 https://issues.apache.org/jira/browse/SPARK-8181 https://issues.apache.org/jira/browse/SPARK-8180 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-8177 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-9115 Regarding `day`and `dayofmonth` are both necessary? ~~I am going to add `Quarter` to this PR as well.~~ Done. ~~As soon as the Scala coding is reviewed and discussed, I'll add the python api.~~ Done Author: Tarek Auel <tarek.auel@googlemail.com> Author: Tarek Auel <tarek.auel@gmail.com> Closes #6981 from tarekauel/SPARK-8199 and squashes the following commits: f7b4c8c [Tarek Auel] [SPARK-8199] fixed bug in tests bb567b6 [Tarek Auel] [SPARK-8199] fixed test 3e095ba [Tarek Auel] [SPARK-8199] style and timezone fix 256c357 [Tarek Auel] [SPARK-8199] code cleanup 5983dcc [Tarek Auel] [SPARK-8199] whitespace fix 6e0c78f [Tarek Auel] [SPARK-8199] removed setTimeZone in tests, according to cloud-fans comment in #7488 4afc09c [Tarek Auel] [SPARK-8199] concise leap year handling ea6c110 [Tarek Auel] [SPARK-8199] fix after merging master 70238e0 [Tarek Auel] Merge branch 'master' into SPARK-8199 3c6ae2e [Tarek Auel] [SPARK-8199] removed binary search fb98ba0 [Tarek Auel] [SPARK-8199] python docstring fix cdfae27 [Tarek Auel] [SPARK-8199] cleanup & python docstring fix 746b80a [Tarek Auel] [SPARK-8199] build fix 0ad6db8 [Tarek Auel] [SPARK-8199] minor fix 523542d [Tarek Auel] [SPARK-8199] address comments 2259299 [Tarek Auel] [SPARK-8199] day_of_month alias d01b977 [Tarek Auel] [SPARK-8199] python underscore 56c4a92 [Tarek Auel] [SPARK-8199] update python docu e223bc0 [Tarek Auel] [SPARK-8199] refactoring d6aa14e [Tarek Auel] [SPARK-8199] fixed Hive compatibility b382267 [Tarek Auel] [SPARK-8199] fixed bug in day calculation; removed set TimeZone in HiveCompatibilitySuite for test purposes; removed Hive tests for second and minute, because we can cast '2015-03-18' to a timestamp and extract a minute/second from it 1b2e540 [Tarek Auel] [SPARK-8119] style fix 0852655 [Tarek Auel] [SPARK-8119] changed from ExpectsInputTypes to implicit casts ec87c69 [Tarek Auel] [SPARK-8119] bug fixing and refactoring 1358cdc [Tarek Auel] Merge remote-tracking branch 'origin/master' into SPARK-8199 740af0e [Tarek Auel] implement date function using a calculation based on days 4fb66da [Tarek Auel] WIP: date functions on calculation only 1a436c9 [Tarek Auel] wip f775f39 [Tarek Auel] fixed return type ad17e96 [Tarek Auel] improved implementation c42b444 [Tarek Auel] Removed merge conflict file ccb723c [Tarek Auel] [SPARK-8199] style and fixed merge issues 10e4ad1 [Tarek Auel] Merge branch 'master' into date-functions-fast 7d9f0eb [Tarek Auel] [SPARK-8199] git renaming issue f3e7a9f [Tarek Auel] [SPARK-8199] revert change in DataFrameFunctionsSuite 6f5d95c [Tarek Auel] [SPARK-8199] fixed year interval d9f8ac3 [Tarek Auel] [SPARK-8199] implement fast track 7bc9d93 [Tarek Auel] Merge branch 'master' into SPARK-8199 5a105d9 [Tarek Auel] [SPARK-8199] rebase after #6985 got merged eb6760d [Tarek Auel] Merge branch 'master' into SPARK-8199 f120415 [Tarek Auel] improved runtime a8edebd [Tarek Auel] use Calendar instead of SimpleDateFormat 5fe74e1 [Tarek Auel] fixed python style 3bfac90 [Tarek Auel] fixed style 356df78 [Tarek Auel] rely on cast mechanism of Spark. Simplified implementation 02efc5d [Tarek Auel] removed doubled code a5ea120 [Tarek Auel] added python api; changed test to be more meaningful b680db6 [Tarek Auel] added codegeneration to all functions c739788 [Tarek Auel] added support for quarter SPARK-8178 849fb41 [Tarek Auel] fixed stupid test 638596f [Tarek Auel] improved codegen 4d8049b [Tarek Auel] fixed tests and added type check 5ebb235 [Tarek Auel] resolved naming conflict d0e2f99 [Tarek Auel] date functions
2015-07-19 01:48:05 -04:00
@since(1.5)
def weekofyear(col):
[SPARK-8199][SPARK-8184][SPARK-8183][SPARK-8182][SPARK-8181][SPARK-8180][SPARK-8179][SPARK-8177][SPARK-8178][SPARK-9115][SQL] date functions Jira: https://issues.apache.org/jira/browse/SPARK-8199 https://issues.apache.org/jira/browse/SPARK-8184 https://issues.apache.org/jira/browse/SPARK-8183 https://issues.apache.org/jira/browse/SPARK-8182 https://issues.apache.org/jira/browse/SPARK-8181 https://issues.apache.org/jira/browse/SPARK-8180 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-8177 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-9115 Regarding `day`and `dayofmonth` are both necessary? ~~I am going to add `Quarter` to this PR as well.~~ Done. ~~As soon as the Scala coding is reviewed and discussed, I'll add the python api.~~ Done Author: Tarek Auel <tarek.auel@googlemail.com> Author: Tarek Auel <tarek.auel@gmail.com> Closes #6981 from tarekauel/SPARK-8199 and squashes the following commits: f7b4c8c [Tarek Auel] [SPARK-8199] fixed bug in tests bb567b6 [Tarek Auel] [SPARK-8199] fixed test 3e095ba [Tarek Auel] [SPARK-8199] style and timezone fix 256c357 [Tarek Auel] [SPARK-8199] code cleanup 5983dcc [Tarek Auel] [SPARK-8199] whitespace fix 6e0c78f [Tarek Auel] [SPARK-8199] removed setTimeZone in tests, according to cloud-fans comment in #7488 4afc09c [Tarek Auel] [SPARK-8199] concise leap year handling ea6c110 [Tarek Auel] [SPARK-8199] fix after merging master 70238e0 [Tarek Auel] Merge branch 'master' into SPARK-8199 3c6ae2e [Tarek Auel] [SPARK-8199] removed binary search fb98ba0 [Tarek Auel] [SPARK-8199] python docstring fix cdfae27 [Tarek Auel] [SPARK-8199] cleanup & python docstring fix 746b80a [Tarek Auel] [SPARK-8199] build fix 0ad6db8 [Tarek Auel] [SPARK-8199] minor fix 523542d [Tarek Auel] [SPARK-8199] address comments 2259299 [Tarek Auel] [SPARK-8199] day_of_month alias d01b977 [Tarek Auel] [SPARK-8199] python underscore 56c4a92 [Tarek Auel] [SPARK-8199] update python docu e223bc0 [Tarek Auel] [SPARK-8199] refactoring d6aa14e [Tarek Auel] [SPARK-8199] fixed Hive compatibility b382267 [Tarek Auel] [SPARK-8199] fixed bug in day calculation; removed set TimeZone in HiveCompatibilitySuite for test purposes; removed Hive tests for second and minute, because we can cast '2015-03-18' to a timestamp and extract a minute/second from it 1b2e540 [Tarek Auel] [SPARK-8119] style fix 0852655 [Tarek Auel] [SPARK-8119] changed from ExpectsInputTypes to implicit casts ec87c69 [Tarek Auel] [SPARK-8119] bug fixing and refactoring 1358cdc [Tarek Auel] Merge remote-tracking branch 'origin/master' into SPARK-8199 740af0e [Tarek Auel] implement date function using a calculation based on days 4fb66da [Tarek Auel] WIP: date functions on calculation only 1a436c9 [Tarek Auel] wip f775f39 [Tarek Auel] fixed return type ad17e96 [Tarek Auel] improved implementation c42b444 [Tarek Auel] Removed merge conflict file ccb723c [Tarek Auel] [SPARK-8199] style and fixed merge issues 10e4ad1 [Tarek Auel] Merge branch 'master' into date-functions-fast 7d9f0eb [Tarek Auel] [SPARK-8199] git renaming issue f3e7a9f [Tarek Auel] [SPARK-8199] revert change in DataFrameFunctionsSuite 6f5d95c [Tarek Auel] [SPARK-8199] fixed year interval d9f8ac3 [Tarek Auel] [SPARK-8199] implement fast track 7bc9d93 [Tarek Auel] Merge branch 'master' into SPARK-8199 5a105d9 [Tarek Auel] [SPARK-8199] rebase after #6985 got merged eb6760d [Tarek Auel] Merge branch 'master' into SPARK-8199 f120415 [Tarek Auel] improved runtime a8edebd [Tarek Auel] use Calendar instead of SimpleDateFormat 5fe74e1 [Tarek Auel] fixed python style 3bfac90 [Tarek Auel] fixed style 356df78 [Tarek Auel] rely on cast mechanism of Spark. Simplified implementation 02efc5d [Tarek Auel] removed doubled code a5ea120 [Tarek Auel] added python api; changed test to be more meaningful b680db6 [Tarek Auel] added codegeneration to all functions c739788 [Tarek Auel] added support for quarter SPARK-8178 849fb41 [Tarek Auel] fixed stupid test 638596f [Tarek Auel] improved codegen 4d8049b [Tarek Auel] fixed tests and added type check 5ebb235 [Tarek Auel] resolved naming conflict d0e2f99 [Tarek Auel] date functions
2015-07-19 01:48:05 -04:00
"""
Extract the week number of a given date as integer.
>>> df = spark.createDataFrame([('2015-04-08',)], ['dt'])
>>> df.select(weekofyear(df.dt).alias('week')).collect()
[SPARK-8199][SPARK-8184][SPARK-8183][SPARK-8182][SPARK-8181][SPARK-8180][SPARK-8179][SPARK-8177][SPARK-8178][SPARK-9115][SQL] date functions Jira: https://issues.apache.org/jira/browse/SPARK-8199 https://issues.apache.org/jira/browse/SPARK-8184 https://issues.apache.org/jira/browse/SPARK-8183 https://issues.apache.org/jira/browse/SPARK-8182 https://issues.apache.org/jira/browse/SPARK-8181 https://issues.apache.org/jira/browse/SPARK-8180 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-8177 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-9115 Regarding `day`and `dayofmonth` are both necessary? ~~I am going to add `Quarter` to this PR as well.~~ Done. ~~As soon as the Scala coding is reviewed and discussed, I'll add the python api.~~ Done Author: Tarek Auel <tarek.auel@googlemail.com> Author: Tarek Auel <tarek.auel@gmail.com> Closes #6981 from tarekauel/SPARK-8199 and squashes the following commits: f7b4c8c [Tarek Auel] [SPARK-8199] fixed bug in tests bb567b6 [Tarek Auel] [SPARK-8199] fixed test 3e095ba [Tarek Auel] [SPARK-8199] style and timezone fix 256c357 [Tarek Auel] [SPARK-8199] code cleanup 5983dcc [Tarek Auel] [SPARK-8199] whitespace fix 6e0c78f [Tarek Auel] [SPARK-8199] removed setTimeZone in tests, according to cloud-fans comment in #7488 4afc09c [Tarek Auel] [SPARK-8199] concise leap year handling ea6c110 [Tarek Auel] [SPARK-8199] fix after merging master 70238e0 [Tarek Auel] Merge branch 'master' into SPARK-8199 3c6ae2e [Tarek Auel] [SPARK-8199] removed binary search fb98ba0 [Tarek Auel] [SPARK-8199] python docstring fix cdfae27 [Tarek Auel] [SPARK-8199] cleanup & python docstring fix 746b80a [Tarek Auel] [SPARK-8199] build fix 0ad6db8 [Tarek Auel] [SPARK-8199] minor fix 523542d [Tarek Auel] [SPARK-8199] address comments 2259299 [Tarek Auel] [SPARK-8199] day_of_month alias d01b977 [Tarek Auel] [SPARK-8199] python underscore 56c4a92 [Tarek Auel] [SPARK-8199] update python docu e223bc0 [Tarek Auel] [SPARK-8199] refactoring d6aa14e [Tarek Auel] [SPARK-8199] fixed Hive compatibility b382267 [Tarek Auel] [SPARK-8199] fixed bug in day calculation; removed set TimeZone in HiveCompatibilitySuite for test purposes; removed Hive tests for second and minute, because we can cast '2015-03-18' to a timestamp and extract a minute/second from it 1b2e540 [Tarek Auel] [SPARK-8119] style fix 0852655 [Tarek Auel] [SPARK-8119] changed from ExpectsInputTypes to implicit casts ec87c69 [Tarek Auel] [SPARK-8119] bug fixing and refactoring 1358cdc [Tarek Auel] Merge remote-tracking branch 'origin/master' into SPARK-8199 740af0e [Tarek Auel] implement date function using a calculation based on days 4fb66da [Tarek Auel] WIP: date functions on calculation only 1a436c9 [Tarek Auel] wip f775f39 [Tarek Auel] fixed return type ad17e96 [Tarek Auel] improved implementation c42b444 [Tarek Auel] Removed merge conflict file ccb723c [Tarek Auel] [SPARK-8199] style and fixed merge issues 10e4ad1 [Tarek Auel] Merge branch 'master' into date-functions-fast 7d9f0eb [Tarek Auel] [SPARK-8199] git renaming issue f3e7a9f [Tarek Auel] [SPARK-8199] revert change in DataFrameFunctionsSuite 6f5d95c [Tarek Auel] [SPARK-8199] fixed year interval d9f8ac3 [Tarek Auel] [SPARK-8199] implement fast track 7bc9d93 [Tarek Auel] Merge branch 'master' into SPARK-8199 5a105d9 [Tarek Auel] [SPARK-8199] rebase after #6985 got merged eb6760d [Tarek Auel] Merge branch 'master' into SPARK-8199 f120415 [Tarek Auel] improved runtime a8edebd [Tarek Auel] use Calendar instead of SimpleDateFormat 5fe74e1 [Tarek Auel] fixed python style 3bfac90 [Tarek Auel] fixed style 356df78 [Tarek Auel] rely on cast mechanism of Spark. Simplified implementation 02efc5d [Tarek Auel] removed doubled code a5ea120 [Tarek Auel] added python api; changed test to be more meaningful b680db6 [Tarek Auel] added codegeneration to all functions c739788 [Tarek Auel] added support for quarter SPARK-8178 849fb41 [Tarek Auel] fixed stupid test 638596f [Tarek Auel] improved codegen 4d8049b [Tarek Auel] fixed tests and added type check 5ebb235 [Tarek Auel] resolved naming conflict d0e2f99 [Tarek Auel] date functions
2015-07-19 01:48:05 -04:00
[Row(week=15)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.weekofyear(_to_java_column(col)))
@since(1.5)
def date_add(start, days):
"""
Returns the date that is `days` days after `start`
>>> df = spark.createDataFrame([('2015-04-08',)], ['dt'])
>>> df.select(date_add(df.dt, 1).alias('next_date')).collect()
[Row(next_date=datetime.date(2015, 4, 9))]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.date_add(_to_java_column(start), days))
@since(1.5)
def date_sub(start, days):
"""
Returns the date that is `days` days before `start`
>>> df = spark.createDataFrame([('2015-04-08',)], ['dt'])
>>> df.select(date_sub(df.dt, 1).alias('prev_date')).collect()
[Row(prev_date=datetime.date(2015, 4, 7))]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.date_sub(_to_java_column(start), days))
@since(1.5)
def datediff(end, start):
"""
Returns the number of days from `start` to `end`.
>>> df = spark.createDataFrame([('2015-04-08','2015-05-10')], ['d1', 'd2'])
>>> df.select(datediff(df.d2, df.d1).alias('diff')).collect()
[Row(diff=32)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.datediff(_to_java_column(end), _to_java_column(start)))
@since(1.5)
def add_months(start, months):
"""
Returns the date that is `months` months after `start`
>>> df = spark.createDataFrame([('2015-04-08',)], ['dt'])
>>> df.select(add_months(df.dt, 1).alias('next_month')).collect()
[Row(next_month=datetime.date(2015, 5, 8))]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.add_months(_to_java_column(start), months))
@since(1.5)
def months_between(date1, date2, roundOff=True):
"""
Returns number of months between dates date1 and date2.
If date1 is later than date2, then the result is positive.
If date1 and date2 are on the same day of month, or both are the last day of month,
returns an integer (time of day will be ignored).
The result is rounded off to 8 digits unless `roundOff` is set to `False`.
>>> df = spark.createDataFrame([('1997-02-28 10:30:00', '1996-10-30')], ['date1', 'date2'])
>>> df.select(months_between(df.date1, df.date2).alias('months')).collect()
[Row(months=3.94959677)]
>>> df.select(months_between(df.date1, df.date2, False).alias('months')).collect()
[Row(months=3.9495967741935485)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.months_between(
_to_java_column(date1), _to_java_column(date2), roundOff))
[SPARK-8199][SPARK-8184][SPARK-8183][SPARK-8182][SPARK-8181][SPARK-8180][SPARK-8179][SPARK-8177][SPARK-8178][SPARK-9115][SQL] date functions Jira: https://issues.apache.org/jira/browse/SPARK-8199 https://issues.apache.org/jira/browse/SPARK-8184 https://issues.apache.org/jira/browse/SPARK-8183 https://issues.apache.org/jira/browse/SPARK-8182 https://issues.apache.org/jira/browse/SPARK-8181 https://issues.apache.org/jira/browse/SPARK-8180 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-8177 https://issues.apache.org/jira/browse/SPARK-8179 https://issues.apache.org/jira/browse/SPARK-9115 Regarding `day`and `dayofmonth` are both necessary? ~~I am going to add `Quarter` to this PR as well.~~ Done. ~~As soon as the Scala coding is reviewed and discussed, I'll add the python api.~~ Done Author: Tarek Auel <tarek.auel@googlemail.com> Author: Tarek Auel <tarek.auel@gmail.com> Closes #6981 from tarekauel/SPARK-8199 and squashes the following commits: f7b4c8c [Tarek Auel] [SPARK-8199] fixed bug in tests bb567b6 [Tarek Auel] [SPARK-8199] fixed test 3e095ba [Tarek Auel] [SPARK-8199] style and timezone fix 256c357 [Tarek Auel] [SPARK-8199] code cleanup 5983dcc [Tarek Auel] [SPARK-8199] whitespace fix 6e0c78f [Tarek Auel] [SPARK-8199] removed setTimeZone in tests, according to cloud-fans comment in #7488 4afc09c [Tarek Auel] [SPARK-8199] concise leap year handling ea6c110 [Tarek Auel] [SPARK-8199] fix after merging master 70238e0 [Tarek Auel] Merge branch 'master' into SPARK-8199 3c6ae2e [Tarek Auel] [SPARK-8199] removed binary search fb98ba0 [Tarek Auel] [SPARK-8199] python docstring fix cdfae27 [Tarek Auel] [SPARK-8199] cleanup & python docstring fix 746b80a [Tarek Auel] [SPARK-8199] build fix 0ad6db8 [Tarek Auel] [SPARK-8199] minor fix 523542d [Tarek Auel] [SPARK-8199] address comments 2259299 [Tarek Auel] [SPARK-8199] day_of_month alias d01b977 [Tarek Auel] [SPARK-8199] python underscore 56c4a92 [Tarek Auel] [SPARK-8199] update python docu e223bc0 [Tarek Auel] [SPARK-8199] refactoring d6aa14e [Tarek Auel] [SPARK-8199] fixed Hive compatibility b382267 [Tarek Auel] [SPARK-8199] fixed bug in day calculation; removed set TimeZone in HiveCompatibilitySuite for test purposes; removed Hive tests for second and minute, because we can cast '2015-03-18' to a timestamp and extract a minute/second from it 1b2e540 [Tarek Auel] [SPARK-8119] style fix 0852655 [Tarek Auel] [SPARK-8119] changed from ExpectsInputTypes to implicit casts ec87c69 [Tarek Auel] [SPARK-8119] bug fixing and refactoring 1358cdc [Tarek Auel] Merge remote-tracking branch 'origin/master' into SPARK-8199 740af0e [Tarek Auel] implement date function using a calculation based on days 4fb66da [Tarek Auel] WIP: date functions on calculation only 1a436c9 [Tarek Auel] wip f775f39 [Tarek Auel] fixed return type ad17e96 [Tarek Auel] improved implementation c42b444 [Tarek Auel] Removed merge conflict file ccb723c [Tarek Auel] [SPARK-8199] style and fixed merge issues 10e4ad1 [Tarek Auel] Merge branch 'master' into date-functions-fast 7d9f0eb [Tarek Auel] [SPARK-8199] git renaming issue f3e7a9f [Tarek Auel] [SPARK-8199] revert change in DataFrameFunctionsSuite 6f5d95c [Tarek Auel] [SPARK-8199] fixed year interval d9f8ac3 [Tarek Auel] [SPARK-8199] implement fast track 7bc9d93 [Tarek Auel] Merge branch 'master' into SPARK-8199 5a105d9 [Tarek Auel] [SPARK-8199] rebase after #6985 got merged eb6760d [Tarek Auel] Merge branch 'master' into SPARK-8199 f120415 [Tarek Auel] improved runtime a8edebd [Tarek Auel] use Calendar instead of SimpleDateFormat 5fe74e1 [Tarek Auel] fixed python style 3bfac90 [Tarek Auel] fixed style 356df78 [Tarek Auel] rely on cast mechanism of Spark. Simplified implementation 02efc5d [Tarek Auel] removed doubled code a5ea120 [Tarek Auel] added python api; changed test to be more meaningful b680db6 [Tarek Auel] added codegeneration to all functions c739788 [Tarek Auel] added support for quarter SPARK-8178 849fb41 [Tarek Auel] fixed stupid test 638596f [Tarek Auel] improved codegen 4d8049b [Tarek Auel] fixed tests and added type check 5ebb235 [Tarek Auel] resolved naming conflict d0e2f99 [Tarek Auel] date functions
2015-07-19 01:48:05 -04:00
[SPARK-16609] Add to_date/to_timestamp with format functions ## What changes were proposed in this pull request? This pull request adds two new user facing functions: - `to_date` which accepts an expression and a format and returns a date. - `to_timestamp` which accepts an expression and a format and returns a timestamp. For example, Given a date in format: `2016-21-05`. (YYYY-dd-MM) ### Date Function *Previously* ``` to_date(unix_timestamp(lit("2016-21-05"), "yyyy-dd-MM").cast("timestamp")) ``` *Current* ``` to_date(lit("2016-21-05"), "yyyy-dd-MM") ``` ### Timestamp Function *Previously* ``` unix_timestamp(lit("2016-21-05"), "yyyy-dd-MM").cast("timestamp") ``` *Current* ``` to_timestamp(lit("2016-21-05"), "yyyy-dd-MM") ``` ### Tasks - [X] Add `to_date` to Scala Functions - [x] Add `to_date` to Python Functions - [x] Add `to_date` to SQL Functions - [X] Add `to_timestamp` to Scala Functions - [x] Add `to_timestamp` to Python Functions - [x] Add `to_timestamp` to SQL Functions - [x] Add function to R ## How was this patch tested? - [x] Add Functions to `DateFunctionsSuite` - Test new `ParseToTimestamp` Expression (*not necessary*) - Test new `ParseToDate` Expression (*not necessary*) - [x] Add test for R - [x] Add test for Python in test.py Please review http://spark.apache.org/contributing.html before opening a pull request. Author: anabranch <wac.chambers@gmail.com> Author: Bill Chambers <bill@databricks.com> Author: anabranch <bill@databricks.com> Closes #16138 from anabranch/SPARK-16609.
2017-02-07 09:50:30 -05:00
@since(2.2)
def to_date(col, format=None):
"""Converts a :class:`Column` of :class:`pyspark.sql.types.StringType` or
:class:`pyspark.sql.types.TimestampType` into :class:`pyspark.sql.types.DateType`
[SPARK-20639][SQL] Add single argument support for to_timestamp in SQL with documentation improvement ## What changes were proposed in this pull request? This PR proposes three things as below: - Use casting rules to a timestamp in `to_timestamp` by default (it was `yyyy-MM-dd HH:mm:ss`). - Support single argument for `to_timestamp` similarly with APIs in other languages. For example, the one below works ``` import org.apache.spark.sql.functions._ Seq("2016-12-31 00:12:00.00").toDF("a").select(to_timestamp(col("a"))).show() ``` prints ``` +----------------------------------------+ |to_timestamp(`a`, 'yyyy-MM-dd HH:mm:ss')| +----------------------------------------+ | 2016-12-31 00:12:00| +----------------------------------------+ ``` whereas this does not work in SQL. **Before** ``` spark-sql> SELECT to_timestamp('2016-12-31 00:12:00'); Error in query: Invalid number of arguments for function to_timestamp; line 1 pos 7 ``` **After** ``` spark-sql> SELECT to_timestamp('2016-12-31 00:12:00'); 2016-12-31 00:12:00 ``` - Related document improvement for SQL function descriptions and other API descriptions accordingly. **Before** ``` spark-sql> DESCRIBE FUNCTION extended to_date; ... Usage: to_date(date_str, fmt) - Parses the `left` expression with the `fmt` expression. Returns null with invalid input. Extended Usage: Examples: > SELECT to_date('2016-12-31', 'yyyy-MM-dd'); 2016-12-31 ``` ``` spark-sql> DESCRIBE FUNCTION extended to_timestamp; ... Usage: to_timestamp(timestamp, fmt) - Parses the `left` expression with the `format` expression to a timestamp. Returns null with invalid input. Extended Usage: Examples: > SELECT to_timestamp('2016-12-31', 'yyyy-MM-dd'); 2016-12-31 00:00:00.0 ``` **After** ``` spark-sql> DESCRIBE FUNCTION extended to_date; ... Usage: to_date(date_str[, fmt]) - Parses the `date_str` expression with the `fmt` expression to a date. Returns null with invalid input. By default, it follows casting rules to a date if the `fmt` is omitted. Extended Usage: Examples: > SELECT to_date('2009-07-30 04:17:52'); 2009-07-30 > SELECT to_date('2016-12-31', 'yyyy-MM-dd'); 2016-12-31 ``` ``` spark-sql> DESCRIBE FUNCTION extended to_timestamp; ... Usage: to_timestamp(timestamp[, fmt]) - Parses the `timestamp` expression with the `fmt` expression to a timestamp. Returns null with invalid input. By default, it follows casting rules to a timestamp if the `fmt` is omitted. Extended Usage: Examples: > SELECT to_timestamp('2016-12-31 00:12:00'); 2016-12-31 00:12:00 > SELECT to_timestamp('2016-12-31', 'yyyy-MM-dd'); 2016-12-31 00:00:00 ``` ## How was this patch tested? Added tests in `datetime.sql`. Author: hyukjinkwon <gurwls223@gmail.com> Closes #17901 from HyukjinKwon/to_timestamp_arg.
2017-05-12 04:42:58 -04:00
using the optionally specified format. Specify formats according to
`DateTimeFormatter <https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html>`_. # noqa
[SPARK-20639][SQL] Add single argument support for to_timestamp in SQL with documentation improvement ## What changes were proposed in this pull request? This PR proposes three things as below: - Use casting rules to a timestamp in `to_timestamp` by default (it was `yyyy-MM-dd HH:mm:ss`). - Support single argument for `to_timestamp` similarly with APIs in other languages. For example, the one below works ``` import org.apache.spark.sql.functions._ Seq("2016-12-31 00:12:00.00").toDF("a").select(to_timestamp(col("a"))).show() ``` prints ``` +----------------------------------------+ |to_timestamp(`a`, 'yyyy-MM-dd HH:mm:ss')| +----------------------------------------+ | 2016-12-31 00:12:00| +----------------------------------------+ ``` whereas this does not work in SQL. **Before** ``` spark-sql> SELECT to_timestamp('2016-12-31 00:12:00'); Error in query: Invalid number of arguments for function to_timestamp; line 1 pos 7 ``` **After** ``` spark-sql> SELECT to_timestamp('2016-12-31 00:12:00'); 2016-12-31 00:12:00 ``` - Related document improvement for SQL function descriptions and other API descriptions accordingly. **Before** ``` spark-sql> DESCRIBE FUNCTION extended to_date; ... Usage: to_date(date_str, fmt) - Parses the `left` expression with the `fmt` expression. Returns null with invalid input. Extended Usage: Examples: > SELECT to_date('2016-12-31', 'yyyy-MM-dd'); 2016-12-31 ``` ``` spark-sql> DESCRIBE FUNCTION extended to_timestamp; ... Usage: to_timestamp(timestamp, fmt) - Parses the `left` expression with the `format` expression to a timestamp. Returns null with invalid input. Extended Usage: Examples: > SELECT to_timestamp('2016-12-31', 'yyyy-MM-dd'); 2016-12-31 00:00:00.0 ``` **After** ``` spark-sql> DESCRIBE FUNCTION extended to_date; ... Usage: to_date(date_str[, fmt]) - Parses the `date_str` expression with the `fmt` expression to a date. Returns null with invalid input. By default, it follows casting rules to a date if the `fmt` is omitted. Extended Usage: Examples: > SELECT to_date('2009-07-30 04:17:52'); 2009-07-30 > SELECT to_date('2016-12-31', 'yyyy-MM-dd'); 2016-12-31 ``` ``` spark-sql> DESCRIBE FUNCTION extended to_timestamp; ... Usage: to_timestamp(timestamp[, fmt]) - Parses the `timestamp` expression with the `fmt` expression to a timestamp. Returns null with invalid input. By default, it follows casting rules to a timestamp if the `fmt` is omitted. Extended Usage: Examples: > SELECT to_timestamp('2016-12-31 00:12:00'); 2016-12-31 00:12:00 > SELECT to_timestamp('2016-12-31', 'yyyy-MM-dd'); 2016-12-31 00:00:00 ``` ## How was this patch tested? Added tests in `datetime.sql`. Author: hyukjinkwon <gurwls223@gmail.com> Closes #17901 from HyukjinKwon/to_timestamp_arg.
2017-05-12 04:42:58 -04:00
By default, it follows casting rules to :class:`pyspark.sql.types.DateType` if the format
is omitted (equivalent to ``col.cast("date")``).
>>> df = spark.createDataFrame([('1997-02-28 10:30:00',)], ['t'])
>>> df.select(to_date(df.t).alias('date')).collect()
[Row(date=datetime.date(1997, 2, 28))]
[SPARK-16609] Add to_date/to_timestamp with format functions ## What changes were proposed in this pull request? This pull request adds two new user facing functions: - `to_date` which accepts an expression and a format and returns a date. - `to_timestamp` which accepts an expression and a format and returns a timestamp. For example, Given a date in format: `2016-21-05`. (YYYY-dd-MM) ### Date Function *Previously* ``` to_date(unix_timestamp(lit("2016-21-05"), "yyyy-dd-MM").cast("timestamp")) ``` *Current* ``` to_date(lit("2016-21-05"), "yyyy-dd-MM") ``` ### Timestamp Function *Previously* ``` unix_timestamp(lit("2016-21-05"), "yyyy-dd-MM").cast("timestamp") ``` *Current* ``` to_timestamp(lit("2016-21-05"), "yyyy-dd-MM") ``` ### Tasks - [X] Add `to_date` to Scala Functions - [x] Add `to_date` to Python Functions - [x] Add `to_date` to SQL Functions - [X] Add `to_timestamp` to Scala Functions - [x] Add `to_timestamp` to Python Functions - [x] Add `to_timestamp` to SQL Functions - [x] Add function to R ## How was this patch tested? - [x] Add Functions to `DateFunctionsSuite` - Test new `ParseToTimestamp` Expression (*not necessary*) - Test new `ParseToDate` Expression (*not necessary*) - [x] Add test for R - [x] Add test for Python in test.py Please review http://spark.apache.org/contributing.html before opening a pull request. Author: anabranch <wac.chambers@gmail.com> Author: Bill Chambers <bill@databricks.com> Author: anabranch <bill@databricks.com> Closes #16138 from anabranch/SPARK-16609.
2017-02-07 09:50:30 -05:00
>>> df = spark.createDataFrame([('1997-02-28 10:30:00',)], ['t'])
>>> df.select(to_date(df.t, 'yyyy-MM-dd HH:mm:ss').alias('date')).collect()
[Row(date=datetime.date(1997, 2, 28))]
"""
sc = SparkContext._active_spark_context
[SPARK-16609] Add to_date/to_timestamp with format functions ## What changes were proposed in this pull request? This pull request adds two new user facing functions: - `to_date` which accepts an expression and a format and returns a date. - `to_timestamp` which accepts an expression and a format and returns a timestamp. For example, Given a date in format: `2016-21-05`. (YYYY-dd-MM) ### Date Function *Previously* ``` to_date(unix_timestamp(lit("2016-21-05"), "yyyy-dd-MM").cast("timestamp")) ``` *Current* ``` to_date(lit("2016-21-05"), "yyyy-dd-MM") ``` ### Timestamp Function *Previously* ``` unix_timestamp(lit("2016-21-05"), "yyyy-dd-MM").cast("timestamp") ``` *Current* ``` to_timestamp(lit("2016-21-05"), "yyyy-dd-MM") ``` ### Tasks - [X] Add `to_date` to Scala Functions - [x] Add `to_date` to Python Functions - [x] Add `to_date` to SQL Functions - [X] Add `to_timestamp` to Scala Functions - [x] Add `to_timestamp` to Python Functions - [x] Add `to_timestamp` to SQL Functions - [x] Add function to R ## How was this patch tested? - [x] Add Functions to `DateFunctionsSuite` - Test new `ParseToTimestamp` Expression (*not necessary*) - Test new `ParseToDate` Expression (*not necessary*) - [x] Add test for R - [x] Add test for Python in test.py Please review http://spark.apache.org/contributing.html before opening a pull request. Author: anabranch <wac.chambers@gmail.com> Author: Bill Chambers <bill@databricks.com> Author: anabranch <bill@databricks.com> Closes #16138 from anabranch/SPARK-16609.
2017-02-07 09:50:30 -05:00
if format is None:
jc = sc._jvm.functions.to_date(_to_java_column(col))
else:
jc = sc._jvm.functions.to_date(_to_java_column(col), format)
return Column(jc)
@since(2.2)
def to_timestamp(col, format=None):
"""Converts a :class:`Column` of :class:`pyspark.sql.types.StringType` or
:class:`pyspark.sql.types.TimestampType` into :class:`pyspark.sql.types.DateType`
[SPARK-20639][SQL] Add single argument support for to_timestamp in SQL with documentation improvement ## What changes were proposed in this pull request? This PR proposes three things as below: - Use casting rules to a timestamp in `to_timestamp` by default (it was `yyyy-MM-dd HH:mm:ss`). - Support single argument for `to_timestamp` similarly with APIs in other languages. For example, the one below works ``` import org.apache.spark.sql.functions._ Seq("2016-12-31 00:12:00.00").toDF("a").select(to_timestamp(col("a"))).show() ``` prints ``` +----------------------------------------+ |to_timestamp(`a`, 'yyyy-MM-dd HH:mm:ss')| +----------------------------------------+ | 2016-12-31 00:12:00| +----------------------------------------+ ``` whereas this does not work in SQL. **Before** ``` spark-sql> SELECT to_timestamp('2016-12-31 00:12:00'); Error in query: Invalid number of arguments for function to_timestamp; line 1 pos 7 ``` **After** ``` spark-sql> SELECT to_timestamp('2016-12-31 00:12:00'); 2016-12-31 00:12:00 ``` - Related document improvement for SQL function descriptions and other API descriptions accordingly. **Before** ``` spark-sql> DESCRIBE FUNCTION extended to_date; ... Usage: to_date(date_str, fmt) - Parses the `left` expression with the `fmt` expression. Returns null with invalid input. Extended Usage: Examples: > SELECT to_date('2016-12-31', 'yyyy-MM-dd'); 2016-12-31 ``` ``` spark-sql> DESCRIBE FUNCTION extended to_timestamp; ... Usage: to_timestamp(timestamp, fmt) - Parses the `left` expression with the `format` expression to a timestamp. Returns null with invalid input. Extended Usage: Examples: > SELECT to_timestamp('2016-12-31', 'yyyy-MM-dd'); 2016-12-31 00:00:00.0 ``` **After** ``` spark-sql> DESCRIBE FUNCTION extended to_date; ... Usage: to_date(date_str[, fmt]) - Parses the `date_str` expression with the `fmt` expression to a date. Returns null with invalid input. By default, it follows casting rules to a date if the `fmt` is omitted. Extended Usage: Examples: > SELECT to_date('2009-07-30 04:17:52'); 2009-07-30 > SELECT to_date('2016-12-31', 'yyyy-MM-dd'); 2016-12-31 ``` ``` spark-sql> DESCRIBE FUNCTION extended to_timestamp; ... Usage: to_timestamp(timestamp[, fmt]) - Parses the `timestamp` expression with the `fmt` expression to a timestamp. Returns null with invalid input. By default, it follows casting rules to a timestamp if the `fmt` is omitted. Extended Usage: Examples: > SELECT to_timestamp('2016-12-31 00:12:00'); 2016-12-31 00:12:00 > SELECT to_timestamp('2016-12-31', 'yyyy-MM-dd'); 2016-12-31 00:00:00 ``` ## How was this patch tested? Added tests in `datetime.sql`. Author: hyukjinkwon <gurwls223@gmail.com> Closes #17901 from HyukjinKwon/to_timestamp_arg.
2017-05-12 04:42:58 -04:00
using the optionally specified format. Specify formats according to
`DateTimeFormatter <https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html>`_. # noqa
[SPARK-20639][SQL] Add single argument support for to_timestamp in SQL with documentation improvement ## What changes were proposed in this pull request? This PR proposes three things as below: - Use casting rules to a timestamp in `to_timestamp` by default (it was `yyyy-MM-dd HH:mm:ss`). - Support single argument for `to_timestamp` similarly with APIs in other languages. For example, the one below works ``` import org.apache.spark.sql.functions._ Seq("2016-12-31 00:12:00.00").toDF("a").select(to_timestamp(col("a"))).show() ``` prints ``` +----------------------------------------+ |to_timestamp(`a`, 'yyyy-MM-dd HH:mm:ss')| +----------------------------------------+ | 2016-12-31 00:12:00| +----------------------------------------+ ``` whereas this does not work in SQL. **Before** ``` spark-sql> SELECT to_timestamp('2016-12-31 00:12:00'); Error in query: Invalid number of arguments for function to_timestamp; line 1 pos 7 ``` **After** ``` spark-sql> SELECT to_timestamp('2016-12-31 00:12:00'); 2016-12-31 00:12:00 ``` - Related document improvement for SQL function descriptions and other API descriptions accordingly. **Before** ``` spark-sql> DESCRIBE FUNCTION extended to_date; ... Usage: to_date(date_str, fmt) - Parses the `left` expression with the `fmt` expression. Returns null with invalid input. Extended Usage: Examples: > SELECT to_date('2016-12-31', 'yyyy-MM-dd'); 2016-12-31 ``` ``` spark-sql> DESCRIBE FUNCTION extended to_timestamp; ... Usage: to_timestamp(timestamp, fmt) - Parses the `left` expression with the `format` expression to a timestamp. Returns null with invalid input. Extended Usage: Examples: > SELECT to_timestamp('2016-12-31', 'yyyy-MM-dd'); 2016-12-31 00:00:00.0 ``` **After** ``` spark-sql> DESCRIBE FUNCTION extended to_date; ... Usage: to_date(date_str[, fmt]) - Parses the `date_str` expression with the `fmt` expression to a date. Returns null with invalid input. By default, it follows casting rules to a date if the `fmt` is omitted. Extended Usage: Examples: > SELECT to_date('2009-07-30 04:17:52'); 2009-07-30 > SELECT to_date('2016-12-31', 'yyyy-MM-dd'); 2016-12-31 ``` ``` spark-sql> DESCRIBE FUNCTION extended to_timestamp; ... Usage: to_timestamp(timestamp[, fmt]) - Parses the `timestamp` expression with the `fmt` expression to a timestamp. Returns null with invalid input. By default, it follows casting rules to a timestamp if the `fmt` is omitted. Extended Usage: Examples: > SELECT to_timestamp('2016-12-31 00:12:00'); 2016-12-31 00:12:00 > SELECT to_timestamp('2016-12-31', 'yyyy-MM-dd'); 2016-12-31 00:00:00 ``` ## How was this patch tested? Added tests in `datetime.sql`. Author: hyukjinkwon <gurwls223@gmail.com> Closes #17901 from HyukjinKwon/to_timestamp_arg.
2017-05-12 04:42:58 -04:00
By default, it follows casting rules to :class:`pyspark.sql.types.TimestampType` if the format
is omitted (equivalent to ``col.cast("timestamp")``).
[SPARK-16609] Add to_date/to_timestamp with format functions ## What changes were proposed in this pull request? This pull request adds two new user facing functions: - `to_date` which accepts an expression and a format and returns a date. - `to_timestamp` which accepts an expression and a format and returns a timestamp. For example, Given a date in format: `2016-21-05`. (YYYY-dd-MM) ### Date Function *Previously* ``` to_date(unix_timestamp(lit("2016-21-05"), "yyyy-dd-MM").cast("timestamp")) ``` *Current* ``` to_date(lit("2016-21-05"), "yyyy-dd-MM") ``` ### Timestamp Function *Previously* ``` unix_timestamp(lit("2016-21-05"), "yyyy-dd-MM").cast("timestamp") ``` *Current* ``` to_timestamp(lit("2016-21-05"), "yyyy-dd-MM") ``` ### Tasks - [X] Add `to_date` to Scala Functions - [x] Add `to_date` to Python Functions - [x] Add `to_date` to SQL Functions - [X] Add `to_timestamp` to Scala Functions - [x] Add `to_timestamp` to Python Functions - [x] Add `to_timestamp` to SQL Functions - [x] Add function to R ## How was this patch tested? - [x] Add Functions to `DateFunctionsSuite` - Test new `ParseToTimestamp` Expression (*not necessary*) - Test new `ParseToDate` Expression (*not necessary*) - [x] Add test for R - [x] Add test for Python in test.py Please review http://spark.apache.org/contributing.html before opening a pull request. Author: anabranch <wac.chambers@gmail.com> Author: Bill Chambers <bill@databricks.com> Author: anabranch <bill@databricks.com> Closes #16138 from anabranch/SPARK-16609.
2017-02-07 09:50:30 -05:00
>>> df = spark.createDataFrame([('1997-02-28 10:30:00',)], ['t'])
>>> df.select(to_timestamp(df.t).alias('dt')).collect()
[Row(dt=datetime.datetime(1997, 2, 28, 10, 30))]
>>> df = spark.createDataFrame([('1997-02-28 10:30:00',)], ['t'])
>>> df.select(to_timestamp(df.t, 'yyyy-MM-dd HH:mm:ss').alias('dt')).collect()
[Row(dt=datetime.datetime(1997, 2, 28, 10, 30))]
"""
sc = SparkContext._active_spark_context
if format is None:
jc = sc._jvm.functions.to_timestamp(_to_java_column(col))
else:
jc = sc._jvm.functions.to_timestamp(_to_java_column(col), format)
return Column(jc)
@since(1.5)
def trunc(date, format):
"""
Returns date truncated to the unit specified by the format.
:param format: 'year', 'yyyy', 'yy' or 'month', 'mon', 'mm'
>>> df = spark.createDataFrame([('1997-02-28',)], ['d'])
>>> df.select(trunc(df.d, 'year').alias('year')).collect()
[Row(year=datetime.date(1997, 1, 1))]
>>> df.select(trunc(df.d, 'mon').alias('month')).collect()
[Row(month=datetime.date(1997, 2, 1))]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.trunc(_to_java_column(date), format))
@since(2.3)
def date_trunc(format, timestamp):
"""
Returns timestamp truncated to the unit specified by the format.
:param format: 'year', 'yyyy', 'yy', 'month', 'mon', 'mm',
'day', 'dd', 'hour', 'minute', 'second', 'week', 'quarter'
>>> df = spark.createDataFrame([('1997-02-28 05:02:11',)], ['t'])
>>> df.select(date_trunc('year', df.t).alias('year')).collect()
[Row(year=datetime.datetime(1997, 1, 1, 0, 0))]
>>> df.select(date_trunc('mon', df.t).alias('month')).collect()
[Row(month=datetime.datetime(1997, 2, 1, 0, 0))]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.date_trunc(format, _to_java_column(timestamp)))
@since(1.5)
def next_day(date, dayOfWeek):
"""
Returns the first date which is later than the value of the date column.
Day of the week parameter is case insensitive, and accepts:
"Mon", "Tue", "Wed", "Thu", "Fri", "Sat", "Sun".
>>> df = spark.createDataFrame([('2015-07-27',)], ['d'])
>>> df.select(next_day(df.d, 'Sun').alias('date')).collect()
[Row(date=datetime.date(2015, 8, 2))]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.next_day(_to_java_column(date), dayOfWeek))
@since(1.5)
def last_day(date):
"""
Returns the last day of the month which the given date belongs to.
>>> df = spark.createDataFrame([('1997-02-10',)], ['d'])
>>> df.select(last_day(df.d).alias('date')).collect()
[Row(date=datetime.date(1997, 2, 28))]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.last_day(_to_java_column(date)))
@ignore_unicode_prefix
@since(1.5)
def from_unixtime(timestamp, format="yyyy-MM-dd HH:mm:ss"):
"""
Converts the number of seconds from unix epoch (1970-01-01 00:00:00 UTC) to a string
representing the timestamp of that moment in the current system time zone in the given
format.
>>> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles")
>>> time_df = spark.createDataFrame([(1428476400,)], ['unix_time'])
>>> time_df.select(from_unixtime('unix_time').alias('ts')).collect()
[Row(ts=u'2015-04-08 00:00:00')]
>>> spark.conf.unset("spark.sql.session.timeZone")
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.from_unixtime(_to_java_column(timestamp), format))
@since(1.5)
def unix_timestamp(timestamp=None, format='yyyy-MM-dd HH:mm:ss'):
"""
Convert time string with given pattern ('yyyy-MM-dd HH:mm:ss', by default)
to Unix time stamp (in seconds), using the default timezone and the default
locale, return null if fail.
if `timestamp` is None, then it returns current timestamp.
>>> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles")
>>> time_df = spark.createDataFrame([('2015-04-08',)], ['dt'])
>>> time_df.select(unix_timestamp('dt', 'yyyy-MM-dd').alias('unix_time')).collect()
[Row(unix_time=1428476400)]
>>> spark.conf.unset("spark.sql.session.timeZone")
"""
sc = SparkContext._active_spark_context
if timestamp is None:
return Column(sc._jvm.functions.unix_timestamp())
return Column(sc._jvm.functions.unix_timestamp(_to_java_column(timestamp), format))
@since(1.5)
def from_utc_timestamp(timestamp, tz):
"""
This is a common function for databases supporting TIMESTAMP WITHOUT TIMEZONE. This function
takes a timestamp which is timezone-agnostic, and interprets it as a timestamp in UTC, and
renders that timestamp as a timestamp in the given time zone.
However, timestamp in Spark represents number of microseconds from the Unix epoch, which is not
timezone-agnostic. So in Spark this function just shift the timestamp value from UTC timezone to
the given timezone.
This function may return confusing result if the input is a string with timezone, e.g.
'2018-03-13T06:18:23+00:00'. The reason is that, Spark firstly cast the string to timestamp
according to the timezone in the string, and finally display the result by converting the
timestamp to string according to the session local timezone.
:param timestamp: the column that contains timestamps
:param tz: a string that has the ID of timezone, e.g. "GMT", "America/Los_Angeles", etc
.. versionchanged:: 2.4
`tz` can take a :class:`Column` containing timezone ID strings.
>>> df = spark.createDataFrame([('1997-02-28 10:30:00', 'JST')], ['ts', 'tz'])
>>> df.select(from_utc_timestamp(df.ts, "PST").alias('local_time')).collect()
[Row(local_time=datetime.datetime(1997, 2, 28, 2, 30))]
>>> df.select(from_utc_timestamp(df.ts, df.tz).alias('local_time')).collect()
[Row(local_time=datetime.datetime(1997, 2, 28, 19, 30))]
.. note:: Deprecated in 3.0. See SPARK-25496
"""
warnings.warn("Deprecated in 3.0. See SPARK-25496", DeprecationWarning)
sc = SparkContext._active_spark_context
if isinstance(tz, Column):
tz = _to_java_column(tz)
return Column(sc._jvm.functions.from_utc_timestamp(_to_java_column(timestamp), tz))
@since(1.5)
def to_utc_timestamp(timestamp, tz):
"""
This is a common function for databases supporting TIMESTAMP WITHOUT TIMEZONE. This function
takes a timestamp which is timezone-agnostic, and interprets it as a timestamp in the given
timezone, and renders that timestamp as a timestamp in UTC.
However, timestamp in Spark represents number of microseconds from the Unix epoch, which is not
timezone-agnostic. So in Spark this function just shift the timestamp value from the given
timezone to UTC timezone.
This function may return confusing result if the input is a string with timezone, e.g.
'2018-03-13T06:18:23+00:00'. The reason is that, Spark firstly cast the string to timestamp
according to the timezone in the string, and finally display the result by converting the
timestamp to string according to the session local timezone.
:param timestamp: the column that contains timestamps
:param tz: a string that has the ID of timezone, e.g. "GMT", "America/Los_Angeles", etc
.. versionchanged:: 2.4
`tz` can take a :class:`Column` containing timezone ID strings.
>>> df = spark.createDataFrame([('1997-02-28 10:30:00', 'JST')], ['ts', 'tz'])
>>> df.select(to_utc_timestamp(df.ts, "PST").alias('utc_time')).collect()
[Row(utc_time=datetime.datetime(1997, 2, 28, 18, 30))]
>>> df.select(to_utc_timestamp(df.ts, df.tz).alias('utc_time')).collect()
[Row(utc_time=datetime.datetime(1997, 2, 28, 1, 30))]
.. note:: Deprecated in 3.0. See SPARK-25496
"""
warnings.warn("Deprecated in 3.0. See SPARK-25496", DeprecationWarning)
sc = SparkContext._active_spark_context
if isinstance(tz, Column):
tz = _to_java_column(tz)
return Column(sc._jvm.functions.to_utc_timestamp(_to_java_column(timestamp), tz))
@since(2.0)
@ignore_unicode_prefix
def window(timeColumn, windowDuration, slideDuration=None, startTime=None):
"""Bucketize rows into one or more time windows given a timestamp specifying column. Window
starts are inclusive but the window ends are exclusive, e.g. 12:05 will be in the window
[12:05,12:10) but not in [12:00,12:05). Windows can support microsecond precision. Windows in
the order of months are not supported.
The time column must be of :class:`pyspark.sql.types.TimestampType`.
Durations are provided as strings, e.g. '1 second', '1 day 12 hours', '2 minutes'. Valid
interval strings are 'week', 'day', 'hour', 'minute', 'second', 'millisecond', 'microsecond'.
If the ``slideDuration`` is not provided, the windows will be tumbling windows.
The startTime is the offset with respect to 1970-01-01 00:00:00 UTC with which to start
window intervals. For example, in order to have hourly tumbling windows that start 15 minutes
past the hour, e.g. 12:15-13:15, 13:15-14:15... provide `startTime` as `15 minutes`.
The output column will be a struct called 'window' by default with the nested columns 'start'
and 'end', where 'start' and 'end' will be of :class:`pyspark.sql.types.TimestampType`.
>>> df = spark.createDataFrame([("2016-03-11 09:00:07", 1)]).toDF("date", "val")
>>> w = df.groupBy(window("date", "5 seconds")).agg(sum("val").alias("sum"))
>>> w.select(w.window.start.cast("string").alias("start"),
... w.window.end.cast("string").alias("end"), "sum").collect()
[Row(start=u'2016-03-11 09:00:05', end=u'2016-03-11 09:00:10', sum=1)]
"""
def check_string_field(field, fieldName):
if not field or type(field) is not str:
raise TypeError("%s should be provided as a string" % fieldName)
sc = SparkContext._active_spark_context
time_col = _to_java_column(timeColumn)
check_string_field(windowDuration, "windowDuration")
if slideDuration and startTime:
check_string_field(slideDuration, "slideDuration")
check_string_field(startTime, "startTime")
res = sc._jvm.functions.window(time_col, windowDuration, slideDuration, startTime)
elif slideDuration:
check_string_field(slideDuration, "slideDuration")
res = sc._jvm.functions.window(time_col, windowDuration, slideDuration)
elif startTime:
check_string_field(startTime, "startTime")
res = sc._jvm.functions.window(time_col, windowDuration, windowDuration, startTime)
else:
res = sc._jvm.functions.window(time_col, windowDuration)
return Column(res)
# ---------------------------- misc functions ----------------------------------
@since(1.5)
@ignore_unicode_prefix
def crc32(col):
"""
Calculates the cyclic redundancy check value (CRC32) of a binary column and
returns the value as a bigint.
>>> spark.createDataFrame([('ABC',)], ['a']).select(crc32('a').alias('crc32')).collect()
[Row(crc32=2743272264)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.crc32(_to_java_column(col)))
@ignore_unicode_prefix
@since(1.5)
def md5(col):
"""Calculates the MD5 digest and returns the value as a 32 character hex string.
>>> spark.createDataFrame([('ABC',)], ['a']).select(md5('a').alias('hash')).collect()
[Row(hash=u'902fbdd2b1df0c4f70b4a5d23525e932')]
"""
sc = SparkContext._active_spark_context
jc = sc._jvm.functions.md5(_to_java_column(col))
return Column(jc)
@ignore_unicode_prefix
@since(1.5)
def sha1(col):
"""Returns the hex string result of SHA-1.
>>> spark.createDataFrame([('ABC',)], ['a']).select(sha1('a').alias('hash')).collect()
[Row(hash=u'3c01bdbb26f358bab27f267924aa2c9a03fcfdb8')]
"""
sc = SparkContext._active_spark_context
jc = sc._jvm.functions.sha1(_to_java_column(col))
return Column(jc)
@ignore_unicode_prefix
@since(1.5)
def sha2(col, numBits):
"""Returns the hex string result of SHA-2 family of hash functions (SHA-224, SHA-256, SHA-384,
and SHA-512). The numBits indicates the desired bit length of the result, which must have a
value of 224, 256, 384, 512, or 0 (which is equivalent to 256).
>>> digests = df.select(sha2(df.name, 256).alias('s')).collect()
>>> digests[0]
Row(s=u'3bc51062973c458d5a6f2d8d64a023246354ad7e064b1e4e009ec8a0699a3043')
>>> digests[1]
Row(s=u'cd9fb1e148ccd8442e5aa74904cc73bf6fb54d1d54d333bd596aa9bb4bb4e961')
"""
sc = SparkContext._active_spark_context
jc = sc._jvm.functions.sha2(_to_java_column(col), numBits)
return Column(jc)
@since(2.0)
def hash(*cols):
"""Calculates the hash code of given columns, and returns the result as an int column.
>>> spark.createDataFrame([('ABC',)], ['a']).select(hash('a').alias('hash')).collect()
[Row(hash=-757602832)]
"""
sc = SparkContext._active_spark_context
jc = sc._jvm.functions.hash(_to_seq(sc, cols, _to_java_column))
return Column(jc)
@since(3.0)
def xxhash64(*cols):
"""Calculates the hash code of given columns using the 64-bit variant of the xxHash algorithm,
and returns the result as a long column.
>>> spark.createDataFrame([('ABC',)], ['a']).select(xxhash64('a').alias('hash')).collect()
[Row(hash=4105715581806190027)]
"""
sc = SparkContext._active_spark_context
jc = sc._jvm.functions.xxhash64(_to_seq(sc, cols, _to_java_column))
return Column(jc)
# ---------------------- String/Binary functions ------------------------------
_string_functions = {
[SPARK-26979][PYTHON][FOLLOW-UP] Make binary math/string functions take string as columns as well ## What changes were proposed in this pull request? This is a followup of https://github.com/apache/spark/pull/23882 to handle binary math/string functions. For instance, see the cases below: **Before:** ```python >>> from pyspark.sql.functions import lit, ascii >>> spark.range(1).select(lit('a').alias("value")).select(ascii("value")) Traceback (most recent call last): File "<stdin>", line 1, in <module> File "/.../spark/python/pyspark/sql/functions.py", line 51, in _ jc = getattr(sc._jvm.functions, name)(col._jc if isinstance(col, Column) else col) File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1286, in __call__ File "/.../spark/python/pyspark/sql/utils.py", line 63, in deco return f(*a, **kw) File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/protocol.py", line 332, in get_return_value py4j.protocol.Py4JError: An error occurred while calling z:org.apache.spark.sql.functions.ascii. Trace: py4j.Py4JException: Method ascii([class java.lang.String]) does not exist at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:318) at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:339) at py4j.Gateway.invoke(Gateway.java:276) at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132) at py4j.commands.CallCommand.execute(CallCommand.java:79) at py4j.GatewayConnection.run(GatewayConnection.java:238) at java.lang.Thread.run(Thread.java:748) ``` ```python >>> from pyspark.sql.functions import atan2 >>> spark.range(1).select(atan2("id", "id")) Traceback (most recent call last): File "<stdin>", line 1, in <module> File "/.../spark/python/pyspark/sql/functions.py", line 78, in _ jc = getattr(sc._jvm.functions, name)(col1._jc if isinstance(col1, Column) else float(col1), ValueError: could not convert string to float: id ``` **After:** ```python >>> from pyspark.sql.functions import lit, ascii >>> spark.range(1).select(lit('a').alias("value")).select(ascii("value")) DataFrame[ascii(value): int] ``` ```python >>> from pyspark.sql.functions import atan2 >>> spark.range(1).select(atan2("id", "id")) DataFrame[ATAN2(id, id): double] ``` Note that, - This PR causes a slight behaviour changes for math functions. For instance, numbers as strings (e.g., `"1"`) were supported as arguments of binary math functions before. After this PR, it recognises it as column names. - I also intentionally didn't document this behaviour changes since we're going ahead for Spark 3.0 and I don't think numbers as strings make much sense in math functions. - There is another exception `when`, which takes string as literal values as below. This PR doeesn't fix this ambiguity. ```python >>> spark.range(1).select(when(lit(True), col("id"))).show() ``` ``` +--------------------------+ |CASE WHEN true THEN id END| +--------------------------+ | 0| +--------------------------+ ``` ```python >>> spark.range(1).select(when(lit(True), "id")).show() ``` ``` +--------------------------+ |CASE WHEN true THEN id END| +--------------------------+ | id| +--------------------------+ ``` This PR also fixes as below: https://github.com/apache/spark/pull/23882 fixed it to: - Rename `_create_function` to `_create_name_function` - Define new `_create_function` to take strings as column names. This PR, I proposes to: - Revert `_create_name_function` name to `_create_function`. - Define new `_create_function_over_column` to take strings as column names. ## How was this patch tested? Some unit tests were added for binary math / string functions. Closes #24121 from HyukjinKwon/SPARK-26979. Authored-by: Hyukjin Kwon <gurwls223@apache.org> Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-19 19:06:10 -04:00
'upper': 'Converts a string expression to upper case.',
'lower': 'Converts a string expression to lower case.',
'ascii': 'Computes the numeric value of the first character of the string column.',
'base64': 'Computes the BASE64 encoding of a binary column and returns it as a string column.',
'unbase64': 'Decodes a BASE64 encoded string column and returns it as a binary column.',
'ltrim': 'Trim the spaces from left end for the specified string value.',
'rtrim': 'Trim the spaces from right end for the specified string value.',
'trim': 'Trim the spaces from both ends for the specified string column.',
}
for _name, _doc in _string_functions.items():
[SPARK-26979][PYTHON][FOLLOW-UP] Make binary math/string functions take string as columns as well ## What changes were proposed in this pull request? This is a followup of https://github.com/apache/spark/pull/23882 to handle binary math/string functions. For instance, see the cases below: **Before:** ```python >>> from pyspark.sql.functions import lit, ascii >>> spark.range(1).select(lit('a').alias("value")).select(ascii("value")) Traceback (most recent call last): File "<stdin>", line 1, in <module> File "/.../spark/python/pyspark/sql/functions.py", line 51, in _ jc = getattr(sc._jvm.functions, name)(col._jc if isinstance(col, Column) else col) File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1286, in __call__ File "/.../spark/python/pyspark/sql/utils.py", line 63, in deco return f(*a, **kw) File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/protocol.py", line 332, in get_return_value py4j.protocol.Py4JError: An error occurred while calling z:org.apache.spark.sql.functions.ascii. Trace: py4j.Py4JException: Method ascii([class java.lang.String]) does not exist at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:318) at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:339) at py4j.Gateway.invoke(Gateway.java:276) at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132) at py4j.commands.CallCommand.execute(CallCommand.java:79) at py4j.GatewayConnection.run(GatewayConnection.java:238) at java.lang.Thread.run(Thread.java:748) ``` ```python >>> from pyspark.sql.functions import atan2 >>> spark.range(1).select(atan2("id", "id")) Traceback (most recent call last): File "<stdin>", line 1, in <module> File "/.../spark/python/pyspark/sql/functions.py", line 78, in _ jc = getattr(sc._jvm.functions, name)(col1._jc if isinstance(col1, Column) else float(col1), ValueError: could not convert string to float: id ``` **After:** ```python >>> from pyspark.sql.functions import lit, ascii >>> spark.range(1).select(lit('a').alias("value")).select(ascii("value")) DataFrame[ascii(value): int] ``` ```python >>> from pyspark.sql.functions import atan2 >>> spark.range(1).select(atan2("id", "id")) DataFrame[ATAN2(id, id): double] ``` Note that, - This PR causes a slight behaviour changes for math functions. For instance, numbers as strings (e.g., `"1"`) were supported as arguments of binary math functions before. After this PR, it recognises it as column names. - I also intentionally didn't document this behaviour changes since we're going ahead for Spark 3.0 and I don't think numbers as strings make much sense in math functions. - There is another exception `when`, which takes string as literal values as below. This PR doeesn't fix this ambiguity. ```python >>> spark.range(1).select(when(lit(True), col("id"))).show() ``` ``` +--------------------------+ |CASE WHEN true THEN id END| +--------------------------+ | 0| +--------------------------+ ``` ```python >>> spark.range(1).select(when(lit(True), "id")).show() ``` ``` +--------------------------+ |CASE WHEN true THEN id END| +--------------------------+ | id| +--------------------------+ ``` This PR also fixes as below: https://github.com/apache/spark/pull/23882 fixed it to: - Rename `_create_function` to `_create_name_function` - Define new `_create_function` to take strings as column names. This PR, I proposes to: - Revert `_create_name_function` name to `_create_function`. - Define new `_create_function_over_column` to take strings as column names. ## How was this patch tested? Some unit tests were added for binary math / string functions. Closes #24121 from HyukjinKwon/SPARK-26979. Authored-by: Hyukjin Kwon <gurwls223@apache.org> Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-19 19:06:10 -04:00
globals()[_name] = since(1.5)(_create_function_over_column(_name, _doc))
del _name, _doc
@since(1.5)
@ignore_unicode_prefix
def concat_ws(sep, *cols):
"""
Concatenates multiple input string columns together into a single string column,
using the given separator.
>>> df = spark.createDataFrame([('abcd','123')], ['s', 'd'])
>>> df.select(concat_ws('-', df.s, df.d).alias('s')).collect()
[Row(s=u'abcd-123')]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.concat_ws(sep, _to_seq(sc, cols, _to_java_column)))
@since(1.5)
def decode(col, charset):
"""
Computes the first argument into a string from a binary using the provided character set
(one of 'US-ASCII', 'ISO-8859-1', 'UTF-8', 'UTF-16BE', 'UTF-16LE', 'UTF-16').
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.decode(_to_java_column(col), charset))
@since(1.5)
def encode(col, charset):
"""
Computes the first argument into a binary from a string using the provided character set
(one of 'US-ASCII', 'ISO-8859-1', 'UTF-8', 'UTF-16BE', 'UTF-16LE', 'UTF-16').
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.encode(_to_java_column(col), charset))
@ignore_unicode_prefix
@since(1.5)
def format_number(col, d):
"""
Formats the number X to a format like '#,--#,--#.--', rounded to d decimal places
with HALF_EVEN round mode, and returns the result as a string.
:param col: the column name of the numeric value to be formatted
:param d: the N decimal places
>>> spark.createDataFrame([(5,)], ['a']).select(format_number('a', 4).alias('v')).collect()
[Row(v=u'5.0000')]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.format_number(_to_java_column(col), d))
@ignore_unicode_prefix
@since(1.5)
def format_string(format, *cols):
"""
Formats the arguments in printf-style and returns the result as a string column.
:param col: the column name of the numeric value to be formatted
:param d: the N decimal places
>>> df = spark.createDataFrame([(5, "hello")], ['a', 'b'])
>>> df.select(format_string('%d %s', df.a, df.b).alias('v')).collect()
[Row(v=u'5 hello')]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.format_string(format, _to_seq(sc, cols, _to_java_column)))
@since(1.5)
def instr(str, substr):
"""
Locate the position of the first occurrence of substr column in the given string.
Returns null if either of the arguments are null.
[MINOR][DOCUMENTATION] Fix some minor descriptions in functions consistently with expressions ## What changes were proposed in this pull request? This PR proposes to improve documentation and fix some descriptions equivalent to several minor fixes identified in https://github.com/apache/spark/pull/15677 Also, this suggests to change `Note:` and `NOTE:` to `.. note::` consistently with the others which marks up pretty. ## How was this patch tested? Jenkins tests and manually. For PySpark, `Note:` and `NOTE:` to `.. note::` make the document as below: **From** ![2016-11-04 6 53 35](https://cloud.githubusercontent.com/assets/6477701/20002648/42989922-a2c5-11e6-8a32-b73eda49e8c3.png) ![2016-11-04 6 53 45](https://cloud.githubusercontent.com/assets/6477701/20002650/429fb310-a2c5-11e6-926b-e030d7eb0185.png) ![2016-11-04 6 54 11](https://cloud.githubusercontent.com/assets/6477701/20002649/429d570a-a2c5-11e6-9e7e-44090f337e32.png) ![2016-11-04 6 53 51](https://cloud.githubusercontent.com/assets/6477701/20002647/4297fc74-a2c5-11e6-801a-b89fbcbfca44.png) ![2016-11-04 6 53 51](https://cloud.githubusercontent.com/assets/6477701/20002697/749f5780-a2c5-11e6-835f-022e1f2f82e3.png) **To** ![2016-11-04 7 03 48](https://cloud.githubusercontent.com/assets/6477701/20002659/4961b504-a2c5-11e6-9ee0-ef0751482f47.png) ![2016-11-04 7 04 03](https://cloud.githubusercontent.com/assets/6477701/20002660/49871d3a-a2c5-11e6-85ea-d9a5d11efeff.png) ![2016-11-04 7 04 28](https://cloud.githubusercontent.com/assets/6477701/20002662/498e0f14-a2c5-11e6-803d-c0c5aeda4153.png) ![2016-11-04 7 33 39](https://cloud.githubusercontent.com/assets/6477701/20002731/a76e30d2-a2c5-11e6-993b-0481b8342d6b.png) ![2016-11-04 7 33 39](https://cloud.githubusercontent.com/assets/6477701/20002731/a76e30d2-a2c5-11e6-993b-0481b8342d6b.png) Author: hyukjinkwon <gurwls223@gmail.com> Closes #15765 from HyukjinKwon/minor-function-doc.
2016-11-06 00:47:33 -04:00
.. note:: The position is not zero based, but 1 based index. Returns 0 if substr
could not be found in str.
>>> df = spark.createDataFrame([('abcd',)], ['s',])
>>> df.select(instr(df.s, 'b').alias('s')).collect()
[Row(s=2)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.instr(_to_java_column(str), substr))
@since(1.5)
@ignore_unicode_prefix
def substring(str, pos, len):
"""
Substring starts at `pos` and is of length `len` when str is String type or
returns the slice of byte array that starts at `pos` in byte and is of length `len`
when str is Binary type.
.. note:: The position is not zero based, but 1 based index.
>>> df = spark.createDataFrame([('abcd',)], ['s',])
>>> df.select(substring(df.s, 1, 2).alias('s')).collect()
[Row(s=u'ab')]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.substring(_to_java_column(str), pos, len))
@since(1.5)
@ignore_unicode_prefix
def substring_index(str, delim, count):
"""
Returns the substring from string str before count occurrences of the delimiter delim.
If count is positive, everything the left of the final delimiter (counting from left) is
returned. If count is negative, every to the right of the final delimiter (counting from the
right) is returned. substring_index performs a case-sensitive match when searching for delim.
>>> df = spark.createDataFrame([('a.b.c.d',)], ['s'])
>>> df.select(substring_index(df.s, '.', 2).alias('s')).collect()
[Row(s=u'a.b')]
>>> df.select(substring_index(df.s, '.', -3).alias('s')).collect()
[Row(s=u'b.c.d')]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.substring_index(_to_java_column(str), delim, count))
@ignore_unicode_prefix
@since(1.5)
def levenshtein(left, right):
"""Computes the Levenshtein distance of the two given strings.
>>> df0 = spark.createDataFrame([('kitten', 'sitting',)], ['l', 'r'])
>>> df0.select(levenshtein('l', 'r').alias('d')).collect()
[Row(d=3)]
"""
sc = SparkContext._active_spark_context
jc = sc._jvm.functions.levenshtein(_to_java_column(left), _to_java_column(right))
return Column(jc)
@since(1.5)
def locate(substr, str, pos=1):
"""
Locate the position of the first occurrence of substr in a string column, after position pos.
[MINOR][DOCUMENTATION] Fix some minor descriptions in functions consistently with expressions ## What changes were proposed in this pull request? This PR proposes to improve documentation and fix some descriptions equivalent to several minor fixes identified in https://github.com/apache/spark/pull/15677 Also, this suggests to change `Note:` and `NOTE:` to `.. note::` consistently with the others which marks up pretty. ## How was this patch tested? Jenkins tests and manually. For PySpark, `Note:` and `NOTE:` to `.. note::` make the document as below: **From** ![2016-11-04 6 53 35](https://cloud.githubusercontent.com/assets/6477701/20002648/42989922-a2c5-11e6-8a32-b73eda49e8c3.png) ![2016-11-04 6 53 45](https://cloud.githubusercontent.com/assets/6477701/20002650/429fb310-a2c5-11e6-926b-e030d7eb0185.png) ![2016-11-04 6 54 11](https://cloud.githubusercontent.com/assets/6477701/20002649/429d570a-a2c5-11e6-9e7e-44090f337e32.png) ![2016-11-04 6 53 51](https://cloud.githubusercontent.com/assets/6477701/20002647/4297fc74-a2c5-11e6-801a-b89fbcbfca44.png) ![2016-11-04 6 53 51](https://cloud.githubusercontent.com/assets/6477701/20002697/749f5780-a2c5-11e6-835f-022e1f2f82e3.png) **To** ![2016-11-04 7 03 48](https://cloud.githubusercontent.com/assets/6477701/20002659/4961b504-a2c5-11e6-9ee0-ef0751482f47.png) ![2016-11-04 7 04 03](https://cloud.githubusercontent.com/assets/6477701/20002660/49871d3a-a2c5-11e6-85ea-d9a5d11efeff.png) ![2016-11-04 7 04 28](https://cloud.githubusercontent.com/assets/6477701/20002662/498e0f14-a2c5-11e6-803d-c0c5aeda4153.png) ![2016-11-04 7 33 39](https://cloud.githubusercontent.com/assets/6477701/20002731/a76e30d2-a2c5-11e6-993b-0481b8342d6b.png) ![2016-11-04 7 33 39](https://cloud.githubusercontent.com/assets/6477701/20002731/a76e30d2-a2c5-11e6-993b-0481b8342d6b.png) Author: hyukjinkwon <gurwls223@gmail.com> Closes #15765 from HyukjinKwon/minor-function-doc.
2016-11-06 00:47:33 -04:00
.. note:: The position is not zero based, but 1 based index. Returns 0 if substr
could not be found in str.
:param substr: a string
:param str: a Column of :class:`pyspark.sql.types.StringType`
:param pos: start position (zero based)
>>> df = spark.createDataFrame([('abcd',)], ['s',])
>>> df.select(locate('b', df.s, 1).alias('s')).collect()
[Row(s=2)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.locate(substr, _to_java_column(str), pos))
@since(1.5)
@ignore_unicode_prefix
def lpad(col, len, pad):
"""
Left-pad the string column to width `len` with `pad`.
>>> df = spark.createDataFrame([('abcd',)], ['s',])
>>> df.select(lpad(df.s, 6, '#').alias('s')).collect()
[Row(s=u'##abcd')]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.lpad(_to_java_column(col), len, pad))
@since(1.5)
@ignore_unicode_prefix
def rpad(col, len, pad):
"""
Right-pad the string column to width `len` with `pad`.
>>> df = spark.createDataFrame([('abcd',)], ['s',])
>>> df.select(rpad(df.s, 6, '#').alias('s')).collect()
[Row(s=u'abcd##')]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.rpad(_to_java_column(col), len, pad))
@since(1.5)
@ignore_unicode_prefix
def repeat(col, n):
"""
Repeats a string column n times, and returns it as a new string column.
>>> df = spark.createDataFrame([('ab',)], ['s',])
>>> df.select(repeat(df.s, 3).alias('s')).collect()
[Row(s=u'ababab')]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.repeat(_to_java_column(col), n))
@since(1.5)
@ignore_unicode_prefix
def split(str, pattern, limit=-1):
"""
Splits str around matches of the given pattern.
:param str: a string expression to split
:param pattern: a string representing a regular expression. The regex string should be
a Java regular expression.
:param limit: an integer which controls the number of times `pattern` is applied.
* ``limit > 0``: The resulting array's length will not be more than `limit`, and the
resulting array's last entry will contain all input beyond the last
matched pattern.
* ``limit <= 0``: `pattern` will be applied as many times as possible, and the resulting
array can be of any size.
.. versionchanged:: 3.0
`split` now takes an optional `limit` field. If not provided, default limit value is -1.
>>> df = spark.createDataFrame([('oneAtwoBthreeC',)], ['s',])
>>> df.select(split(df.s, '[ABC]', 2).alias('s')).collect()
[Row(s=[u'one', u'twoBthreeC'])]
>>> df.select(split(df.s, '[ABC]', -1).alias('s')).collect()
[Row(s=[u'one', u'two', u'three', u''])]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.split(_to_java_column(str), pattern, limit))
@ignore_unicode_prefix
@since(1.5)
def regexp_extract(str, pattern, idx):
r"""Extract a specific group matched by a Java regex, from the specified string column.
If the regex did not match, or the specified group did not match, an empty string is returned.
>>> df = spark.createDataFrame([('100-200',)], ['str'])
>>> df.select(regexp_extract('str', r'(\d+)-(\d+)', 1).alias('d')).collect()
[Row(d=u'100')]
>>> df = spark.createDataFrame([('foo',)], ['str'])
>>> df.select(regexp_extract('str', r'(\d+)', 1).alias('d')).collect()
[Row(d=u'')]
>>> df = spark.createDataFrame([('aaaac',)], ['str'])
>>> df.select(regexp_extract('str', '(a+)(b)?(c)', 2).alias('d')).collect()
[Row(d=u'')]
"""
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):
r"""Replace all substrings of the specified string value that match regexp with rep.
>>> df = spark.createDataFrame([('100-200',)], ['str'])
>>> df.select(regexp_replace('str', r'(\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 initcap(col):
"""Translate the first letter of each word to upper case in the sentence.
>>> spark.createDataFrame([('ab cd',)], ['a']).select(initcap("a").alias('v')).collect()
[Row(v=u'Ab Cd')]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.initcap(_to_java_column(col)))
@since(1.5)
@ignore_unicode_prefix
def soundex(col):
"""
Returns the SoundEx encoding for a string
>>> df = spark.createDataFrame([("Peters",),("Uhrbach",)], ['name'])
>>> df.select(soundex(df.name).alias("soundex")).collect()
[Row(soundex=u'P362'), Row(soundex=u'U612')]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.soundex(_to_java_column(col)))
@ignore_unicode_prefix
@since(1.5)
def bin(col):
"""Returns the string representation of the binary value of the given column.
>>> df.select(bin(df.age).alias('c')).collect()
[Row(c=u'10'), Row(c=u'101')]
"""
sc = SparkContext._active_spark_context
jc = sc._jvm.functions.bin(_to_java_column(col))
return Column(jc)
@ignore_unicode_prefix
@since(1.5)
def hex(col):
"""Computes hex value of the given column, which could be :class:`pyspark.sql.types.StringType`,
:class:`pyspark.sql.types.BinaryType`, :class:`pyspark.sql.types.IntegerType` or
:class:`pyspark.sql.types.LongType`.
>>> spark.createDataFrame([('ABC', 3)], ['a', 'b']).select(hex('a'), hex('b')).collect()
[Row(hex(a)=u'414243', hex(b)=u'3')]
"""
sc = SparkContext._active_spark_context
jc = sc._jvm.functions.hex(_to_java_column(col))
return Column(jc)
@ignore_unicode_prefix
@since(1.5)
def unhex(col):
"""Inverse of hex. Interprets each pair of characters as a hexadecimal number
and converts to the byte representation of number.
>>> spark.createDataFrame([('414243',)], ['a']).select(unhex('a')).collect()
[Row(unhex(a)=bytearray(b'ABC'))]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.unhex(_to_java_column(col)))
@ignore_unicode_prefix
@since(1.5)
def length(col):
"""Computes the character length of string data or number of bytes of binary data.
The length of character data includes the trailing spaces. The length of binary data
includes binary zeros.
>>> spark.createDataFrame([('ABC ',)], ['a']).select(length('a').alias('length')).collect()
[Row(length=4)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.length(_to_java_column(col)))
@ignore_unicode_prefix
@since(1.5)
def translate(srcCol, matching, replace):
"""A function translate any character in the `srcCol` by a character in `matching`.
The characters in `replace` is corresponding to the characters in `matching`.
The translate will happen when any character in the string matching with the character
in the `matching`.
>>> spark.createDataFrame([('translate',)], ['a']).select(translate('a', "rnlt", "123") \\
... .alias('r')).collect()
[Row(r=u'1a2s3ae')]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.translate(_to_java_column(srcCol), matching, replace))
# ---------------------- Collection functions ------------------------------
@ignore_unicode_prefix
@since(2.0)
def create_map(*cols):
"""Creates a new map column.
:param cols: list of column names (string) or list of :class:`Column` expressions that are
grouped as key-value pairs, e.g. (key1, value1, key2, value2, ...).
>>> df.select(create_map('name', 'age').alias("map")).collect()
[Row(map={u'Alice': 2}), Row(map={u'Bob': 5})]
>>> df.select(create_map([df.name, df.age]).alias("map")).collect()
[Row(map={u'Alice': 2}), Row(map={u'Bob': 5})]
"""
sc = SparkContext._active_spark_context
if len(cols) == 1 and isinstance(cols[0], (list, set)):
cols = cols[0]
jc = sc._jvm.functions.map(_to_seq(sc, cols, _to_java_column))
return Column(jc)
@since(2.4)
def map_from_arrays(col1, col2):
"""Creates a new map from two arrays.
:param col1: name of column containing a set of keys. All elements should not be null
:param col2: name of column containing a set of values
>>> df = spark.createDataFrame([([2, 5], ['a', 'b'])], ['k', 'v'])
>>> df.select(map_from_arrays(df.k, df.v).alias("map")).show()
+----------------+
| map|
+----------------+
|[2 -> a, 5 -> b]|
+----------------+
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.map_from_arrays(_to_java_column(col1), _to_java_column(col2)))
@since(1.4)
def array(*cols):
"""Creates a new array column.
:param cols: list of column names (string) or list of :class:`Column` expressions that have
the same data type.
>>> df.select(array('age', 'age').alias("arr")).collect()
[Row(arr=[2, 2]), Row(arr=[5, 5])]
>>> df.select(array([df.age, df.age]).alias("arr")).collect()
[Row(arr=[2, 2]), Row(arr=[5, 5])]
"""
sc = SparkContext._active_spark_context
if len(cols) == 1 and isinstance(cols[0], (list, set)):
cols = cols[0]
jc = sc._jvm.functions.array(_to_seq(sc, cols, _to_java_column))
return Column(jc)
[SPARK-8231] [SQL] Add array_contains This PR is based on #7580 , thanks to EntilZha PR for work on https://issues.apache.org/jira/browse/SPARK-8231 Currently, I have an initial implementation for contains. Based on discussion on JIRA, it should behave same as Hive: https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFArrayContains.java#L102-L128 Main points are: 1. If the array is empty, null, or the value is null, return false 2. If there is a type mismatch, throw error 3. If comparison is not supported, throw error Closes #7580 Author: Pedro Rodriguez <prodriguez@trulia.com> Author: Pedro Rodriguez <ski.rodriguez@gmail.com> Author: Davies Liu <davies@databricks.com> Closes #7949 from davies/array_contains and squashes the following commits: d3c08bc [Davies Liu] use foreach() to avoid copy bc3d1fe [Davies Liu] fix array_contains 719e37d [Davies Liu] Merge branch 'master' of github.com:apache/spark into array_contains e352cf9 [Pedro Rodriguez] fixed diff from master 4d5b0ff [Pedro Rodriguez] added docs and another type check ffc0591 [Pedro Rodriguez] fixed unit test 7a22deb [Pedro Rodriguez] Changed test to use strings instead of long/ints which are different between python 2 an 3 b5ffae8 [Pedro Rodriguez] fixed pyspark test 4e7dce3 [Pedro Rodriguez] added more docs 3082399 [Pedro Rodriguez] fixed unit test 46f9789 [Pedro Rodriguez] reverted change d3ca013 [Pedro Rodriguez] Fixed type checking to match hive behavior, then added tests to insure this 8528027 [Pedro Rodriguez] added more tests 686e029 [Pedro Rodriguez] fix scala style d262e9d [Pedro Rodriguez] reworked type checking code and added more tests 2517a58 [Pedro Rodriguez] removed unused import 28b4f71 [Pedro Rodriguez] fixed bug with type conversions and re-added tests 12f8795 [Pedro Rodriguez] fix scala style checks e8a20a9 [Pedro Rodriguez] added python df (broken atm) 65b562c [Pedro Rodriguez] made array_contains nullable false 33b45aa [Pedro Rodriguez] reordered test 9623c64 [Pedro Rodriguez] fixed test 4b4425b [Pedro Rodriguez] changed Arrays in tests to Seqs 72cb4b1 [Pedro Rodriguez] added checkInputTypes and docs 69c46fb [Pedro Rodriguez] added tests and codegen 9e0bfc4 [Pedro Rodriguez] initial attempt at implementation
2015-08-05 01:32:21 -04:00
@since(1.5)
def array_contains(col, value):
"""
Collection function: returns null if the array is null, true if the array contains the
given value, and false otherwise.
[SPARK-8231] [SQL] Add array_contains This PR is based on #7580 , thanks to EntilZha PR for work on https://issues.apache.org/jira/browse/SPARK-8231 Currently, I have an initial implementation for contains. Based on discussion on JIRA, it should behave same as Hive: https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFArrayContains.java#L102-L128 Main points are: 1. If the array is empty, null, or the value is null, return false 2. If there is a type mismatch, throw error 3. If comparison is not supported, throw error Closes #7580 Author: Pedro Rodriguez <prodriguez@trulia.com> Author: Pedro Rodriguez <ski.rodriguez@gmail.com> Author: Davies Liu <davies@databricks.com> Closes #7949 from davies/array_contains and squashes the following commits: d3c08bc [Davies Liu] use foreach() to avoid copy bc3d1fe [Davies Liu] fix array_contains 719e37d [Davies Liu] Merge branch 'master' of github.com:apache/spark into array_contains e352cf9 [Pedro Rodriguez] fixed diff from master 4d5b0ff [Pedro Rodriguez] added docs and another type check ffc0591 [Pedro Rodriguez] fixed unit test 7a22deb [Pedro Rodriguez] Changed test to use strings instead of long/ints which are different between python 2 an 3 b5ffae8 [Pedro Rodriguez] fixed pyspark test 4e7dce3 [Pedro Rodriguez] added more docs 3082399 [Pedro Rodriguez] fixed unit test 46f9789 [Pedro Rodriguez] reverted change d3ca013 [Pedro Rodriguez] Fixed type checking to match hive behavior, then added tests to insure this 8528027 [Pedro Rodriguez] added more tests 686e029 [Pedro Rodriguez] fix scala style d262e9d [Pedro Rodriguez] reworked type checking code and added more tests 2517a58 [Pedro Rodriguez] removed unused import 28b4f71 [Pedro Rodriguez] fixed bug with type conversions and re-added tests 12f8795 [Pedro Rodriguez] fix scala style checks e8a20a9 [Pedro Rodriguez] added python df (broken atm) 65b562c [Pedro Rodriguez] made array_contains nullable false 33b45aa [Pedro Rodriguez] reordered test 9623c64 [Pedro Rodriguez] fixed test 4b4425b [Pedro Rodriguez] changed Arrays in tests to Seqs 72cb4b1 [Pedro Rodriguez] added checkInputTypes and docs 69c46fb [Pedro Rodriguez] added tests and codegen 9e0bfc4 [Pedro Rodriguez] initial attempt at implementation
2015-08-05 01:32:21 -04:00
:param col: name of column containing array
:param value: value to check for in array
>>> df = spark.createDataFrame([(["a", "b", "c"],), ([],)], ['data'])
[SPARK-8231] [SQL] Add array_contains This PR is based on #7580 , thanks to EntilZha PR for work on https://issues.apache.org/jira/browse/SPARK-8231 Currently, I have an initial implementation for contains. Based on discussion on JIRA, it should behave same as Hive: https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFArrayContains.java#L102-L128 Main points are: 1. If the array is empty, null, or the value is null, return false 2. If there is a type mismatch, throw error 3. If comparison is not supported, throw error Closes #7580 Author: Pedro Rodriguez <prodriguez@trulia.com> Author: Pedro Rodriguez <ski.rodriguez@gmail.com> Author: Davies Liu <davies@databricks.com> Closes #7949 from davies/array_contains and squashes the following commits: d3c08bc [Davies Liu] use foreach() to avoid copy bc3d1fe [Davies Liu] fix array_contains 719e37d [Davies Liu] Merge branch 'master' of github.com:apache/spark into array_contains e352cf9 [Pedro Rodriguez] fixed diff from master 4d5b0ff [Pedro Rodriguez] added docs and another type check ffc0591 [Pedro Rodriguez] fixed unit test 7a22deb [Pedro Rodriguez] Changed test to use strings instead of long/ints which are different between python 2 an 3 b5ffae8 [Pedro Rodriguez] fixed pyspark test 4e7dce3 [Pedro Rodriguez] added more docs 3082399 [Pedro Rodriguez] fixed unit test 46f9789 [Pedro Rodriguez] reverted change d3ca013 [Pedro Rodriguez] Fixed type checking to match hive behavior, then added tests to insure this 8528027 [Pedro Rodriguez] added more tests 686e029 [Pedro Rodriguez] fix scala style d262e9d [Pedro Rodriguez] reworked type checking code and added more tests 2517a58 [Pedro Rodriguez] removed unused import 28b4f71 [Pedro Rodriguez] fixed bug with type conversions and re-added tests 12f8795 [Pedro Rodriguez] fix scala style checks e8a20a9 [Pedro Rodriguez] added python df (broken atm) 65b562c [Pedro Rodriguez] made array_contains nullable false 33b45aa [Pedro Rodriguez] reordered test 9623c64 [Pedro Rodriguez] fixed test 4b4425b [Pedro Rodriguez] changed Arrays in tests to Seqs 72cb4b1 [Pedro Rodriguez] added checkInputTypes and docs 69c46fb [Pedro Rodriguez] added tests and codegen 9e0bfc4 [Pedro Rodriguez] initial attempt at implementation
2015-08-05 01:32:21 -04:00
>>> df.select(array_contains(df.data, "a")).collect()
[Row(array_contains(data, a)=True), Row(array_contains(data, a)=False)]
[SPARK-8231] [SQL] Add array_contains This PR is based on #7580 , thanks to EntilZha PR for work on https://issues.apache.org/jira/browse/SPARK-8231 Currently, I have an initial implementation for contains. Based on discussion on JIRA, it should behave same as Hive: https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFArrayContains.java#L102-L128 Main points are: 1. If the array is empty, null, or the value is null, return false 2. If there is a type mismatch, throw error 3. If comparison is not supported, throw error Closes #7580 Author: Pedro Rodriguez <prodriguez@trulia.com> Author: Pedro Rodriguez <ski.rodriguez@gmail.com> Author: Davies Liu <davies@databricks.com> Closes #7949 from davies/array_contains and squashes the following commits: d3c08bc [Davies Liu] use foreach() to avoid copy bc3d1fe [Davies Liu] fix array_contains 719e37d [Davies Liu] Merge branch 'master' of github.com:apache/spark into array_contains e352cf9 [Pedro Rodriguez] fixed diff from master 4d5b0ff [Pedro Rodriguez] added docs and another type check ffc0591 [Pedro Rodriguez] fixed unit test 7a22deb [Pedro Rodriguez] Changed test to use strings instead of long/ints which are different between python 2 an 3 b5ffae8 [Pedro Rodriguez] fixed pyspark test 4e7dce3 [Pedro Rodriguez] added more docs 3082399 [Pedro Rodriguez] fixed unit test 46f9789 [Pedro Rodriguez] reverted change d3ca013 [Pedro Rodriguez] Fixed type checking to match hive behavior, then added tests to insure this 8528027 [Pedro Rodriguez] added more tests 686e029 [Pedro Rodriguez] fix scala style d262e9d [Pedro Rodriguez] reworked type checking code and added more tests 2517a58 [Pedro Rodriguez] removed unused import 28b4f71 [Pedro Rodriguez] fixed bug with type conversions and re-added tests 12f8795 [Pedro Rodriguez] fix scala style checks e8a20a9 [Pedro Rodriguez] added python df (broken atm) 65b562c [Pedro Rodriguez] made array_contains nullable false 33b45aa [Pedro Rodriguez] reordered test 9623c64 [Pedro Rodriguez] fixed test 4b4425b [Pedro Rodriguez] changed Arrays in tests to Seqs 72cb4b1 [Pedro Rodriguez] added checkInputTypes and docs 69c46fb [Pedro Rodriguez] added tests and codegen 9e0bfc4 [Pedro Rodriguez] initial attempt at implementation
2015-08-05 01:32:21 -04:00
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.array_contains(_to_java_column(col), value))
@since(2.4)
def arrays_overlap(a1, a2):
"""
Collection function: returns true if the arrays contain any common non-null element; if not,
returns null if both the arrays are non-empty and any of them contains a null element; returns
false otherwise.
>>> df = spark.createDataFrame([(["a", "b"], ["b", "c"]), (["a"], ["b", "c"])], ['x', 'y'])
>>> df.select(arrays_overlap(df.x, df.y).alias("overlap")).collect()
[Row(overlap=True), Row(overlap=False)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.arrays_overlap(_to_java_column(a1), _to_java_column(a2)))
@since(2.4)
def slice(x, start, length):
"""
Collection function: returns an array containing all the elements in `x` from index `start`
(or starting from the end if `start` is negative) with the specified `length`.
>>> df = spark.createDataFrame([([1, 2, 3],), ([4, 5],)], ['x'])
>>> df.select(slice(df.x, 2, 2).alias("sliced")).collect()
[Row(sliced=[2, 3]), Row(sliced=[5])]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.slice(_to_java_column(x), start, length))
@ignore_unicode_prefix
@since(2.4)
def array_join(col, delimiter, null_replacement=None):
"""
Concatenates the elements of `column` using the `delimiter`. Null values are replaced with
`null_replacement` if set, otherwise they are ignored.
>>> df = spark.createDataFrame([(["a", "b", "c"],), (["a", None],)], ['data'])
>>> df.select(array_join(df.data, ",").alias("joined")).collect()
[Row(joined=u'a,b,c'), Row(joined=u'a')]
>>> df.select(array_join(df.data, ",", "NULL").alias("joined")).collect()
[Row(joined=u'a,b,c'), Row(joined=u'a,NULL')]
"""
sc = SparkContext._active_spark_context
if null_replacement is None:
return Column(sc._jvm.functions.array_join(_to_java_column(col), delimiter))
else:
return Column(sc._jvm.functions.array_join(
_to_java_column(col), delimiter, null_replacement))
[SPARK-23736][SQL] Extending the concat function to support array columns ## What changes were proposed in this pull request? The PR adds a logic for easy concatenation of multiple array columns and covers: - Concat expression has been extended to support array columns - A Python wrapper ## How was this patch tested? New tests added into: - CollectionExpressionsSuite - DataFrameFunctionsSuite - typeCoercion/native/concat.sql ## Codegen examples ### Primitive-type elements ``` val df = Seq( (Seq(1 ,2), Seq(3, 4)), (Seq(1, 2, 3), null) ).toDF("a", "b") df.filter('a.isNotNull).select(concat('a, 'b)).debugCodegen() ``` Result: ``` /* 033 */ boolean inputadapter_isNull = inputadapter_row.isNullAt(0); /* 034 */ ArrayData inputadapter_value = inputadapter_isNull ? /* 035 */ null : (inputadapter_row.getArray(0)); /* 036 */ /* 037 */ if (!(!inputadapter_isNull)) continue; /* 038 */ /* 039 */ ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1); /* 040 */ /* 041 */ ArrayData[] project_args = new ArrayData[2]; /* 042 */ /* 043 */ if (!false) { /* 044 */ project_args[0] = inputadapter_value; /* 045 */ } /* 046 */ /* 047 */ boolean inputadapter_isNull1 = inputadapter_row.isNullAt(1); /* 048 */ ArrayData inputadapter_value1 = inputadapter_isNull1 ? /* 049 */ null : (inputadapter_row.getArray(1)); /* 050 */ if (!inputadapter_isNull1) { /* 051 */ project_args[1] = inputadapter_value1; /* 052 */ } /* 053 */ /* 054 */ ArrayData project_value = new Object() { /* 055 */ public ArrayData concat(ArrayData[] args) { /* 056 */ for (int z = 0; z < 2; z++) { /* 057 */ if (args[z] == null) return null; /* 058 */ } /* 059 */ /* 060 */ long project_numElements = 0L; /* 061 */ for (int z = 0; z < 2; z++) { /* 062 */ project_numElements += args[z].numElements(); /* 063 */ } /* 064 */ if (project_numElements > 2147483632) { /* 065 */ throw new RuntimeException("Unsuccessful try to concat arrays with " + project_numElements + /* 066 */ " elements due to exceeding the array size limit 2147483632."); /* 067 */ } /* 068 */ /* 069 */ long project_size = UnsafeArrayData.calculateSizeOfUnderlyingByteArray( /* 070 */ project_numElements, /* 071 */ 4); /* 072 */ if (project_size > 2147483632) { /* 073 */ throw new RuntimeException("Unsuccessful try to concat arrays with " + project_size + /* 074 */ " bytes of data due to exceeding the limit 2147483632 bytes" + /* 075 */ " for UnsafeArrayData."); /* 076 */ } /* 077 */ /* 078 */ byte[] project_array = new byte[(int)project_size]; /* 079 */ UnsafeArrayData project_arrayData = new UnsafeArrayData(); /* 080 */ Platform.putLong(project_array, 16, project_numElements); /* 081 */ project_arrayData.pointTo(project_array, 16, (int)project_size); /* 082 */ int project_counter = 0; /* 083 */ for (int y = 0; y < 2; y++) { /* 084 */ for (int z = 0; z < args[y].numElements(); z++) { /* 085 */ if (args[y].isNullAt(z)) { /* 086 */ project_arrayData.setNullAt(project_counter); /* 087 */ } else { /* 088 */ project_arrayData.setInt( /* 089 */ project_counter, /* 090 */ args[y].getInt(z) /* 091 */ ); /* 092 */ } /* 093 */ project_counter++; /* 094 */ } /* 095 */ } /* 096 */ return project_arrayData; /* 097 */ } /* 098 */ }.concat(project_args); /* 099 */ boolean project_isNull = project_value == null; ``` ### Non-primitive-type elements ``` val df = Seq( (Seq("aa" ,"bb"), Seq("ccc", "ddd")), (Seq("x", "y"), null) ).toDF("a", "b") df.filter('a.isNotNull).select(concat('a, 'b)).debugCodegen() ``` Result: ``` /* 033 */ boolean inputadapter_isNull = inputadapter_row.isNullAt(0); /* 034 */ ArrayData inputadapter_value = inputadapter_isNull ? /* 035 */ null : (inputadapter_row.getArray(0)); /* 036 */ /* 037 */ if (!(!inputadapter_isNull)) continue; /* 038 */ /* 039 */ ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1); /* 040 */ /* 041 */ ArrayData[] project_args = new ArrayData[2]; /* 042 */ /* 043 */ if (!false) { /* 044 */ project_args[0] = inputadapter_value; /* 045 */ } /* 046 */ /* 047 */ boolean inputadapter_isNull1 = inputadapter_row.isNullAt(1); /* 048 */ ArrayData inputadapter_value1 = inputadapter_isNull1 ? /* 049 */ null : (inputadapter_row.getArray(1)); /* 050 */ if (!inputadapter_isNull1) { /* 051 */ project_args[1] = inputadapter_value1; /* 052 */ } /* 053 */ /* 054 */ ArrayData project_value = new Object() { /* 055 */ public ArrayData concat(ArrayData[] args) { /* 056 */ for (int z = 0; z < 2; z++) { /* 057 */ if (args[z] == null) return null; /* 058 */ } /* 059 */ /* 060 */ long project_numElements = 0L; /* 061 */ for (int z = 0; z < 2; z++) { /* 062 */ project_numElements += args[z].numElements(); /* 063 */ } /* 064 */ if (project_numElements > 2147483632) { /* 065 */ throw new RuntimeException("Unsuccessful try to concat arrays with " + project_numElements + /* 066 */ " elements due to exceeding the array size limit 2147483632."); /* 067 */ } /* 068 */ /* 069 */ Object[] project_arrayObjects = new Object[(int)project_numElements]; /* 070 */ int project_counter = 0; /* 071 */ for (int y = 0; y < 2; y++) { /* 072 */ for (int z = 0; z < args[y].numElements(); z++) { /* 073 */ project_arrayObjects[project_counter] = args[y].getUTF8String(z); /* 074 */ project_counter++; /* 075 */ } /* 076 */ } /* 077 */ return new org.apache.spark.sql.catalyst.util.GenericArrayData(project_arrayObjects); /* 078 */ } /* 079 */ }.concat(project_args); /* 080 */ boolean project_isNull = project_value == null; ``` Author: mn-mikke <mrkAha12346github> Closes #20858 from mn-mikke/feature/array-api-concat_arrays-to-master.
2018-04-20 01:58:11 -04:00
@since(1.5)
@ignore_unicode_prefix
def concat(*cols):
"""
Concatenates multiple input columns together into a single column.
The function works with strings, binary and compatible array columns.
>>> df = spark.createDataFrame([('abcd','123')], ['s', 'd'])
>>> df.select(concat(df.s, df.d).alias('s')).collect()
[Row(s=u'abcd123')]
>>> df = spark.createDataFrame([([1, 2], [3, 4], [5]), ([1, 2], None, [3])], ['a', 'b', 'c'])
>>> df.select(concat(df.a, df.b, df.c).alias("arr")).collect()
[Row(arr=[1, 2, 3, 4, 5]), Row(arr=None)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.concat(_to_seq(sc, cols, _to_java_column)))
@since(2.4)
def array_position(col, value):
"""
Collection function: Locates the position of the first occurrence of the given value
in the given array. Returns null if either of the arguments are null.
.. note:: The position is not zero based, but 1 based index. Returns 0 if the given
value could not be found in the array.
>>> df = spark.createDataFrame([(["c", "b", "a"],), ([],)], ['data'])
>>> df.select(array_position(df.data, "a")).collect()
[Row(array_position(data, a)=3), Row(array_position(data, a)=0)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.array_position(_to_java_column(col), value))
@ignore_unicode_prefix
@since(2.4)
def element_at(col, extraction):
"""
Collection function: Returns element of array at given index in extraction if col is array.
Returns value for the given key in extraction if col is map.
:param col: name of column containing array or map
:param extraction: index to check for in array or key to check for in map
.. note:: The position is not zero based, but 1 based index.
>>> df = spark.createDataFrame([(["a", "b", "c"],), ([],)], ['data'])
>>> df.select(element_at(df.data, 1)).collect()
[Row(element_at(data, 1)=u'a'), Row(element_at(data, 1)=None)]
>>> df = spark.createDataFrame([({"a": 1.0, "b": 2.0},), ({},)], ['data'])
>>> df.select(element_at(df.data, "a")).collect()
[Row(element_at(data, a)=1.0), Row(element_at(data, a)=None)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.element_at(_to_java_column(col), extraction))
@since(2.4)
def array_remove(col, element):
"""
Collection function: Remove all elements that equal to element from the given array.
:param col: name of column containing array
:param element: element to be removed from the array
>>> df = spark.createDataFrame([([1, 2, 3, 1, 1],), ([],)], ['data'])
>>> df.select(array_remove(df.data, 1)).collect()
[Row(array_remove(data, 1)=[2, 3]), Row(array_remove(data, 1)=[])]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.array_remove(_to_java_column(col), element))
@since(2.4)
def array_distinct(col):
"""
Collection function: removes duplicate values from the array.
:param col: name of column or expression
>>> df = spark.createDataFrame([([1, 2, 3, 2],), ([4, 5, 5, 4],)], ['data'])
>>> df.select(array_distinct(df.data)).collect()
[Row(array_distinct(data)=[1, 2, 3]), Row(array_distinct(data)=[4, 5])]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.array_distinct(_to_java_column(col)))
@ignore_unicode_prefix
@since(2.4)
def array_intersect(col1, col2):
"""
Collection function: returns an array of the elements in the intersection of col1 and col2,
without duplicates.
:param col1: name of column containing array
:param col2: name of column containing array
>>> from pyspark.sql import Row
>>> df = spark.createDataFrame([Row(c1=["b", "a", "c"], c2=["c", "d", "a", "f"])])
>>> df.select(array_intersect(df.c1, df.c2)).collect()
[Row(array_intersect(c1, c2)=[u'a', u'c'])]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.array_intersect(_to_java_column(col1), _to_java_column(col2)))
@ignore_unicode_prefix
@since(2.4)
def array_union(col1, col2):
"""
Collection function: returns an array of the elements in the union of col1 and col2,
without duplicates.
:param col1: name of column containing array
:param col2: name of column containing array
>>> from pyspark.sql import Row
>>> df = spark.createDataFrame([Row(c1=["b", "a", "c"], c2=["c", "d", "a", "f"])])
>>> df.select(array_union(df.c1, df.c2)).collect()
[Row(array_union(c1, c2)=[u'b', u'a', u'c', u'd', u'f'])]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.array_union(_to_java_column(col1), _to_java_column(col2)))
@ignore_unicode_prefix
@since(2.4)
def array_except(col1, col2):
"""
Collection function: returns an array of the elements in col1 but not in col2,
without duplicates.
:param col1: name of column containing array
:param col2: name of column containing array
>>> from pyspark.sql import Row
>>> df = spark.createDataFrame([Row(c1=["b", "a", "c"], c2=["c", "d", "a", "f"])])
>>> df.select(array_except(df.c1, df.c2)).collect()
[Row(array_except(c1, c2)=[u'b'])]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.array_except(_to_java_column(col1), _to_java_column(col2)))
@since(1.4)
def explode(col):
"""
Returns a new row for each element in the given array or map.
Uses the default column name `col` for elements in the array and
`key` and `value` for elements in the map unless specified otherwise.
>>> from pyspark.sql import Row
>>> eDF = spark.createDataFrame([Row(a=1, intlist=[1,2,3], mapfield={"a": "b"})])
>>> eDF.select(explode(eDF.intlist).alias("anInt")).collect()
[Row(anInt=1), Row(anInt=2), Row(anInt=3)]
>>> eDF.select(explode(eDF.mapfield).alias("key", "value")).show()
+---+-----+
|key|value|
+---+-----+
| a| b|
+---+-----+
"""
sc = SparkContext._active_spark_context
jc = sc._jvm.functions.explode(_to_java_column(col))
return Column(jc)
@since(2.1)
def posexplode(col):
"""
Returns a new row for each element with position in the given array or map.
Uses the default column name `pos` for position, and `col` for elements in the
array and `key` and `value` for elements in the map unless specified otherwise.
>>> from pyspark.sql import Row
>>> eDF = spark.createDataFrame([Row(a=1, intlist=[1,2,3], mapfield={"a": "b"})])
>>> eDF.select(posexplode(eDF.intlist)).collect()
[Row(pos=0, col=1), Row(pos=1, col=2), Row(pos=2, col=3)]
>>> eDF.select(posexplode(eDF.mapfield)).show()
+---+---+-----+
|pos|key|value|
+---+---+-----+
| 0| a| b|
+---+---+-----+
"""
sc = SparkContext._active_spark_context
jc = sc._jvm.functions.posexplode(_to_java_column(col))
return Column(jc)
@since(2.3)
def explode_outer(col):
"""
Returns a new row for each element in the given array or map.
Unlike explode, if the array/map is null or empty then null is produced.
Uses the default column name `col` for elements in the array and
`key` and `value` for elements in the map unless specified otherwise.
>>> df = spark.createDataFrame(
... [(1, ["foo", "bar"], {"x": 1.0}), (2, [], {}), (3, None, None)],
... ("id", "an_array", "a_map")
... )
>>> df.select("id", "an_array", explode_outer("a_map")).show()
+---+----------+----+-----+
| id| an_array| key|value|
+---+----------+----+-----+
| 1|[foo, bar]| x| 1.0|
| 2| []|null| null|
| 3| null|null| null|
+---+----------+----+-----+
>>> df.select("id", "a_map", explode_outer("an_array")).show()
+---+----------+----+
| id| a_map| col|
+---+----------+----+
| 1|[x -> 1.0]| foo|
| 1|[x -> 1.0]| bar|
| 2| []|null|
| 3| null|null|
+---+----------+----+
"""
sc = SparkContext._active_spark_context
jc = sc._jvm.functions.explode_outer(_to_java_column(col))
return Column(jc)
@since(2.3)
def posexplode_outer(col):
"""
Returns a new row for each element with position in the given array or map.
Unlike posexplode, if the array/map is null or empty then the row (null, null) is produced.
Uses the default column name `pos` for position, and `col` for elements in the
array and `key` and `value` for elements in the map unless specified otherwise.
>>> df = spark.createDataFrame(
... [(1, ["foo", "bar"], {"x": 1.0}), (2, [], {}), (3, None, None)],
... ("id", "an_array", "a_map")
... )
>>> df.select("id", "an_array", posexplode_outer("a_map")).show()
+---+----------+----+----+-----+
| id| an_array| pos| key|value|
+---+----------+----+----+-----+
| 1|[foo, bar]| 0| x| 1.0|
| 2| []|null|null| null|
| 3| null|null|null| null|
+---+----------+----+----+-----+
>>> df.select("id", "a_map", posexplode_outer("an_array")).show()
+---+----------+----+----+
| id| a_map| pos| col|
+---+----------+----+----+
| 1|[x -> 1.0]| 0| foo|
| 1|[x -> 1.0]| 1| bar|
| 2| []|null|null|
| 3| null|null|null|
+---+----------+----+----+
"""
sc = SparkContext._active_spark_context
jc = sc._jvm.functions.posexplode_outer(_to_java_column(col))
return Column(jc)
@ignore_unicode_prefix
@since(1.6)
def get_json_object(col, path):
"""
Extracts json object from a json string based on json path specified, and returns json string
of the extracted json object. It will return null if the input json string is invalid.
:param col: string column in json format
:param path: path to the json object to extract
>>> data = [("1", '''{"f1": "value1", "f2": "value2"}'''), ("2", '''{"f1": "value12"}''')]
>>> df = spark.createDataFrame(data, ("key", "jstring"))
>>> df.select(df.key, get_json_object(df.jstring, '$.f1').alias("c0"), \\
... get_json_object(df.jstring, '$.f2').alias("c1") ).collect()
[Row(key=u'1', c0=u'value1', c1=u'value2'), Row(key=u'2', c0=u'value12', c1=None)]
"""
sc = SparkContext._active_spark_context
jc = sc._jvm.functions.get_json_object(_to_java_column(col), path)
return Column(jc)
@ignore_unicode_prefix
@since(1.6)
def json_tuple(col, *fields):
"""Creates a new row for a json column according to the given field names.
:param col: string column in json format
:param fields: list of fields to extract
>>> data = [("1", '''{"f1": "value1", "f2": "value2"}'''), ("2", '''{"f1": "value12"}''')]
>>> df = spark.createDataFrame(data, ("key", "jstring"))
>>> df.select(df.key, json_tuple(df.jstring, 'f1', 'f2')).collect()
[Row(key=u'1', c0=u'value1', c1=u'value2'), Row(key=u'2', c0=u'value12', c1=None)]
"""
sc = SparkContext._active_spark_context
jc = sc._jvm.functions.json_tuple(_to_java_column(col), _to_seq(sc, fields))
return Column(jc)
@ignore_unicode_prefix
@since(2.1)
def from_json(col, schema, options={}):
"""
Parses a column containing a JSON string into a :class:`MapType` with :class:`StringType`
as keys type, :class:`StructType` or :class:`ArrayType` with
the specified schema. Returns `null`, in the case of an unparseable string.
:param col: string column in json format
:param schema: a StructType or ArrayType of StructType to use when parsing the json column.
:param options: options to control parsing. accepts the same options as the json datasource
.. note:: Since Spark 2.3, the DDL-formatted string or a JSON format string is also
supported for ``schema``.
>>> from pyspark.sql.types import *
>>> data = [(1, '''{"a": 1}''')]
>>> schema = StructType([StructField("a", IntegerType())])
>>> df = spark.createDataFrame(data, ("key", "value"))
>>> df.select(from_json(df.value, schema).alias("json")).collect()
[Row(json=Row(a=1))]
>>> df.select(from_json(df.value, "a INT").alias("json")).collect()
[Row(json=Row(a=1))]
>>> df.select(from_json(df.value, "MAP<STRING,INT>").alias("json")).collect()
[Row(json={u'a': 1})]
[SPARK-19595][SQL] Support json array in from_json ## What changes were proposed in this pull request? This PR proposes to both, **Do not allow json arrays with multiple elements and return null in `from_json` with `StructType` as the schema.** Currently, it only reads the single row when the input is a json array. So, the codes below: ```scala import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ val schema = StructType(StructField("a", IntegerType) :: Nil) Seq(("""[{"a": 1}, {"a": 2}]""")).toDF("struct").select(from_json(col("struct"), schema)).show() ``` prints ``` +--------------------+ |jsontostruct(struct)| +--------------------+ | [1]| +--------------------+ ``` This PR simply suggests to print this as `null` if the schema is `StructType` and input is json array.with multiple elements ``` +--------------------+ |jsontostruct(struct)| +--------------------+ | null| +--------------------+ ``` **Support json arrays in `from_json` with `ArrayType` as the schema.** ```scala import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ val schema = ArrayType(StructType(StructField("a", IntegerType) :: Nil)) Seq(("""[{"a": 1}, {"a": 2}]""")).toDF("array").select(from_json(col("array"), schema)).show() ``` prints ``` +-------------------+ |jsontostruct(array)| +-------------------+ | [[1], [2]]| +-------------------+ ``` ## How was this patch tested? Unit test in `JsonExpressionsSuite`, `JsonFunctionsSuite`, Python doctests and manual test. Author: hyukjinkwon <gurwls223@gmail.com> Closes #16929 from HyukjinKwon/disallow-array.
2017-03-05 17:35:06 -05:00
>>> data = [(1, '''[{"a": 1}]''')]
>>> schema = ArrayType(StructType([StructField("a", IntegerType())]))
>>> df = spark.createDataFrame(data, ("key", "value"))
>>> df.select(from_json(df.value, schema).alias("json")).collect()
[Row(json=[Row(a=1)])]
>>> schema = schema_of_json(lit('''{"a": 0}'''))
>>> df.select(from_json(df.value, schema).alias("json")).collect()
[Row(json=Row(a=None))]
>>> data = [(1, '''[1, 2, 3]''')]
>>> schema = ArrayType(IntegerType())
>>> df = spark.createDataFrame(data, ("key", "value"))
>>> df.select(from_json(df.value, schema).alias("json")).collect()
[Row(json=[1, 2, 3])]
"""
sc = SparkContext._active_spark_context
if isinstance(schema, DataType):
schema = schema.json()
elif isinstance(schema, Column):
schema = _to_java_column(schema)
jc = sc._jvm.functions.from_json(_to_java_column(col), schema, options)
return Column(jc)
@ignore_unicode_prefix
@since(2.1)
def to_json(col, options={}):
"""
Converts a column containing a :class:`StructType`, :class:`ArrayType` or a :class:`MapType`
into a JSON string. Throws an exception, in the case of an unsupported type.
:param col: name of column containing a struct, an array or a map.
:param options: options to control converting. accepts the same options as the JSON datasource.
Additionally the function supports the `pretty` option which enables
pretty JSON generation.
>>> from pyspark.sql import Row
>>> from pyspark.sql.types import *
>>> data = [(1, Row(name='Alice', age=2))]
>>> df = spark.createDataFrame(data, ("key", "value"))
>>> df.select(to_json(df.value).alias("json")).collect()
[Row(json=u'{"age":2,"name":"Alice"}')]
[SPARK-19849][SQL] Support ArrayType in to_json to produce JSON array ## What changes were proposed in this pull request? This PR proposes to support an array of struct type in `to_json` as below: ```scala import org.apache.spark.sql.functions._ val df = Seq(Tuple1(Tuple1(1) :: Nil)).toDF("a") df.select(to_json($"a").as("json")).show() ``` ``` +----------+ | json| +----------+ |[{"_1":1}]| +----------+ ``` Currently, it throws an exception as below (a newline manually inserted for readability): ``` org.apache.spark.sql.AnalysisException: cannot resolve 'structtojson(`array`)' due to data type mismatch: structtojson requires that the expression is a struct expression.;; ``` This allows the roundtrip with `from_json` as below: ```scala import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ val schema = ArrayType(StructType(StructField("a", IntegerType) :: Nil)) val df = Seq("""[{"a":1}, {"a":2}]""").toDF("json").select(from_json($"json", schema).as("array")) df.show() // Read back. df.select(to_json($"array").as("json")).show() ``` ``` +----------+ | array| +----------+ |[[1], [2]]| +----------+ +-----------------+ | json| +-----------------+ |[{"a":1},{"a":2}]| +-----------------+ ``` Also, this PR proposes to rename from `StructToJson` to `StructsToJson ` and `JsonToStruct` to `JsonToStructs`. ## How was this patch tested? Unit tests in `JsonFunctionsSuite` and `JsonExpressionsSuite` for Scala, doctest for Python and test in `test_sparkSQL.R` for R. Author: hyukjinkwon <gurwls223@gmail.com> Closes #17192 from HyukjinKwon/SPARK-19849.
2017-03-20 01:33:01 -04:00
>>> data = [(1, [Row(name='Alice', age=2), Row(name='Bob', age=3)])]
>>> df = spark.createDataFrame(data, ("key", "value"))
>>> df.select(to_json(df.value).alias("json")).collect()
[Row(json=u'[{"age":2,"name":"Alice"},{"age":3,"name":"Bob"}]')]
>>> data = [(1, {"name": "Alice"})]
>>> df = spark.createDataFrame(data, ("key", "value"))
>>> df.select(to_json(df.value).alias("json")).collect()
[Row(json=u'{"name":"Alice"}')]
>>> data = [(1, [{"name": "Alice"}, {"name": "Bob"}])]
>>> df = spark.createDataFrame(data, ("key", "value"))
>>> df.select(to_json(df.value).alias("json")).collect()
[Row(json=u'[{"name":"Alice"},{"name":"Bob"}]')]
>>> data = [(1, ["Alice", "Bob"])]
>>> df = spark.createDataFrame(data, ("key", "value"))
>>> df.select(to_json(df.value).alias("json")).collect()
[Row(json=u'["Alice","Bob"]')]
"""
sc = SparkContext._active_spark_context
jc = sc._jvm.functions.to_json(_to_java_column(col), options)
return Column(jc)
@ignore_unicode_prefix
@since(2.4)
def schema_of_json(json, options={}):
"""
Parses a JSON string and infers its schema in DDL format.
:param json: a JSON string or a string literal containing a JSON string.
:param options: options to control parsing. accepts the same options as the JSON datasource
.. versionchanged:: 3.0
It accepts `options` parameter to control schema inferring.
>>> df = spark.range(1)
>>> df.select(schema_of_json(lit('{"a": 0}')).alias("json")).collect()
[Row(json=u'struct<a:bigint>')]
>>> schema = schema_of_json('{a: 1}', {'allowUnquotedFieldNames':'true'})
>>> df.select(schema.alias("json")).collect()
[Row(json=u'struct<a:bigint>')]
"""
if isinstance(json, basestring):
col = _create_column_from_literal(json)
elif isinstance(json, Column):
col = _to_java_column(json)
else:
raise TypeError("schema argument should be a column or string")
sc = SparkContext._active_spark_context
jc = sc._jvm.functions.schema_of_json(col, options)
return Column(jc)
@ignore_unicode_prefix
@since(3.0)
def schema_of_csv(csv, options={}):
"""
Parses a CSV string and infers its schema in DDL format.
:param col: a CSV string or a string literal containing a CSV string.
:param options: options to control parsing. accepts the same options as the CSV datasource
>>> df = spark.range(1)
>>> df.select(schema_of_csv(lit('1|a'), {'sep':'|'}).alias("csv")).collect()
[Row(csv=u'struct<_c0:int,_c1:string>')]
>>> df.select(schema_of_csv('1|a', {'sep':'|'}).alias("csv")).collect()
[Row(csv=u'struct<_c0:int,_c1:string>')]
"""
if isinstance(csv, basestring):
col = _create_column_from_literal(csv)
elif isinstance(csv, Column):
col = _to_java_column(csv)
else:
raise TypeError("schema argument should be a column or string")
sc = SparkContext._active_spark_context
jc = sc._jvm.functions.schema_of_csv(col, options)
return Column(jc)
@ignore_unicode_prefix
@since(3.0)
def to_csv(col, options={}):
"""
Converts a column containing a :class:`StructType` into a CSV string.
Throws an exception, in the case of an unsupported type.
:param col: name of column containing a struct.
:param options: options to control converting. accepts the same options as the CSV datasource.
>>> from pyspark.sql import Row
>>> data = [(1, Row(name='Alice', age=2))]
>>> df = spark.createDataFrame(data, ("key", "value"))
>>> df.select(to_csv(df.value).alias("csv")).collect()
[Row(csv=u'2,Alice')]
"""
sc = SparkContext._active_spark_context
jc = sc._jvm.functions.to_csv(_to_java_column(col), options)
return Column(jc)
[SPARK-8230][SQL] Add array/map size method Pull Request for: https://issues.apache.org/jira/browse/SPARK-8230 Primary issue resolved is to implement array/map size for Spark SQL. Code is ready for review by a committer. Chen Hao is on the JIRA ticket, but I don't know his username on github, rxin is also on JIRA ticket. Things to review: 1. Where to put added functions namespace wise, they seem to be part of a few operations on collections which includes `sort_array` and `array_contains`. Hence the name given `collectionOperations.scala` and `_collection_functions` in python. 2. In Python code, should it be in a `1.5.0` function array or in a collections array? 3. Are there any missing methods on the `Size` case class? Looks like many of these functions have generated Java code, is that also needed in this case? 4. Something else? Author: Pedro Rodriguez <ski.rodriguez@gmail.com> Author: Pedro Rodriguez <prodriguez@trulia.com> Closes #7462 from EntilZha/SPARK-8230 and squashes the following commits: 9a442ae [Pedro Rodriguez] fixed functions and sorted __all__ 9aea3bb [Pedro Rodriguez] removed imports from python docs 15d4bf1 [Pedro Rodriguez] Added null test case and changed to nullSafeCodeGen d88247c [Pedro Rodriguez] removed python code bd5f0e4 [Pedro Rodriguez] removed duplicate function from rebase/merge 59931b4 [Pedro Rodriguez] fixed compile bug instroduced when merging c187175 [Pedro Rodriguez] updated code to add size to __all__ directly and removed redundent pretty print 130839f [Pedro Rodriguez] fixed failing test aa9bade [Pedro Rodriguez] fix style e093473 [Pedro Rodriguez] updated python code with docs, switched classes/traits implemented, added (failing) expression tests 0449377 [Pedro Rodriguez] refactored code to use better abstract classes/traits and implementations 9a1a2ff [Pedro Rodriguez] added unit tests for map size 2bfbcb6 [Pedro Rodriguez] added unit test for size 20df2b4 [Pedro Rodriguez] Finished working version of size function and added it to python b503e75 [Pedro Rodriguez] First attempt at implementing size for maps and arrays 99a6a5c [Pedro Rodriguez] fixed failing test cac75ac [Pedro Rodriguez] fix style 933d843 [Pedro Rodriguez] updated python code with docs, switched classes/traits implemented, added (failing) expression tests 42bb7d4 [Pedro Rodriguez] refactored code to use better abstract classes/traits and implementations f9c3b8a [Pedro Rodriguez] added unit tests for map size 2515d9f [Pedro Rodriguez] added documentation 0e60541 [Pedro Rodriguez] added unit test for size acf9853 [Pedro Rodriguez] Finished working version of size function and added it to python 84a5d38 [Pedro Rodriguez] First attempt at implementing size for maps and arrays
2015-07-21 03:53:20 -04:00
@since(1.5)
def size(col):
"""
Collection function: returns the length of the array or map stored in the column.
[SPARK-8230][SQL] Add array/map size method Pull Request for: https://issues.apache.org/jira/browse/SPARK-8230 Primary issue resolved is to implement array/map size for Spark SQL. Code is ready for review by a committer. Chen Hao is on the JIRA ticket, but I don't know his username on github, rxin is also on JIRA ticket. Things to review: 1. Where to put added functions namespace wise, they seem to be part of a few operations on collections which includes `sort_array` and `array_contains`. Hence the name given `collectionOperations.scala` and `_collection_functions` in python. 2. In Python code, should it be in a `1.5.0` function array or in a collections array? 3. Are there any missing methods on the `Size` case class? Looks like many of these functions have generated Java code, is that also needed in this case? 4. Something else? Author: Pedro Rodriguez <ski.rodriguez@gmail.com> Author: Pedro Rodriguez <prodriguez@trulia.com> Closes #7462 from EntilZha/SPARK-8230 and squashes the following commits: 9a442ae [Pedro Rodriguez] fixed functions and sorted __all__ 9aea3bb [Pedro Rodriguez] removed imports from python docs 15d4bf1 [Pedro Rodriguez] Added null test case and changed to nullSafeCodeGen d88247c [Pedro Rodriguez] removed python code bd5f0e4 [Pedro Rodriguez] removed duplicate function from rebase/merge 59931b4 [Pedro Rodriguez] fixed compile bug instroduced when merging c187175 [Pedro Rodriguez] updated code to add size to __all__ directly and removed redundent pretty print 130839f [Pedro Rodriguez] fixed failing test aa9bade [Pedro Rodriguez] fix style e093473 [Pedro Rodriguez] updated python code with docs, switched classes/traits implemented, added (failing) expression tests 0449377 [Pedro Rodriguez] refactored code to use better abstract classes/traits and implementations 9a1a2ff [Pedro Rodriguez] added unit tests for map size 2bfbcb6 [Pedro Rodriguez] added unit test for size 20df2b4 [Pedro Rodriguez] Finished working version of size function and added it to python b503e75 [Pedro Rodriguez] First attempt at implementing size for maps and arrays 99a6a5c [Pedro Rodriguez] fixed failing test cac75ac [Pedro Rodriguez] fix style 933d843 [Pedro Rodriguez] updated python code with docs, switched classes/traits implemented, added (failing) expression tests 42bb7d4 [Pedro Rodriguez] refactored code to use better abstract classes/traits and implementations f9c3b8a [Pedro Rodriguez] added unit tests for map size 2515d9f [Pedro Rodriguez] added documentation 0e60541 [Pedro Rodriguez] added unit test for size acf9853 [Pedro Rodriguez] Finished working version of size function and added it to python 84a5d38 [Pedro Rodriguez] First attempt at implementing size for maps and arrays
2015-07-21 03:53:20 -04:00
:param col: name of column or expression
>>> df = spark.createDataFrame([([1, 2, 3],),([1],),([],)], ['data'])
[SPARK-8230][SQL] Add array/map size method Pull Request for: https://issues.apache.org/jira/browse/SPARK-8230 Primary issue resolved is to implement array/map size for Spark SQL. Code is ready for review by a committer. Chen Hao is on the JIRA ticket, but I don't know his username on github, rxin is also on JIRA ticket. Things to review: 1. Where to put added functions namespace wise, they seem to be part of a few operations on collections which includes `sort_array` and `array_contains`. Hence the name given `collectionOperations.scala` and `_collection_functions` in python. 2. In Python code, should it be in a `1.5.0` function array or in a collections array? 3. Are there any missing methods on the `Size` case class? Looks like many of these functions have generated Java code, is that also needed in this case? 4. Something else? Author: Pedro Rodriguez <ski.rodriguez@gmail.com> Author: Pedro Rodriguez <prodriguez@trulia.com> Closes #7462 from EntilZha/SPARK-8230 and squashes the following commits: 9a442ae [Pedro Rodriguez] fixed functions and sorted __all__ 9aea3bb [Pedro Rodriguez] removed imports from python docs 15d4bf1 [Pedro Rodriguez] Added null test case and changed to nullSafeCodeGen d88247c [Pedro Rodriguez] removed python code bd5f0e4 [Pedro Rodriguez] removed duplicate function from rebase/merge 59931b4 [Pedro Rodriguez] fixed compile bug instroduced when merging c187175 [Pedro Rodriguez] updated code to add size to __all__ directly and removed redundent pretty print 130839f [Pedro Rodriguez] fixed failing test aa9bade [Pedro Rodriguez] fix style e093473 [Pedro Rodriguez] updated python code with docs, switched classes/traits implemented, added (failing) expression tests 0449377 [Pedro Rodriguez] refactored code to use better abstract classes/traits and implementations 9a1a2ff [Pedro Rodriguez] added unit tests for map size 2bfbcb6 [Pedro Rodriguez] added unit test for size 20df2b4 [Pedro Rodriguez] Finished working version of size function and added it to python b503e75 [Pedro Rodriguez] First attempt at implementing size for maps and arrays 99a6a5c [Pedro Rodriguez] fixed failing test cac75ac [Pedro Rodriguez] fix style 933d843 [Pedro Rodriguez] updated python code with docs, switched classes/traits implemented, added (failing) expression tests 42bb7d4 [Pedro Rodriguez] refactored code to use better abstract classes/traits and implementations f9c3b8a [Pedro Rodriguez] added unit tests for map size 2515d9f [Pedro Rodriguez] added documentation 0e60541 [Pedro Rodriguez] added unit test for size acf9853 [Pedro Rodriguez] Finished working version of size function and added it to python 84a5d38 [Pedro Rodriguez] First attempt at implementing size for maps and arrays
2015-07-21 03:53:20 -04:00
>>> df.select(size(df.data)).collect()
[Row(size(data)=3), Row(size(data)=1), Row(size(data)=0)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.size(_to_java_column(col)))
@since(2.4)
def array_min(col):
"""
Collection function: returns the minimum value of the array.
:param col: name of column or expression
>>> df = spark.createDataFrame([([2, 1, 3],), ([None, 10, -1],)], ['data'])
>>> df.select(array_min(df.data).alias('min')).collect()
[Row(min=1), Row(min=-1)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.array_min(_to_java_column(col)))
@since(2.4)
def array_max(col):
"""
Collection function: returns the maximum value of the array.
:param col: name of column or expression
>>> df = spark.createDataFrame([([2, 1, 3],), ([None, 10, -1],)], ['data'])
>>> df.select(array_max(df.data).alias('max')).collect()
[Row(max=3), Row(max=10)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.array_max(_to_java_column(col)))
@since(1.5)
def sort_array(col, asc=True):
"""
[MINOR][DOCUMENTATION] Fix some minor descriptions in functions consistently with expressions ## What changes were proposed in this pull request? This PR proposes to improve documentation and fix some descriptions equivalent to several minor fixes identified in https://github.com/apache/spark/pull/15677 Also, this suggests to change `Note:` and `NOTE:` to `.. note::` consistently with the others which marks up pretty. ## How was this patch tested? Jenkins tests and manually. For PySpark, `Note:` and `NOTE:` to `.. note::` make the document as below: **From** ![2016-11-04 6 53 35](https://cloud.githubusercontent.com/assets/6477701/20002648/42989922-a2c5-11e6-8a32-b73eda49e8c3.png) ![2016-11-04 6 53 45](https://cloud.githubusercontent.com/assets/6477701/20002650/429fb310-a2c5-11e6-926b-e030d7eb0185.png) ![2016-11-04 6 54 11](https://cloud.githubusercontent.com/assets/6477701/20002649/429d570a-a2c5-11e6-9e7e-44090f337e32.png) ![2016-11-04 6 53 51](https://cloud.githubusercontent.com/assets/6477701/20002647/4297fc74-a2c5-11e6-801a-b89fbcbfca44.png) ![2016-11-04 6 53 51](https://cloud.githubusercontent.com/assets/6477701/20002697/749f5780-a2c5-11e6-835f-022e1f2f82e3.png) **To** ![2016-11-04 7 03 48](https://cloud.githubusercontent.com/assets/6477701/20002659/4961b504-a2c5-11e6-9ee0-ef0751482f47.png) ![2016-11-04 7 04 03](https://cloud.githubusercontent.com/assets/6477701/20002660/49871d3a-a2c5-11e6-85ea-d9a5d11efeff.png) ![2016-11-04 7 04 28](https://cloud.githubusercontent.com/assets/6477701/20002662/498e0f14-a2c5-11e6-803d-c0c5aeda4153.png) ![2016-11-04 7 33 39](https://cloud.githubusercontent.com/assets/6477701/20002731/a76e30d2-a2c5-11e6-993b-0481b8342d6b.png) ![2016-11-04 7 33 39](https://cloud.githubusercontent.com/assets/6477701/20002731/a76e30d2-a2c5-11e6-993b-0481b8342d6b.png) Author: hyukjinkwon <gurwls223@gmail.com> Closes #15765 from HyukjinKwon/minor-function-doc.
2016-11-06 00:47:33 -04:00
Collection function: sorts the input array in ascending or descending order according
to the natural ordering of the array elements. Null elements will be placed at the beginning
of the returned array in ascending order or at the end of the returned array in descending
order.
:param col: name of column or expression
>>> df = spark.createDataFrame([([2, 1, None, 3],),([1],),([],)], ['data'])
>>> df.select(sort_array(df.data).alias('r')).collect()
[Row(r=[None, 1, 2, 3]), Row(r=[1]), Row(r=[])]
>>> df.select(sort_array(df.data, asc=False).alias('r')).collect()
[Row(r=[3, 2, 1, None]), Row(r=[1]), Row(r=[])]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.sort_array(_to_java_column(col), asc))
@since(2.4)
def array_sort(col):
"""
Collection function: sorts the input array in ascending order. The elements of the input array
must be orderable. Null elements will be placed at the end of the returned array.
:param col: name of column or expression
>>> df = spark.createDataFrame([([2, 1, None, 3],),([1],),([],)], ['data'])
>>> df.select(array_sort(df.data).alias('r')).collect()
[Row(r=[1, 2, 3, None]), Row(r=[1]), Row(r=[])]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.array_sort(_to_java_column(col)))
@since(2.4)
def shuffle(col):
"""
Collection function: Generates a random permutation of the given array.
.. note:: The function is non-deterministic.
:param col: name of column or expression
>>> df = spark.createDataFrame([([1, 20, 3, 5],), ([1, 20, None, 3],)], ['data'])
>>> df.select(shuffle(df.data).alias('s')).collect() # doctest: +SKIP
[Row(s=[3, 1, 5, 20]), Row(s=[20, None, 3, 1])]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.shuffle(_to_java_column(col)))
[SPARK-23926][SQL] Extending reverse function to support ArrayType arguments ## What changes were proposed in this pull request? This PR extends `reverse` functions to be able to operate over array columns and covers: - Introduction of `Reverse` expression that represents logic for reversing arrays and also strings - Removal of `StringReverse` expression - A wrapper for PySpark ## How was this patch tested? New tests added into: - CollectionExpressionsSuite - DataFrameFunctionsSuite ## Codegen examples ### Primitive type ``` val df = Seq( Seq(1, 3, 4, 2), null ).toDF("i") df.filter($"i".isNotNull || $"i".isNull).select(reverse($"i")).debugCodegen ``` Result: ``` /* 032 */ boolean inputadapter_isNull = inputadapter_row.isNullAt(0); /* 033 */ ArrayData inputadapter_value = inputadapter_isNull ? /* 034 */ null : (inputadapter_row.getArray(0)); /* 035 */ /* 036 */ boolean filter_value = true; /* 037 */ /* 038 */ if (!(!inputadapter_isNull)) { /* 039 */ filter_value = inputadapter_isNull; /* 040 */ } /* 041 */ if (!filter_value) continue; /* 042 */ /* 043 */ ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1); /* 044 */ /* 045 */ boolean project_isNull = inputadapter_isNull; /* 046 */ ArrayData project_value = null; /* 047 */ /* 048 */ if (!inputadapter_isNull) { /* 049 */ final int project_length = inputadapter_value.numElements(); /* 050 */ project_value = inputadapter_value.copy(); /* 051 */ for(int k = 0; k < project_length / 2; k++) { /* 052 */ int l = project_length - k - 1; /* 053 */ boolean isNullAtK = project_value.isNullAt(k); /* 054 */ boolean isNullAtL = project_value.isNullAt(l); /* 055 */ if(!isNullAtK) { /* 056 */ int el = project_value.getInt(k); /* 057 */ if(!isNullAtL) { /* 058 */ project_value.setInt(k, project_value.getInt(l)); /* 059 */ } else { /* 060 */ project_value.setNullAt(k); /* 061 */ } /* 062 */ project_value.setInt(l, el); /* 063 */ } else if (!isNullAtL) { /* 064 */ project_value.setInt(k, project_value.getInt(l)); /* 065 */ project_value.setNullAt(l); /* 066 */ } /* 067 */ } /* 068 */ /* 069 */ } ``` ### Non-primitive type ``` val df = Seq( Seq("a", "c", "d", "b"), null ).toDF("s") df.filter($"s".isNotNull || $"s".isNull).select(reverse($"s")).debugCodegen ``` Result: ``` /* 032 */ boolean inputadapter_isNull = inputadapter_row.isNullAt(0); /* 033 */ ArrayData inputadapter_value = inputadapter_isNull ? /* 034 */ null : (inputadapter_row.getArray(0)); /* 035 */ /* 036 */ boolean filter_value = true; /* 037 */ /* 038 */ if (!(!inputadapter_isNull)) { /* 039 */ filter_value = inputadapter_isNull; /* 040 */ } /* 041 */ if (!filter_value) continue; /* 042 */ /* 043 */ ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1); /* 044 */ /* 045 */ boolean project_isNull = inputadapter_isNull; /* 046 */ ArrayData project_value = null; /* 047 */ /* 048 */ if (!inputadapter_isNull) { /* 049 */ final int project_length = inputadapter_value.numElements(); /* 050 */ project_value = new org.apache.spark.sql.catalyst.util.GenericArrayData(new Object[project_length]); /* 051 */ for(int k = 0; k < project_length; k++) { /* 052 */ int l = project_length - k - 1; /* 053 */ project_value.update(k, inputadapter_value.getUTF8String(l)); /* 054 */ } /* 055 */ /* 056 */ } ``` Author: mn-mikke <mrkAha12346github> Closes #21034 from mn-mikke/feature/array-api-reverse-to-master.
2018-04-18 05:41:55 -04:00
@since(1.5)
@ignore_unicode_prefix
def reverse(col):
"""
Collection function: returns a reversed string or an array with reverse order of elements.
:param col: name of column or expression
>>> df = spark.createDataFrame([('Spark SQL',)], ['data'])
>>> df.select(reverse(df.data).alias('s')).collect()
[Row(s=u'LQS krapS')]
>>> df = spark.createDataFrame([([2, 1, 3],) ,([1],) ,([],)], ['data'])
>>> df.select(reverse(df.data).alias('r')).collect()
[Row(r=[3, 1, 2]), Row(r=[1]), Row(r=[])]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.reverse(_to_java_column(col)))
[SPARK-23821][SQL] Collection function: flatten ## What changes were proposed in this pull request? This PR adds a new collection function that transforms an array of arrays into a single array. The PR comprises: - An expression for flattening array structure - Flatten function - A wrapper for PySpark ## How was this patch tested? New tests added into: - CollectionExpressionsSuite - DataFrameFunctionsSuite ## Codegen examples ### Primitive type ``` val df = Seq( Seq(Seq(1, 2), Seq(4, 5)), Seq(null, Seq(1)) ).toDF("i") df.filter($"i".isNotNull || $"i".isNull).select(flatten($"i")).debugCodegen ``` Result: ``` /* 033 */ boolean inputadapter_isNull = inputadapter_row.isNullAt(0); /* 034 */ ArrayData inputadapter_value = inputadapter_isNull ? /* 035 */ null : (inputadapter_row.getArray(0)); /* 036 */ /* 037 */ boolean filter_value = true; /* 038 */ /* 039 */ if (!(!inputadapter_isNull)) { /* 040 */ filter_value = inputadapter_isNull; /* 041 */ } /* 042 */ if (!filter_value) continue; /* 043 */ /* 044 */ ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1); /* 045 */ /* 046 */ boolean project_isNull = inputadapter_isNull; /* 047 */ ArrayData project_value = null; /* 048 */ /* 049 */ if (!inputadapter_isNull) { /* 050 */ for (int z = 0; !project_isNull && z < inputadapter_value.numElements(); z++) { /* 051 */ project_isNull |= inputadapter_value.isNullAt(z); /* 052 */ } /* 053 */ if (!project_isNull) { /* 054 */ long project_numElements = 0; /* 055 */ for (int z = 0; z < inputadapter_value.numElements(); z++) { /* 056 */ project_numElements += inputadapter_value.getArray(z).numElements(); /* 057 */ } /* 058 */ if (project_numElements > 2147483632) { /* 059 */ throw new RuntimeException("Unsuccessful try to flatten an array of arrays with " + /* 060 */ project_numElements + " elements due to exceeding the array size limit 2147483632."); /* 061 */ } /* 062 */ /* 063 */ long project_size = UnsafeArrayData.calculateSizeOfUnderlyingByteArray( /* 064 */ project_numElements, /* 065 */ 4); /* 066 */ if (project_size > 2147483632) { /* 067 */ throw new RuntimeException("Unsuccessful try to flatten an array of arrays with " + /* 068 */ project_size + " bytes of data due to exceeding the limit 2147483632" + /* 069 */ " bytes for UnsafeArrayData."); /* 070 */ } /* 071 */ /* 072 */ byte[] project_array = new byte[(int)project_size]; /* 073 */ UnsafeArrayData project_tempArrayData = new UnsafeArrayData(); /* 074 */ Platform.putLong(project_array, 16, project_numElements); /* 075 */ project_tempArrayData.pointTo(project_array, 16, (int)project_size); /* 076 */ int project_counter = 0; /* 077 */ for (int k = 0; k < inputadapter_value.numElements(); k++) { /* 078 */ ArrayData arr = inputadapter_value.getArray(k); /* 079 */ for (int l = 0; l < arr.numElements(); l++) { /* 080 */ if (arr.isNullAt(l)) { /* 081 */ project_tempArrayData.setNullAt(project_counter); /* 082 */ } else { /* 083 */ project_tempArrayData.setInt( /* 084 */ project_counter, /* 085 */ arr.getInt(l) /* 086 */ ); /* 087 */ } /* 088 */ project_counter++; /* 089 */ } /* 090 */ } /* 091 */ project_value = project_tempArrayData; /* 092 */ /* 093 */ } /* 094 */ /* 095 */ } ``` ### Non-primitive type ``` val df = Seq( Seq(Seq("a", "b"), Seq(null, "d")), Seq(null, Seq("a")) ).toDF("s") df.filter($"s".isNotNull || $"s".isNull).select(flatten($"s")).debugCodegen ``` Result: ``` /* 033 */ boolean inputadapter_isNull = inputadapter_row.isNullAt(0); /* 034 */ ArrayData inputadapter_value = inputadapter_isNull ? /* 035 */ null : (inputadapter_row.getArray(0)); /* 036 */ /* 037 */ boolean filter_value = true; /* 038 */ /* 039 */ if (!(!inputadapter_isNull)) { /* 040 */ filter_value = inputadapter_isNull; /* 041 */ } /* 042 */ if (!filter_value) continue; /* 043 */ /* 044 */ ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1); /* 045 */ /* 046 */ boolean project_isNull = inputadapter_isNull; /* 047 */ ArrayData project_value = null; /* 048 */ /* 049 */ if (!inputadapter_isNull) { /* 050 */ for (int z = 0; !project_isNull && z < inputadapter_value.numElements(); z++) { /* 051 */ project_isNull |= inputadapter_value.isNullAt(z); /* 052 */ } /* 053 */ if (!project_isNull) { /* 054 */ long project_numElements = 0; /* 055 */ for (int z = 0; z < inputadapter_value.numElements(); z++) { /* 056 */ project_numElements += inputadapter_value.getArray(z).numElements(); /* 057 */ } /* 058 */ if (project_numElements > 2147483632) { /* 059 */ throw new RuntimeException("Unsuccessful try to flatten an array of arrays with " + /* 060 */ project_numElements + " elements due to exceeding the array size limit 2147483632."); /* 061 */ } /* 062 */ /* 063 */ Object[] project_arrayObject = new Object[(int)project_numElements]; /* 064 */ int project_counter = 0; /* 065 */ for (int k = 0; k < inputadapter_value.numElements(); k++) { /* 066 */ ArrayData arr = inputadapter_value.getArray(k); /* 067 */ for (int l = 0; l < arr.numElements(); l++) { /* 068 */ project_arrayObject[project_counter] = arr.getUTF8String(l); /* 069 */ project_counter++; /* 070 */ } /* 071 */ } /* 072 */ project_value = new org.apache.spark.sql.catalyst.util.GenericArrayData(project_arrayObject); /* 073 */ /* 074 */ } /* 075 */ /* 076 */ } ``` Author: mn-mikke <mrkAha12346github> Closes #20938 from mn-mikke/feature/array-api-flatten-to-master.
2018-04-24 22:19:08 -04:00
@since(2.4)
def flatten(col):
"""
Collection function: creates a single array from an array of arrays.
If a structure of nested arrays is deeper than two levels,
only one level of nesting is removed.
:param col: name of column or expression
>>> df = spark.createDataFrame([([[1, 2, 3], [4, 5], [6]],), ([None, [4, 5]],)], ['data'])
>>> df.select(flatten(df.data).alias('r')).collect()
[Row(r=[1, 2, 3, 4, 5, 6]), Row(r=None)]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.flatten(_to_java_column(col)))
@since(2.3)
def map_keys(col):
"""
Collection function: Returns an unordered array containing the keys of the map.
:param col: name of column or expression
>>> from pyspark.sql.functions import map_keys
>>> df = spark.sql("SELECT map(1, 'a', 2, 'b') as data")
>>> df.select(map_keys("data").alias("keys")).show()
+------+
| keys|
+------+
|[1, 2]|
+------+
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.map_keys(_to_java_column(col)))
@since(2.3)
def map_values(col):
"""
Collection function: Returns an unordered array containing the values of the map.
:param col: name of column or expression
>>> from pyspark.sql.functions import map_values
>>> df = spark.sql("SELECT map(1, 'a', 2, 'b') as data")
>>> df.select(map_values("data").alias("values")).show()
+------+
|values|
+------+
|[a, b]|
+------+
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.map_values(_to_java_column(col)))
@since(3.0)
[SPARK-23935][SQL] Adding map_entries function ## What changes were proposed in this pull request? This PR adds `map_entries` function that returns an unordered array of all entries in the given map. ## How was this patch tested? New tests added into: - `CollectionExpressionSuite` - `DataFrameFunctionsSuite` ## CodeGen examples ### Primitive types ``` val df = Seq(Map(1 -> 5, 2 -> 6)).toDF("m") df.filter('m.isNotNull).select(map_entries('m)).debugCodegen ``` Result: ``` /* 042 */ boolean project_isNull_0 = false; /* 043 */ /* 044 */ ArrayData project_value_0 = null; /* 045 */ /* 046 */ final int project_numElements_0 = inputadapter_value_0.numElements(); /* 047 */ final ArrayData project_keys_0 = inputadapter_value_0.keyArray(); /* 048 */ final ArrayData project_values_0 = inputadapter_value_0.valueArray(); /* 049 */ /* 050 */ final long project_size_0 = UnsafeArrayData.calculateSizeOfUnderlyingByteArray( /* 051 */ project_numElements_0, /* 052 */ 32); /* 053 */ if (project_size_0 > 2147483632) { /* 054 */ final Object[] project_internalRowArray_0 = new Object[project_numElements_0]; /* 055 */ for (int z = 0; z < project_numElements_0; z++) { /* 056 */ project_internalRowArray_0[z] = new org.apache.spark.sql.catalyst.expressions.GenericInternalRow(new Object[]{project_keys_0.getInt(z), project_values_0.getInt(z)}); /* 057 */ } /* 058 */ project_value_0 = new org.apache.spark.sql.catalyst.util.GenericArrayData(project_internalRowArray_0); /* 059 */ /* 060 */ } else { /* 061 */ final byte[] project_arrayBytes_0 = new byte[(int)project_size_0]; /* 062 */ UnsafeArrayData project_unsafeArrayData_0 = new UnsafeArrayData(); /* 063 */ Platform.putLong(project_arrayBytes_0, 16, project_numElements_0); /* 064 */ project_unsafeArrayData_0.pointTo(project_arrayBytes_0, 16, (int)project_size_0); /* 065 */ /* 066 */ final int project_structsOffset_0 = UnsafeArrayData.calculateHeaderPortionInBytes(project_numElements_0) + project_numElements_0 * 8; /* 067 */ UnsafeRow project_unsafeRow_0 = new UnsafeRow(2); /* 068 */ for (int z = 0; z < project_numElements_0; z++) { /* 069 */ long offset = project_structsOffset_0 + z * 24L; /* 070 */ project_unsafeArrayData_0.setLong(z, (offset << 32) + 24L); /* 071 */ project_unsafeRow_0.pointTo(project_arrayBytes_0, 16 + offset, 24); /* 072 */ project_unsafeRow_0.setInt(0, project_keys_0.getInt(z)); /* 073 */ project_unsafeRow_0.setInt(1, project_values_0.getInt(z)); /* 074 */ } /* 075 */ project_value_0 = project_unsafeArrayData_0; /* 076 */ /* 077 */ } ``` ### Non-primitive types ``` val df = Seq(Map("a" -> "foo", "b" -> null)).toDF("m") df.filter('m.isNotNull).select(map_entries('m)).debugCodegen ``` Result: ``` /* 042 */ boolean project_isNull_0 = false; /* 043 */ /* 044 */ ArrayData project_value_0 = null; /* 045 */ /* 046 */ final int project_numElements_0 = inputadapter_value_0.numElements(); /* 047 */ final ArrayData project_keys_0 = inputadapter_value_0.keyArray(); /* 048 */ final ArrayData project_values_0 = inputadapter_value_0.valueArray(); /* 049 */ /* 050 */ final Object[] project_internalRowArray_0 = new Object[project_numElements_0]; /* 051 */ for (int z = 0; z < project_numElements_0; z++) { /* 052 */ project_internalRowArray_0[z] = new org.apache.spark.sql.catalyst.expressions.GenericInternalRow(new Object[]{project_keys_0.getUTF8String(z), project_values_0.getUTF8String(z)}); /* 053 */ } /* 054 */ project_value_0 = new org.apache.spark.sql.catalyst.util.GenericArrayData(project_internalRowArray_0); ``` Author: Marek Novotny <mn.mikke@gmail.com> Closes #21236 from mn-mikke/feature/array-api-map_entries-to-master.
2018-05-21 10:14:03 -04:00
def map_entries(col):
"""
Collection function: Returns an unordered array of all entries in the given map.
:param col: name of column or expression
>>> from pyspark.sql.functions import map_entries
>>> df = spark.sql("SELECT map(1, 'a', 2, 'b') as data")
>>> df.select(map_entries("data").alias("entries")).show()
+----------------+
| entries|
+----------------+
|[[1, a], [2, b]]|
+----------------+
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.map_entries(_to_java_column(col)))
[SPARK-23934][SQL] Adding map_from_entries function ## What changes were proposed in this pull request? The PR adds the `map_from_entries` function that returns a map created from the given array of entries. ## How was this patch tested? New tests added into: - `CollectionExpressionSuite` - `DataFrameFunctionSuite` ## CodeGen Examples ### Primitive-type Keys and Values ``` val idf = Seq( Seq((1, 10), (2, 20), (3, 10)), Seq((1, 10), null, (2, 20)) ).toDF("a") idf.filter('a.isNotNull).select(map_from_entries('a)).debugCodegen ``` Result: ``` /* 042 */ boolean project_isNull_0 = false; /* 043 */ MapData project_value_0 = null; /* 044 */ /* 045 */ for (int project_idx_2 = 0; !project_isNull_0 && project_idx_2 < inputadapter_value_0.numElements(); project_idx_2++) { /* 046 */ project_isNull_0 |= inputadapter_value_0.isNullAt(project_idx_2); /* 047 */ } /* 048 */ if (!project_isNull_0) { /* 049 */ final int project_numEntries_0 = inputadapter_value_0.numElements(); /* 050 */ /* 051 */ final long project_keySectionSize_0 = UnsafeArrayData.calculateSizeOfUnderlyingByteArray(project_numEntries_0, 4); /* 052 */ final long project_valueSectionSize_0 = UnsafeArrayData.calculateSizeOfUnderlyingByteArray(project_numEntries_0, 4); /* 053 */ final long project_byteArraySize_0 = 8 + project_keySectionSize_0 + project_valueSectionSize_0; /* 054 */ if (project_byteArraySize_0 > 2147483632) { /* 055 */ final Object[] project_keys_0 = new Object[project_numEntries_0]; /* 056 */ final Object[] project_values_0 = new Object[project_numEntries_0]; /* 057 */ /* 058 */ for (int project_idx_1 = 0; project_idx_1 < project_numEntries_0; project_idx_1++) { /* 059 */ InternalRow project_entry_1 = inputadapter_value_0.getStruct(project_idx_1, 2); /* 060 */ /* 061 */ project_keys_0[project_idx_1] = project_entry_1.getInt(0); /* 062 */ project_values_0[project_idx_1] = project_entry_1.getInt(1); /* 063 */ } /* 064 */ /* 065 */ project_value_0 = org.apache.spark.sql.catalyst.util.ArrayBasedMapData.apply(project_keys_0, project_values_0); /* 066 */ /* 067 */ } else { /* 068 */ final byte[] project_byteArray_0 = new byte[(int)project_byteArraySize_0]; /* 069 */ UnsafeMapData project_unsafeMapData_0 = new UnsafeMapData(); /* 070 */ Platform.putLong(project_byteArray_0, 16, project_keySectionSize_0); /* 071 */ Platform.putLong(project_byteArray_0, 24, project_numEntries_0); /* 072 */ Platform.putLong(project_byteArray_0, 24 + project_keySectionSize_0, project_numEntries_0); /* 073 */ project_unsafeMapData_0.pointTo(project_byteArray_0, 16, (int)project_byteArraySize_0); /* 074 */ ArrayData project_keyArrayData_0 = project_unsafeMapData_0.keyArray(); /* 075 */ ArrayData project_valueArrayData_0 = project_unsafeMapData_0.valueArray(); /* 076 */ /* 077 */ for (int project_idx_0 = 0; project_idx_0 < project_numEntries_0; project_idx_0++) { /* 078 */ InternalRow project_entry_0 = inputadapter_value_0.getStruct(project_idx_0, 2); /* 079 */ /* 080 */ project_keyArrayData_0.setInt(project_idx_0, project_entry_0.getInt(0)); /* 081 */ project_valueArrayData_0.setInt(project_idx_0, project_entry_0.getInt(1)); /* 082 */ } /* 083 */ /* 084 */ project_value_0 = project_unsafeMapData_0; /* 085 */ } /* 086 */ /* 087 */ } ``` ### Non-primitive-type Keys and Values ``` val sdf = Seq( Seq(("a", null), ("b", "bb"), ("c", "aa")), Seq(("a", "aa"), null, (null, "bb")) ).toDF("a") sdf.filter('a.isNotNull).select(map_from_entries('a)).debugCodegen ``` Result: ``` /* 042 */ boolean project_isNull_0 = false; /* 043 */ MapData project_value_0 = null; /* 044 */ /* 045 */ for (int project_idx_1 = 0; !project_isNull_0 && project_idx_1 < inputadapter_value_0.numElements(); project_idx_1++) { /* 046 */ project_isNull_0 |= inputadapter_value_0.isNullAt(project_idx_1); /* 047 */ } /* 048 */ if (!project_isNull_0) { /* 049 */ final int project_numEntries_0 = inputadapter_value_0.numElements(); /* 050 */ /* 051 */ final Object[] project_keys_0 = new Object[project_numEntries_0]; /* 052 */ final Object[] project_values_0 = new Object[project_numEntries_0]; /* 053 */ /* 054 */ for (int project_idx_0 = 0; project_idx_0 < project_numEntries_0; project_idx_0++) { /* 055 */ InternalRow project_entry_0 = inputadapter_value_0.getStruct(project_idx_0, 2); /* 056 */ /* 057 */ if (project_entry_0.isNullAt(0)) { /* 058 */ throw new RuntimeException("The first field from a struct (key) can't be null."); /* 059 */ } /* 060 */ /* 061 */ project_keys_0[project_idx_0] = project_entry_0.getUTF8String(0); /* 062 */ project_values_0[project_idx_0] = project_entry_0.getUTF8String(1); /* 063 */ } /* 064 */ /* 065 */ project_value_0 = org.apache.spark.sql.catalyst.util.ArrayBasedMapData.apply(project_keys_0, project_values_0); /* 066 */ /* 067 */ } ``` Author: Marek Novotny <mn.mikke@gmail.com> Closes #21282 from mn-mikke/feature/array-api-map_from_entries-to-master.
2018-06-22 03:18:22 -04:00
@since(2.4)
def map_from_entries(col):
"""
Collection function: Returns a map created from the given array of entries.
:param col: name of column or expression
>>> from pyspark.sql.functions import map_from_entries
>>> df = spark.sql("SELECT array(struct(1, 'a'), struct(2, 'b')) as data")
>>> df.select(map_from_entries("data").alias("map")).show()
+----------------+
| map|
+----------------+
|[1 -> a, 2 -> b]|
+----------------+
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.map_from_entries(_to_java_column(col)))
@ignore_unicode_prefix
@since(2.4)
def array_repeat(col, count):
"""
Collection function: creates an array containing a column repeated count times.
>>> df = spark.createDataFrame([('ab',)], ['data'])
>>> df.select(array_repeat(df.data, 3).alias('r')).collect()
[Row(r=[u'ab', u'ab', u'ab'])]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.array_repeat(_to_java_column(col), count))
@since(2.4)
def arrays_zip(*cols):
"""
Collection function: Returns a merged array of structs in which the N-th struct contains all
N-th values of input arrays.
:param cols: columns of arrays to be merged.
>>> from pyspark.sql.functions import arrays_zip
>>> df = spark.createDataFrame([(([1, 2, 3], [2, 3, 4]))], ['vals1', 'vals2'])
>>> df.select(arrays_zip(df.vals1, df.vals2).alias('zipped')).collect()
[Row(zipped=[Row(vals1=1, vals2=2), Row(vals1=2, vals2=3), Row(vals1=3, vals2=4)])]
"""
sc = SparkContext._active_spark_context
return Column(sc._jvm.functions.arrays_zip(_to_seq(sc, cols, _to_java_column)))
@since(2.4)
def map_concat(*cols):
"""Returns the union of all the given maps.
:param cols: list of column names (string) or list of :class:`Column` expressions
>>> from pyspark.sql.functions import map_concat
>>> df = spark.sql("SELECT map(1, 'a', 2, 'b') as map1, map(3, 'c', 1, 'd') as map2")
>>> df.select(map_concat("map1", "map2").alias("map3")).show(truncate=False)
[SPARK-25829][SQL] remove duplicated map keys with last wins policy ## What changes were proposed in this pull request? Currently duplicated map keys are not handled consistently. For example, map look up respects the duplicated key appears first, `Dataset.collect` only keeps the duplicated key appears last, `MapKeys` returns duplicated keys, etc. This PR proposes to remove duplicated map keys with last wins policy, to follow Java/Scala and Presto. It only applies to built-in functions, as users can create map with duplicated map keys via private APIs anyway. updated functions: `CreateMap`, `MapFromArrays`, `MapFromEntries`, `StringToMap`, `MapConcat`, `TransformKeys`. For other places: 1. data source v1 doesn't have this problem, as users need to provide a java/scala map, which can't have duplicated keys. 2. data source v2 may have this problem. I've added a note to `ArrayBasedMapData` to ask the caller to take care of duplicated keys. In the future we should enforce it in the stable data APIs for data source v2. 3. UDF doesn't have this problem, as users need to provide a java/scala map. Same as data source v1. 4. file format. I checked all of them and only parquet does not enforce it. For backward compatibility reasons I change nothing but leave a note saying that the behavior will be undefined if users write map with duplicated keys to parquet files. Maybe we can add a config and fail by default if parquet files have map with duplicated keys. This can be done in followup. ## How was this patch tested? updated tests and new tests Closes #23124 from cloud-fan/map. Authored-by: Wenchen Fan <wenchen@databricks.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-11-28 10:42:13 -05:00
+------------------------+
|map3 |
+------------------------+
|[1 -> d, 2 -> b, 3 -> c]|
+------------------------+
"""
sc = SparkContext._active_spark_context
if len(cols) == 1 and isinstance(cols[0], (list, set)):
cols = cols[0]
jc = sc._jvm.functions.map_concat(_to_seq(sc, cols, _to_java_column))
return Column(jc)
@since(2.4)
def sequence(start, stop, step=None):
"""
Generate a sequence of integers from `start` to `stop`, incrementing by `step`.
If `step` is not set, incrementing by 1 if `start` is less than or equal to `stop`,
otherwise -1.
>>> df1 = spark.createDataFrame([(-2, 2)], ('C1', 'C2'))
>>> df1.select(sequence('C1', 'C2').alias('r')).collect()
[Row(r=[-2, -1, 0, 1, 2])]
>>> df2 = spark.createDataFrame([(4, -4, -2)], ('C1', 'C2', 'C3'))
>>> df2.select(sequence('C1', 'C2', 'C3').alias('r')).collect()
[Row(r=[4, 2, 0, -2, -4])]
"""
sc = SparkContext._active_spark_context
if step is None:
return Column(sc._jvm.functions.sequence(_to_java_column(start), _to_java_column(stop)))
else:
return Column(sc._jvm.functions.sequence(
_to_java_column(start), _to_java_column(stop), _to_java_column(step)))
@ignore_unicode_prefix
@since(3.0)
def from_csv(col, schema, options={}):
"""
Parses a column containing a CSV string to a row with the specified schema.
Returns `null`, in the case of an unparseable string.
:param col: string column in CSV format
:param schema: a string with schema in DDL format to use when parsing the CSV column.
:param options: options to control parsing. accepts the same options as the CSV datasource
>>> data = [("1,2,3",)]
>>> df = spark.createDataFrame(data, ("value",))
>>> df.select(from_csv(df.value, "a INT, b INT, c INT").alias("csv")).collect()
[Row(csv=Row(a=1, b=2, c=3))]
>>> value = data[0][0]
>>> df.select(from_csv(df.value, schema_of_csv(value)).alias("csv")).collect()
[Row(csv=Row(_c0=1, _c1=2, _c2=3))]
"""
sc = SparkContext._active_spark_context
if isinstance(schema, basestring):
schema = _create_column_from_literal(schema)
elif isinstance(schema, Column):
schema = _to_java_column(schema)
else:
raise TypeError("schema argument should be a column or string")
jc = sc._jvm.functions.from_csv(_to_java_column(col), schema, options)
return Column(jc)
# ---------------------------- User Defined Function ----------------------------------
class PandasUDFType(object):
"""Pandas UDF Types. See :meth:`pyspark.sql.functions.pandas_udf`.
"""
SCALAR = PythonEvalType.SQL_SCALAR_PANDAS_UDF
[SPARK-26412][PYSPARK][SQL] Allow Pandas UDF to take an iterator of pd.Series or an iterator of tuple of pd.Series ## What changes were proposed in this pull request? Allow Pandas UDF to take an iterator of pd.Series or an iterator of tuple of pd.Series. Note the UDF input args will be always one iterator: * if the udf take only column as input, the iterator's element will be pd.Series (corresponding to the column values batch) * if the udf take multiple columns as inputs, the iterator's element will be a tuple composed of multiple `pd.Series`s, each one corresponding to the multiple columns as inputs (keep the same order). For example: ``` pandas_udf("int", PandasUDFType.SCALAR_ITER) def the_udf(iterator): for col1_batch, col2_batch in iterator: yield col1_batch + col2_batch df.select(the_udf("col1", "col2")) ``` The udf above will add col1 and col2. I haven't add unit tests, but manually tests show it works fine. So it is ready for first pass review. We can test several typical cases: ``` from pyspark.sql import SparkSession from pyspark.sql.functions import pandas_udf, PandasUDFType from pyspark.sql.functions import udf from pyspark.taskcontext import TaskContext df = spark.createDataFrame([(1, 20), (3, 40)], ["a", "b"]) pandas_udf("int", PandasUDFType.SCALAR_ITER) def fi1(it): pid = TaskContext.get().partitionId() print("DBG: fi1: do init stuff, partitionId=" + str(pid)) for batch in it: yield batch + 100 print("DBG: fi1: do close stuff, partitionId=" + str(pid)) pandas_udf("int", PandasUDFType.SCALAR_ITER) def fi2(it): pid = TaskContext.get().partitionId() print("DBG: fi2: do init stuff, partitionId=" + str(pid)) for batch in it: yield batch + 10000 print("DBG: fi2: do close stuff, partitionId=" + str(pid)) pandas_udf("int", PandasUDFType.SCALAR_ITER) def fi3(it): pid = TaskContext.get().partitionId() print("DBG: fi3: do init stuff, partitionId=" + str(pid)) for x, y in it: yield x + y * 10 + 100000 print("DBG: fi3: do close stuff, partitionId=" + str(pid)) pandas_udf("int", PandasUDFType.SCALAR) def fp1(x): return x + 1000 udf("int") def fu1(x): return x + 10 # test select "pandas iter udf/pandas udf/sql udf" expressions at the same time. # Note this case the `fi1("a"), fi2("b"), fi3("a", "b")` will generate only one plan, # and `fu1("a")`, `fp1("a")` will generate another two separate plans. df.select(fi1("a"), fi2("b"), fi3("a", "b"), fu1("a"), fp1("a")).show() # test chain two pandas iter udf together # Note this case `fi2(fi1("a"))` will generate only one plan # Also note the init stuff/close stuff call order will be like: # (debug output following) # DBG: fi2: do init stuff, partitionId=0 # DBG: fi1: do init stuff, partitionId=0 # DBG: fi1: do close stuff, partitionId=0 # DBG: fi2: do close stuff, partitionId=0 df.select(fi2(fi1("a"))).show() # test more complex chain # Note this case `fi1("a"), fi2("a")` will generate one plan, # and `fi3(fi1_output, fi2_output)` will generate another plan df.select(fi3(fi1("a"), fi2("a"))).show() ``` ## How was this patch tested? To be added. Please review http://spark.apache.org/contributing.html before opening a pull request. Closes #24643 from WeichenXu123/pandas_udf_iter. Lead-authored-by: WeichenXu <weichen.xu@databricks.com> Co-authored-by: Xiangrui Meng <meng@databricks.com> Signed-off-by: Xiangrui Meng <meng@databricks.com>
2019-06-15 11:29:20 -04:00
SCALAR_ITER = PythonEvalType.SQL_SCALAR_PANDAS_ITER_UDF
GROUPED_MAP = PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF
GROUPED_AGG = PythonEvalType.SQL_GROUPED_AGG_PANDAS_UDF
@since(1.3)
def udf(f=None, returnType=StringType()):
2017-10-10 18:32:01 -04:00
"""Creates a user defined function (UDF).
.. note:: The user-defined functions are considered deterministic by default. Due to
optimization, duplicate invocations may be eliminated or the function may even be invoked
more times than it is present in the query. If your function is not deterministic, call
`asNondeterministic` on the user defined function. E.g.:
>>> from pyspark.sql.types import IntegerType
>>> import random
>>> random_udf = udf(lambda: int(random.random() * 100), IntegerType()).asNondeterministic()
[SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames - The old implicit would convert RDDs directly to DataFrames, and that added too many methods. - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed Python changes: - toDataFrame -> toDF - Dsl -> functions package - addColumn -> withColumn - renameColumn -> withColumnRenamed - add toDF functions to RDD on SQLContext init - add flatMap to DataFrame Author: Reynold Xin <rxin@databricks.com> Author: Davies Liu <davies@databricks.com> Closes #4556 from rxin/SPARK-5752 and squashes the following commits: 5ef9910 [Reynold Xin] More fix 61d3fca [Reynold Xin] Merge branch 'df5' of github.com:davies/spark into SPARK-5752 ff5832c [Reynold Xin] Fix python 749c675 [Reynold Xin] count(*) fixes. 5806df0 [Reynold Xin] Fix build break again. d941f3d [Reynold Xin] Fixed explode compilation break. fe1267a [Davies Liu] flatMap c4afb8e [Reynold Xin] style d9de47f [Davies Liu] add comment b783994 [Davies Liu] add comment for toDF e2154e5 [Davies Liu] schema() -> schema 3a1004f [Davies Liu] Dsl -> functions, toDF() fb256af [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 0dd74eb [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames 97dd47c [Davies Liu] fix mistake 6168f74 [Davies Liu] fix test 1fc0199 [Davies Liu] fix test a075cd5 [Davies Liu] clean up, toPandas 663d314 [Davies Liu] add test for agg('*') 9e214d5 [Reynold Xin] count(*) fixes. 1ed7136 [Reynold Xin] Fix build break again. 921b2e3 [Reynold Xin] Fixed explode compilation break. 14698d4 [Davies Liu] flatMap ba3e12d [Reynold Xin] style d08c92d [Davies Liu] add comment 5c8b524 [Davies Liu] add comment for toDF a4e5e66 [Davies Liu] schema() -> schema d377fc9 [Davies Liu] Dsl -> functions, toDF() 6b3086c [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 807e8b1 [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames
2015-02-14 02:03:22 -05:00
.. note:: The user-defined functions do not support conditional expressions or short circuiting
in boolean expressions and it ends up with being executed all internally. If the functions
can fail on special rows, the workaround is to incorporate the condition into the functions.
[SPARK-23645][MINOR][DOCS][PYTHON] Add docs RE `pandas_udf` with keyword args ## What changes were proposed in this pull request? Add documentation about the limitations of `pandas_udf` with keyword arguments and related concepts, like `functools.partial` fn objects. NOTE: intermediate commits on this PR show some of the steps that can be taken to fix some (but not all) of these pain points. ### Survey of problems we face today: (Initialize) Note: python 3.6 and spark 2.4snapshot. ``` from pyspark.sql import SparkSession import inspect, functools from pyspark.sql.functions import pandas_udf, PandasUDFType, col, lit, udf spark = SparkSession.builder.getOrCreate() print(spark.version) df = spark.range(1,6).withColumn('b', col('id') * 2) def ok(a,b): return a+b ``` Using a keyword argument at the call site `b=...` (and yes, *full* stack trace below, haha): ``` ---> 14 df.withColumn('ok', pandas_udf(f=ok, returnType='bigint')('id', b='id')).show() # no kwargs TypeError: wrapper() got an unexpected keyword argument 'b' ``` Using partial with a keyword argument where the kw-arg is the first argument of the fn: *(Aside: kind of interesting that lines 15,16 work great and then 17 explodes)* ``` --------------------------------------------------------------------------- ValueError Traceback (most recent call last) <ipython-input-9-e9f31b8799c1> in <module>() 15 df.withColumn('ok', pandas_udf(f=functools.partial(ok, 7), returnType='bigint')('id')).show() 16 df.withColumn('ok', pandas_udf(f=functools.partial(ok, b=7), returnType='bigint')('id')).show() ---> 17 df.withColumn('ok', pandas_udf(f=functools.partial(ok, a=7), returnType='bigint')('id')).show() /Users/stu/ZZ/spark/python/pyspark/sql/functions.py in pandas_udf(f, returnType, functionType) 2378 return functools.partial(_create_udf, returnType=return_type, evalType=eval_type) 2379 else: -> 2380 return _create_udf(f=f, returnType=return_type, evalType=eval_type) 2381 2382 /Users/stu/ZZ/spark/python/pyspark/sql/udf.py in _create_udf(f, returnType, evalType) 54 argspec.varargs is None: 55 raise ValueError( ---> 56 "Invalid function: 0-arg pandas_udfs are not supported. " 57 "Instead, create a 1-arg pandas_udf and ignore the arg in your function." 58 ) ValueError: Invalid function: 0-arg pandas_udfs are not supported. Instead, create a 1-arg pandas_udf and ignore the arg in your function. ``` Author: Michael (Stu) Stewart <mstewart141@gmail.com> Closes #20900 from mstewart141/udfkw2.
2018-03-25 23:45:45 -04:00
.. note:: The user-defined functions do not take keyword arguments on the calling side.
:param f: python function if used as a standalone function
:param returnType: the return type of the user-defined function. The value can be either a
:class:`pyspark.sql.types.DataType` object or a DDL-formatted type string.
>>> from pyspark.sql.types import IntegerType
[SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames - The old implicit would convert RDDs directly to DataFrames, and that added too many methods. - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed Python changes: - toDataFrame -> toDF - Dsl -> functions package - addColumn -> withColumn - renameColumn -> withColumnRenamed - add toDF functions to RDD on SQLContext init - add flatMap to DataFrame Author: Reynold Xin <rxin@databricks.com> Author: Davies Liu <davies@databricks.com> Closes #4556 from rxin/SPARK-5752 and squashes the following commits: 5ef9910 [Reynold Xin] More fix 61d3fca [Reynold Xin] Merge branch 'df5' of github.com:davies/spark into SPARK-5752 ff5832c [Reynold Xin] Fix python 749c675 [Reynold Xin] count(*) fixes. 5806df0 [Reynold Xin] Fix build break again. d941f3d [Reynold Xin] Fixed explode compilation break. fe1267a [Davies Liu] flatMap c4afb8e [Reynold Xin] style d9de47f [Davies Liu] add comment b783994 [Davies Liu] add comment for toDF e2154e5 [Davies Liu] schema() -> schema 3a1004f [Davies Liu] Dsl -> functions, toDF() fb256af [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 0dd74eb [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames 97dd47c [Davies Liu] fix mistake 6168f74 [Davies Liu] fix test 1fc0199 [Davies Liu] fix test a075cd5 [Davies Liu] clean up, toPandas 663d314 [Davies Liu] add test for agg('*') 9e214d5 [Reynold Xin] count(*) fixes. 1ed7136 [Reynold Xin] Fix build break again. 921b2e3 [Reynold Xin] Fixed explode compilation break. 14698d4 [Davies Liu] flatMap ba3e12d [Reynold Xin] style d08c92d [Davies Liu] add comment 5c8b524 [Davies Liu] add comment for toDF a4e5e66 [Davies Liu] schema() -> schema d377fc9 [Davies Liu] Dsl -> functions, toDF() 6b3086c [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 807e8b1 [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames
2015-02-14 02:03:22 -05:00
>>> slen = udf(lambda s: len(s), IntegerType())
>>> @udf
... def to_upper(s):
... if s is not None:
... return s.upper()
...
>>> @udf(returnType=IntegerType())
... def add_one(x):
... if x is not None:
... return x + 1
...
>>> df = spark.createDataFrame([(1, "John Doe", 21)], ("id", "name", "age"))
>>> df.select(slen("name").alias("slen(name)"), to_upper("name"), add_one("age")).show()
+----------+--------------+------------+
|slen(name)|to_upper(name)|add_one(age)|
+----------+--------------+------------+
| 8| JOHN DOE| 22|
+----------+--------------+------------+
"""
[SPARK-25666][PYTHON] Internally document type conversion between Python data and SQL types in normal UDFs ### What changes were proposed in this pull request? We are facing some problems about type conversions between Python data and SQL types in UDFs (Pandas UDFs as well). It's even difficult to identify the problems (see https://github.com/apache/spark/pull/20163 and https://github.com/apache/spark/pull/22610). This PR targets to internally document the type conversion table. Some of them looks buggy and we should fix them. ```python import sys import array import datetime from decimal import Decimal from pyspark.sql import Row from pyspark.sql.types import * from pyspark.sql.functions import udf if sys.version >= '3': long = int data = [ None, True, 1, long(1), "a", u"a", datetime.date(1970, 1, 1), datetime.datetime(1970, 1, 1, 0, 0), 1.0, array.array("i", [1]), [1], (1,), bytearray([65, 66, 67]), Decimal(1), {"a": 1}, Row(kwargs=1), Row("namedtuple")(1), ] types = [ BooleanType(), ByteType(), ShortType(), IntegerType(), LongType(), StringType(), DateType(), TimestampType(), FloatType(), DoubleType(), ArrayType(IntegerType()), BinaryType(), DecimalType(10, 0), MapType(StringType(), IntegerType()), StructType([StructField("_1", IntegerType())]), ] df = spark.range(1) results = [] count = 0 total = len(types) * len(data) spark.sparkContext.setLogLevel("FATAL") for t in types: result = [] for v in data: try: row = df.select(udf(lambda: v, t)()).first() ret_str = repr(row[0]) except Exception: ret_str = "X" result.append(ret_str) progress = "SQL Type: [%s]\n Python Value: [%s(%s)]\n Result Python Value: [%s]" % ( t.simpleString(), str(v), type(v).__name__, ret_str) count += 1 print("%s/%s:\n %s" % (count, total, progress)) results.append([t.simpleString()] + list(map(str, result))) schema = ["SQL Type \\ Python Value(Type)"] + list(map(lambda v: "%s(%s)" % (str(v), type(v).__name__), data)) strings = spark.createDataFrame(results, schema=schema)._jdf.showString(20, 20, False) print("\n".join(map(lambda line: " # %s # noqa" % line, strings.strip().split("\n")))) ``` This table was generated under Python 2 but the code above is Python 3 compatible as well. ## How was this patch tested? Manually tested and lint check. Closes #22655 from HyukjinKwon/SPARK-25666. Authored-by: hyukjinkwon <gurwls223@apache.org> Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-10-08 03:47:15 -04:00
# The following table shows most of Python data and SQL type conversions in normal UDFs that
# are not yet visible to the user. Some of behaviors are buggy and might be changed in the near
# future. The table might have to be eventually documented externally.
# Please see SPARK-25666's PR to see the codes in order to generate the table below.
#
# +-----------------------------+--------------+----------+------+-------+---------------+---------------+--------------------+-----------------------------+----------+----------------------+---------+--------------------+-----------------+------------+--------------+------------------+----------------------+ # noqa
# |SQL Type \ Python Value(Type)|None(NoneType)|True(bool)|1(int)|1(long)| a(str)| a(unicode)| 1970-01-01(date)|1970-01-01 00:00:00(datetime)|1.0(float)|array('i', [1])(array)|[1](list)| (1,)(tuple)| ABC(bytearray)| 1(Decimal)|{'a': 1}(dict)|Row(kwargs=1)(Row)|Row(namedtuple=1)(Row)| # noqa
# +-----------------------------+--------------+----------+------+-------+---------------+---------------+--------------------+-----------------------------+----------+----------------------+---------+--------------------+-----------------+------------+--------------+------------------+----------------------+ # noqa
# | boolean| None| True| None| None| None| None| None| None| None| None| None| None| None| None| None| X| X| # noqa
# | tinyint| None| None| 1| 1| None| None| None| None| None| None| None| None| None| None| None| X| X| # noqa
# | smallint| None| None| 1| 1| None| None| None| None| None| None| None| None| None| None| None| X| X| # noqa
# | int| None| None| 1| 1| None| None| None| None| None| None| None| None| None| None| None| X| X| # noqa
# | bigint| None| None| 1| 1| None| None| None| None| None| None| None| None| None| None| None| X| X| # noqa
# | string| None| u'true'| u'1'| u'1'| u'a'| u'a'|u'java.util.Grego...| u'java.util.Grego...| u'1.0'| u'[I@24a83055'| u'[1]'|u'[Ljava.lang.Obj...| u'[B@49093632'| u'1'| u'{a=1}'| X| X| # noqa
# | date| None| X| X| X| X| X|datetime.date(197...| datetime.date(197...| X| X| X| X| X| X| X| X| X| # noqa
# | timestamp| None| X| X| X| X| X| X| datetime.datetime...| X| X| X| X| X| X| X| X| X| # noqa
# | float| None| None| None| None| None| None| None| None| 1.0| None| None| None| None| None| None| X| X| # noqa
# | double| None| None| None| None| None| None| None| None| 1.0| None| None| None| None| None| None| X| X| # noqa
# | array<int>| None| None| None| None| None| None| None| None| None| [1]| [1]| [1]| [65, 66, 67]| None| None| X| X| # noqa
# | binary| None| None| None| None|bytearray(b'a')|bytearray(b'a')| None| None| None| None| None| None|bytearray(b'ABC')| None| None| X| X| # noqa
# | decimal(10,0)| None| None| None| None| None| None| None| None| None| None| None| None| None|Decimal('1')| None| X| X| # noqa
# | map<string,int>| None| None| None| None| None| None| None| None| None| None| None| None| None| None| {u'a': 1}| X| X| # noqa
# | struct<_1:int>| None| X| X| X| X| X| X| X| X| X|Row(_1=1)| Row(_1=1)| X| X| Row(_1=None)| Row(_1=1)| Row(_1=1)| # noqa
# +-----------------------------+--------------+----------+------+-------+---------------+---------------+--------------------+-----------------------------+----------+----------------------+---------+--------------------+-----------------+------------+--------------+------------------+----------------------+ # noqa
#
# Note: DDL formatted string is used for 'SQL Type' for simplicity. This string can be
# used in `returnType`.
# Note: The values inside of the table are generated by `repr`.
# Note: Python 2 is used to generate this table since it is used to check the backward
# compatibility often in practice.
# Note: 'X' means it throws an exception during the conversion.
# decorator @udf, @udf(), @udf(dataType())
if f is None or isinstance(f, (str, DataType)):
# If DataType has been passed as a positional argument
# for decorator use it as a returnType
return_type = f or returnType
return functools.partial(_create_udf, returnType=return_type,
evalType=PythonEvalType.SQL_BATCHED_UDF)
else:
return _create_udf(f=f, returnType=returnType,
evalType=PythonEvalType.SQL_BATCHED_UDF)
@since(2.3)
def pandas_udf(f=None, returnType=None, functionType=None):
"""
2017-10-10 18:32:01 -04:00
Creates a vectorized user defined function (UDF).
2017-10-10 18:32:01 -04:00
:param f: user-defined function. A python function if used as a standalone function
:param returnType: the return type of the user-defined function. The value can be either a
:class:`pyspark.sql.types.DataType` object or a DDL-formatted type string.
:param functionType: an enum value in :class:`pyspark.sql.functions.PandasUDFType`.
Default: SCALAR.
.. note:: Experimental
The function type of the UDF can be one of the following:
2017-10-10 18:32:01 -04:00
1. SCALAR
2017-10-10 18:32:01 -04:00
A scalar UDF defines a transformation: One or more `pandas.Series` -> A `pandas.Series`.
2017-10-10 18:32:01 -04:00
The length of the returned `pandas.Series` must be of the same as the input `pandas.Series`.
If the return type is :class:`StructType`, the returned value should be a `pandas.DataFrame`.
:class:`MapType`, nested :class:`StructType` are currently not supported as output types.
2017-10-10 18:32:01 -04:00
Scalar UDFs are used with :meth:`pyspark.sql.DataFrame.withColumn` and
:meth:`pyspark.sql.DataFrame.select`.
>>> from pyspark.sql.functions import pandas_udf, PandasUDFType
2017-10-10 18:32:01 -04:00
>>> from pyspark.sql.types import IntegerType, StringType
>>> slen = pandas_udf(lambda s: s.str.len(), IntegerType()) # doctest: +SKIP
>>> @pandas_udf(StringType()) # doctest: +SKIP
2017-10-10 18:32:01 -04:00
... def to_upper(s):
... return s.str.upper()
...
>>> @pandas_udf("integer", PandasUDFType.SCALAR) # doctest: +SKIP
2017-10-10 18:32:01 -04:00
... def add_one(x):
... return x + 1
...
>>> df = spark.createDataFrame([(1, "John Doe", 21)],
... ("id", "name", "age")) # doctest: +SKIP
2017-10-10 18:32:01 -04:00
>>> df.select(slen("name").alias("slen(name)"), to_upper("name"), add_one("age")) \\
... .show() # doctest: +SKIP
+----------+--------------+------------+
|slen(name)|to_upper(name)|add_one(age)|
+----------+--------------+------------+
| 8| JOHN DOE| 22|
+----------+--------------+------------+
>>> @pandas_udf("first string, last string") # doctest: +SKIP
... def split_expand(n):
... return n.str.split(expand=True)
>>> df.select(split_expand("name")).show() # doctest: +SKIP
+------------------+
|split_expand(name)|
+------------------+
| [John, Doe]|
+------------------+
2017-10-10 18:32:01 -04:00
.. note:: The length of `pandas.Series` within a scalar UDF is not that of the whole input
column, but is the length of an internal batch used for each call to the function.
Therefore, this can be used, for example, to ensure the length of each returned
`pandas.Series`, and can not be used as the column length.
2. GROUPED_MAP
2017-10-10 18:32:01 -04:00
A grouped map UDF defines transformation: A `pandas.DataFrame` -> A `pandas.DataFrame`
2017-10-10 18:32:01 -04:00
The returnType should be a :class:`StructType` describing the schema of the returned
`pandas.DataFrame`. The column labels of the returned `pandas.DataFrame` must either match
the field names in the defined returnType schema if specified as strings, or match the
field data types by position if not strings, e.g. integer indices.
The length of the returned `pandas.DataFrame` can be arbitrary.
Grouped map UDFs are used with :meth:`pyspark.sql.GroupedData.apply`.
2017-10-10 18:32:01 -04:00
>>> from pyspark.sql.functions import pandas_udf, PandasUDFType
2017-10-10 18:32:01 -04:00
>>> df = spark.createDataFrame(
... [(1, 1.0), (1, 2.0), (2, 3.0), (2, 5.0), (2, 10.0)],
... ("id", "v")) # doctest: +SKIP
>>> @pandas_udf("id long, v double", PandasUDFType.GROUPED_MAP) # doctest: +SKIP
2017-10-10 18:32:01 -04:00
... def normalize(pdf):
... v = pdf.v
... return pdf.assign(v=(v - v.mean()) / v.std())
>>> df.groupby("id").apply(normalize).show() # doctest: +SKIP
2017-10-10 18:32:01 -04:00
+---+-------------------+
| id| v|
+---+-------------------+
| 1|-0.7071067811865475|
| 1| 0.7071067811865475|
| 2|-0.8320502943378437|
| 2|-0.2773500981126146|
| 2| 1.1094003924504583|
+---+-------------------+
Alternatively, the user can define a function that takes two arguments.
In this case, the grouping key(s) will be passed as the first argument and the data will
be passed as the second argument. The grouping key(s) will be passed as a tuple of numpy
data types, e.g., `numpy.int32` and `numpy.float64`. The data will still be passed in
as a `pandas.DataFrame` containing all columns from the original Spark DataFrame.
This is useful when the user does not want to hardcode grouping key(s) in the function.
>>> import pandas as pd # doctest: +SKIP
>>> from pyspark.sql.functions import pandas_udf, PandasUDFType
>>> df = spark.createDataFrame(
... [(1, 1.0), (1, 2.0), (2, 3.0), (2, 5.0), (2, 10.0)],
... ("id", "v")) # doctest: +SKIP
>>> @pandas_udf("id long, v double", PandasUDFType.GROUPED_MAP) # doctest: +SKIP
... def mean_udf(key, pdf):
... # key is a tuple of one numpy.int64, which is the value
... # of 'id' for the current group
... return pd.DataFrame([key + (pdf.v.mean(),)])
>>> df.groupby('id').apply(mean_udf).show() # doctest: +SKIP
+---+---+
| id| v|
+---+---+
| 1|1.5|
| 2|6.0|
+---+---+
>>> @pandas_udf(
... "id long, `ceil(v / 2)` long, v double",
... PandasUDFType.GROUPED_MAP) # doctest: +SKIP
>>> def sum_udf(key, pdf):
... # key is a tuple of two numpy.int64s, which is the values
... # of 'id' and 'ceil(df.v / 2)' for the current group
... return pd.DataFrame([key + (pdf.v.sum(),)])
>>> df.groupby(df.id, ceil(df.v / 2)).apply(sum_udf).show() # doctest: +SKIP
+---+-----------+----+
| id|ceil(v / 2)| v|
+---+-----------+----+
| 2| 5|10.0|
| 1| 1| 3.0|
| 2| 3| 5.0|
| 2| 2| 3.0|
+---+-----------+----+
.. note:: If returning a new `pandas.DataFrame` constructed with a dictionary, it is
recommended to explicitly index the columns by name to ensure the positions are correct,
or alternatively use an `OrderedDict`.
For example, `pd.DataFrame({'id': ids, 'a': data}, columns=['id', 'a'])` or
`pd.DataFrame(OrderedDict([('id', ids), ('a', data)]))`.
2017-10-10 18:32:01 -04:00
.. seealso:: :meth:`pyspark.sql.GroupedData.apply`
3. GROUPED_AGG
A grouped aggregate UDF defines a transformation: One or more `pandas.Series` -> A scalar
The `returnType` should be a primitive data type, e.g., :class:`DoubleType`.
The returned scalar can be either a python primitive type, e.g., `int` or `float`
or a numpy data type, e.g., `numpy.int64` or `numpy.float64`.
2018-06-12 21:10:52 -04:00
:class:`MapType` and :class:`StructType` are currently not supported as output types.
2018-06-12 21:10:52 -04:00
Group aggregate UDFs are used with :meth:`pyspark.sql.GroupedData.agg` and
:class:`pyspark.sql.Window`
This example shows using grouped aggregated UDFs with groupby:
>>> from pyspark.sql.functions import pandas_udf, PandasUDFType
>>> df = spark.createDataFrame(
... [(1, 1.0), (1, 2.0), (2, 3.0), (2, 5.0), (2, 10.0)],
... ("id", "v"))
>>> @pandas_udf("double", PandasUDFType.GROUPED_AGG) # doctest: +SKIP
... def mean_udf(v):
... return v.mean()
>>> df.groupby("id").agg(mean_udf(df['v'])).show() # doctest: +SKIP
+---+-----------+
| id|mean_udf(v)|
+---+-----------+
| 1| 1.5|
| 2| 6.0|
+---+-----------+
[SPARK-24561][SQL][PYTHON] User-defined window aggregation functions with Pandas UDF (bounded window) ## What changes were proposed in this pull request? This PR implements a new feature - window aggregation Pandas UDF for bounded window. #### Doc: https://docs.google.com/document/d/14EjeY5z4-NC27-SmIP9CsMPCANeTcvxN44a7SIJtZPc/edit#heading=h.c87w44wcj3wj #### Example: ``` from pyspark.sql.functions import pandas_udf, PandasUDFType from pyspark.sql.window import Window df = spark.range(0, 10, 2).toDF('v') w1 = Window.partitionBy().orderBy('v').rangeBetween(-2, 4) w2 = Window.partitionBy().orderBy('v').rowsBetween(-2, 2) pandas_udf('double', PandasUDFType.GROUPED_AGG) def avg(v): return v.mean() df.withColumn('v_mean', avg(df['v']).over(w1)).show() # +---+------+ # | v|v_mean| # +---+------+ # | 0| 1.0| # | 2| 2.0| # | 4| 4.0| # | 6| 6.0| # | 8| 7.0| # +---+------+ df.withColumn('v_mean', avg(df['v']).over(w2)).show() # +---+------+ # | v|v_mean| # +---+------+ # | 0| 2.0| # | 2| 3.0| # | 4| 4.0| # | 6| 5.0| # | 8| 6.0| # +---+------+ ``` #### High level changes: This PR modifies the existing WindowInPandasExec physical node to deal with unbounded (growing, shrinking and sliding) windows. * `WindowInPandasExec` now share the same base class as `WindowExec` and share utility functions. See `WindowExecBase` * `WindowFunctionFrame` now has two new functions `currentLowerBound` and `currentUpperBound` - to return the lower and upper window bound for the current output row. It is also modified to allow `AggregateProcessor` == null. Null aggregator processor is used for `WindowInPandasExec` where we don't have an aggregator and only uses lower and upper bound functions from `WindowFunctionFrame` * The biggest change is in `WindowInPandasExec`, where it is modified to take `currentLowerBound` and `currentUpperBound` and write those values together with the input data to the python process for rolling window aggregation. See `WindowInPandasExec` for more details. #### Discussion In benchmarking, I found numpy variant of the rolling window UDF is much faster than the pandas version: Spark SQL window function: 20s Pandas variant: ~80s Numpy variant: 10s Numpy variant with numba: 4s Allowing numpy variant of the vectorized UDFs is something I want to discuss because of the performance improvement, but doesn't have to be in this PR. ## How was this patch tested? New tests Closes #22305 from icexelloss/SPARK-24561-bounded-window-udf. Authored-by: Li Jin <ice.xelloss@gmail.com> Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2018-12-17 20:15:21 -05:00
This example shows using grouped aggregated UDFs as window functions.
2018-06-12 21:10:52 -04:00
>>> from pyspark.sql.functions import pandas_udf, PandasUDFType
>>> from pyspark.sql import Window
>>> df = spark.createDataFrame(
... [(1, 1.0), (1, 2.0), (2, 3.0), (2, 5.0), (2, 10.0)],
... ("id", "v"))
>>> @pandas_udf("double", PandasUDFType.GROUPED_AGG) # doctest: +SKIP
... def mean_udf(v):
... return v.mean()
[SPARK-24561][SQL][PYTHON] User-defined window aggregation functions with Pandas UDF (bounded window) ## What changes were proposed in this pull request? This PR implements a new feature - window aggregation Pandas UDF for bounded window. #### Doc: https://docs.google.com/document/d/14EjeY5z4-NC27-SmIP9CsMPCANeTcvxN44a7SIJtZPc/edit#heading=h.c87w44wcj3wj #### Example: ``` from pyspark.sql.functions import pandas_udf, PandasUDFType from pyspark.sql.window import Window df = spark.range(0, 10, 2).toDF('v') w1 = Window.partitionBy().orderBy('v').rangeBetween(-2, 4) w2 = Window.partitionBy().orderBy('v').rowsBetween(-2, 2) pandas_udf('double', PandasUDFType.GROUPED_AGG) def avg(v): return v.mean() df.withColumn('v_mean', avg(df['v']).over(w1)).show() # +---+------+ # | v|v_mean| # +---+------+ # | 0| 1.0| # | 2| 2.0| # | 4| 4.0| # | 6| 6.0| # | 8| 7.0| # +---+------+ df.withColumn('v_mean', avg(df['v']).over(w2)).show() # +---+------+ # | v|v_mean| # +---+------+ # | 0| 2.0| # | 2| 3.0| # | 4| 4.0| # | 6| 5.0| # | 8| 6.0| # +---+------+ ``` #### High level changes: This PR modifies the existing WindowInPandasExec physical node to deal with unbounded (growing, shrinking and sliding) windows. * `WindowInPandasExec` now share the same base class as `WindowExec` and share utility functions. See `WindowExecBase` * `WindowFunctionFrame` now has two new functions `currentLowerBound` and `currentUpperBound` - to return the lower and upper window bound for the current output row. It is also modified to allow `AggregateProcessor` == null. Null aggregator processor is used for `WindowInPandasExec` where we don't have an aggregator and only uses lower and upper bound functions from `WindowFunctionFrame` * The biggest change is in `WindowInPandasExec`, where it is modified to take `currentLowerBound` and `currentUpperBound` and write those values together with the input data to the python process for rolling window aggregation. See `WindowInPandasExec` for more details. #### Discussion In benchmarking, I found numpy variant of the rolling window UDF is much faster than the pandas version: Spark SQL window function: 20s Pandas variant: ~80s Numpy variant: 10s Numpy variant with numba: 4s Allowing numpy variant of the vectorized UDFs is something I want to discuss because of the performance improvement, but doesn't have to be in this PR. ## How was this patch tested? New tests Closes #22305 from icexelloss/SPARK-24561-bounded-window-udf. Authored-by: Li Jin <ice.xelloss@gmail.com> Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2018-12-17 20:15:21 -05:00
>>> w = (Window.partitionBy('id')
... .orderBy('v')
... .rowsBetween(-1, 0))
2018-06-12 21:10:52 -04:00
>>> df.withColumn('mean_v', mean_udf(df['v']).over(w)).show() # doctest: +SKIP
+---+----+------+
| id| v|mean_v|
+---+----+------+
[SPARK-24561][SQL][PYTHON] User-defined window aggregation functions with Pandas UDF (bounded window) ## What changes were proposed in this pull request? This PR implements a new feature - window aggregation Pandas UDF for bounded window. #### Doc: https://docs.google.com/document/d/14EjeY5z4-NC27-SmIP9CsMPCANeTcvxN44a7SIJtZPc/edit#heading=h.c87w44wcj3wj #### Example: ``` from pyspark.sql.functions import pandas_udf, PandasUDFType from pyspark.sql.window import Window df = spark.range(0, 10, 2).toDF('v') w1 = Window.partitionBy().orderBy('v').rangeBetween(-2, 4) w2 = Window.partitionBy().orderBy('v').rowsBetween(-2, 2) pandas_udf('double', PandasUDFType.GROUPED_AGG) def avg(v): return v.mean() df.withColumn('v_mean', avg(df['v']).over(w1)).show() # +---+------+ # | v|v_mean| # +---+------+ # | 0| 1.0| # | 2| 2.0| # | 4| 4.0| # | 6| 6.0| # | 8| 7.0| # +---+------+ df.withColumn('v_mean', avg(df['v']).over(w2)).show() # +---+------+ # | v|v_mean| # +---+------+ # | 0| 2.0| # | 2| 3.0| # | 4| 4.0| # | 6| 5.0| # | 8| 6.0| # +---+------+ ``` #### High level changes: This PR modifies the existing WindowInPandasExec physical node to deal with unbounded (growing, shrinking and sliding) windows. * `WindowInPandasExec` now share the same base class as `WindowExec` and share utility functions. See `WindowExecBase` * `WindowFunctionFrame` now has two new functions `currentLowerBound` and `currentUpperBound` - to return the lower and upper window bound for the current output row. It is also modified to allow `AggregateProcessor` == null. Null aggregator processor is used for `WindowInPandasExec` where we don't have an aggregator and only uses lower and upper bound functions from `WindowFunctionFrame` * The biggest change is in `WindowInPandasExec`, where it is modified to take `currentLowerBound` and `currentUpperBound` and write those values together with the input data to the python process for rolling window aggregation. See `WindowInPandasExec` for more details. #### Discussion In benchmarking, I found numpy variant of the rolling window UDF is much faster than the pandas version: Spark SQL window function: 20s Pandas variant: ~80s Numpy variant: 10s Numpy variant with numba: 4s Allowing numpy variant of the vectorized UDFs is something I want to discuss because of the performance improvement, but doesn't have to be in this PR. ## How was this patch tested? New tests Closes #22305 from icexelloss/SPARK-24561-bounded-window-udf. Authored-by: Li Jin <ice.xelloss@gmail.com> Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2018-12-17 20:15:21 -05:00
| 1| 1.0| 1.0|
2018-06-12 21:10:52 -04:00
| 1| 2.0| 1.5|
[SPARK-24561][SQL][PYTHON] User-defined window aggregation functions with Pandas UDF (bounded window) ## What changes were proposed in this pull request? This PR implements a new feature - window aggregation Pandas UDF for bounded window. #### Doc: https://docs.google.com/document/d/14EjeY5z4-NC27-SmIP9CsMPCANeTcvxN44a7SIJtZPc/edit#heading=h.c87w44wcj3wj #### Example: ``` from pyspark.sql.functions import pandas_udf, PandasUDFType from pyspark.sql.window import Window df = spark.range(0, 10, 2).toDF('v') w1 = Window.partitionBy().orderBy('v').rangeBetween(-2, 4) w2 = Window.partitionBy().orderBy('v').rowsBetween(-2, 2) pandas_udf('double', PandasUDFType.GROUPED_AGG) def avg(v): return v.mean() df.withColumn('v_mean', avg(df['v']).over(w1)).show() # +---+------+ # | v|v_mean| # +---+------+ # | 0| 1.0| # | 2| 2.0| # | 4| 4.0| # | 6| 6.0| # | 8| 7.0| # +---+------+ df.withColumn('v_mean', avg(df['v']).over(w2)).show() # +---+------+ # | v|v_mean| # +---+------+ # | 0| 2.0| # | 2| 3.0| # | 4| 4.0| # | 6| 5.0| # | 8| 6.0| # +---+------+ ``` #### High level changes: This PR modifies the existing WindowInPandasExec physical node to deal with unbounded (growing, shrinking and sliding) windows. * `WindowInPandasExec` now share the same base class as `WindowExec` and share utility functions. See `WindowExecBase` * `WindowFunctionFrame` now has two new functions `currentLowerBound` and `currentUpperBound` - to return the lower and upper window bound for the current output row. It is also modified to allow `AggregateProcessor` == null. Null aggregator processor is used for `WindowInPandasExec` where we don't have an aggregator and only uses lower and upper bound functions from `WindowFunctionFrame` * The biggest change is in `WindowInPandasExec`, where it is modified to take `currentLowerBound` and `currentUpperBound` and write those values together with the input data to the python process for rolling window aggregation. See `WindowInPandasExec` for more details. #### Discussion In benchmarking, I found numpy variant of the rolling window UDF is much faster than the pandas version: Spark SQL window function: 20s Pandas variant: ~80s Numpy variant: 10s Numpy variant with numba: 4s Allowing numpy variant of the vectorized UDFs is something I want to discuss because of the performance improvement, but doesn't have to be in this PR. ## How was this patch tested? New tests Closes #22305 from icexelloss/SPARK-24561-bounded-window-udf. Authored-by: Li Jin <ice.xelloss@gmail.com> Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2018-12-17 20:15:21 -05:00
| 2| 3.0| 3.0|
| 2| 5.0| 4.0|
| 2|10.0| 7.5|
2018-06-12 21:10:52 -04:00
+---+----+------+
[SPARK-24561][SQL][PYTHON] User-defined window aggregation functions with Pandas UDF (bounded window) ## What changes were proposed in this pull request? This PR implements a new feature - window aggregation Pandas UDF for bounded window. #### Doc: https://docs.google.com/document/d/14EjeY5z4-NC27-SmIP9CsMPCANeTcvxN44a7SIJtZPc/edit#heading=h.c87w44wcj3wj #### Example: ``` from pyspark.sql.functions import pandas_udf, PandasUDFType from pyspark.sql.window import Window df = spark.range(0, 10, 2).toDF('v') w1 = Window.partitionBy().orderBy('v').rangeBetween(-2, 4) w2 = Window.partitionBy().orderBy('v').rowsBetween(-2, 2) pandas_udf('double', PandasUDFType.GROUPED_AGG) def avg(v): return v.mean() df.withColumn('v_mean', avg(df['v']).over(w1)).show() # +---+------+ # | v|v_mean| # +---+------+ # | 0| 1.0| # | 2| 2.0| # | 4| 4.0| # | 6| 6.0| # | 8| 7.0| # +---+------+ df.withColumn('v_mean', avg(df['v']).over(w2)).show() # +---+------+ # | v|v_mean| # +---+------+ # | 0| 2.0| # | 2| 3.0| # | 4| 4.0| # | 6| 5.0| # | 8| 6.0| # +---+------+ ``` #### High level changes: This PR modifies the existing WindowInPandasExec physical node to deal with unbounded (growing, shrinking and sliding) windows. * `WindowInPandasExec` now share the same base class as `WindowExec` and share utility functions. See `WindowExecBase` * `WindowFunctionFrame` now has two new functions `currentLowerBound` and `currentUpperBound` - to return the lower and upper window bound for the current output row. It is also modified to allow `AggregateProcessor` == null. Null aggregator processor is used for `WindowInPandasExec` where we don't have an aggregator and only uses lower and upper bound functions from `WindowFunctionFrame` * The biggest change is in `WindowInPandasExec`, where it is modified to take `currentLowerBound` and `currentUpperBound` and write those values together with the input data to the python process for rolling window aggregation. See `WindowInPandasExec` for more details. #### Discussion In benchmarking, I found numpy variant of the rolling window UDF is much faster than the pandas version: Spark SQL window function: 20s Pandas variant: ~80s Numpy variant: 10s Numpy variant with numba: 4s Allowing numpy variant of the vectorized UDFs is something I want to discuss because of the performance improvement, but doesn't have to be in this PR. ## How was this patch tested? New tests Closes #22305 from icexelloss/SPARK-24561-bounded-window-udf. Authored-by: Li Jin <ice.xelloss@gmail.com> Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2018-12-17 20:15:21 -05:00
.. note:: For performance reasons, the input series to window functions are not copied.
Therefore, mutating the input series is not allowed and will cause incorrect results.
For the same reason, users should also not rely on the index of the input series.
2018-06-12 21:10:52 -04:00
.. seealso:: :meth:`pyspark.sql.GroupedData.agg` and :class:`pyspark.sql.Window`
.. note:: The user-defined functions are considered deterministic by default. Due to
optimization, duplicate invocations may be eliminated or the function may even be invoked
more times than it is present in the query. If your function is not deterministic, call
`asNondeterministic` on the user defined function. E.g.:
>>> @pandas_udf('double', PandasUDFType.SCALAR) # doctest: +SKIP
... def random(v):
... import numpy as np
... import pandas as pd
... return pd.Series(np.random.randn(len(v))
>>> random = random.asNondeterministic() # doctest: +SKIP
.. note:: The user-defined functions do not support conditional expressions or short circuiting
in boolean expressions and it ends up with being executed all internally. If the functions
can fail on special rows, the workaround is to incorporate the condition into the functions.
[SPARK-23645][MINOR][DOCS][PYTHON] Add docs RE `pandas_udf` with keyword args ## What changes were proposed in this pull request? Add documentation about the limitations of `pandas_udf` with keyword arguments and related concepts, like `functools.partial` fn objects. NOTE: intermediate commits on this PR show some of the steps that can be taken to fix some (but not all) of these pain points. ### Survey of problems we face today: (Initialize) Note: python 3.6 and spark 2.4snapshot. ``` from pyspark.sql import SparkSession import inspect, functools from pyspark.sql.functions import pandas_udf, PandasUDFType, col, lit, udf spark = SparkSession.builder.getOrCreate() print(spark.version) df = spark.range(1,6).withColumn('b', col('id') * 2) def ok(a,b): return a+b ``` Using a keyword argument at the call site `b=...` (and yes, *full* stack trace below, haha): ``` ---> 14 df.withColumn('ok', pandas_udf(f=ok, returnType='bigint')('id', b='id')).show() # no kwargs TypeError: wrapper() got an unexpected keyword argument 'b' ``` Using partial with a keyword argument where the kw-arg is the first argument of the fn: *(Aside: kind of interesting that lines 15,16 work great and then 17 explodes)* ``` --------------------------------------------------------------------------- ValueError Traceback (most recent call last) <ipython-input-9-e9f31b8799c1> in <module>() 15 df.withColumn('ok', pandas_udf(f=functools.partial(ok, 7), returnType='bigint')('id')).show() 16 df.withColumn('ok', pandas_udf(f=functools.partial(ok, b=7), returnType='bigint')('id')).show() ---> 17 df.withColumn('ok', pandas_udf(f=functools.partial(ok, a=7), returnType='bigint')('id')).show() /Users/stu/ZZ/spark/python/pyspark/sql/functions.py in pandas_udf(f, returnType, functionType) 2378 return functools.partial(_create_udf, returnType=return_type, evalType=eval_type) 2379 else: -> 2380 return _create_udf(f=f, returnType=return_type, evalType=eval_type) 2381 2382 /Users/stu/ZZ/spark/python/pyspark/sql/udf.py in _create_udf(f, returnType, evalType) 54 argspec.varargs is None: 55 raise ValueError( ---> 56 "Invalid function: 0-arg pandas_udfs are not supported. " 57 "Instead, create a 1-arg pandas_udf and ignore the arg in your function." 58 ) ValueError: Invalid function: 0-arg pandas_udfs are not supported. Instead, create a 1-arg pandas_udf and ignore the arg in your function. ``` Author: Michael (Stu) Stewart <mstewart141@gmail.com> Closes #20900 from mstewart141/udfkw2.
2018-03-25 23:45:45 -04:00
.. note:: The user-defined functions do not take keyword arguments on the calling side.
.. note:: The data type of returned `pandas.Series` from the user-defined functions should be
matched with defined returnType (see :meth:`types.to_arrow_type` and
:meth:`types.from_arrow_type`). When there is mismatch between them, Spark might do
conversion on returned data. The conversion is not guaranteed to be correct and results
should be checked for accuracy by users.
"""
[SPARK-25798][PYTHON] Internally document type conversion between Pandas data and SQL types in Pandas UDFs ## What changes were proposed in this pull request? We are facing some problems about type conversions between Pandas data and SQL types in Pandas UDFs. It's even difficult to identify the problems (see #20163 and #22610). This PR targets to internally document the type conversion table. Some of them looks buggy and we should fix them. Table can be generated via the codes below: ```python from pyspark.sql.types import * from pyspark.sql.functions import pandas_udf columns = [ ('none', 'object(NoneType)'), ('bool', 'bool'), ('int8', 'int8'), ('int16', 'int16'), ('int32', 'int32'), ('int64', 'int64'), ('uint8', 'uint8'), ('uint16', 'uint16'), ('uint32', 'uint32'), ('uint64', 'uint64'), ('float64', 'float16'), ('float64', 'float32'), ('float64', 'float64'), ('date', 'datetime64[ns]'), ('tz_aware_dates', 'datetime64[ns, US/Eastern]'), ('string', 'object(string)'), ('decimal', 'object(Decimal)'), ('array', 'object(array[int32])'), ('float128', 'float128'), ('complex64', 'complex64'), ('complex128', 'complex128'), ('category', 'category'), ('tdeltas', 'timedelta64[ns]'), ] def create_dataframe(): import pandas as pd import numpy as np import decimal pdf = pd.DataFrame({ 'none': [None, None], 'bool': [True, False], 'int8': np.arange(1, 3).astype('int8'), 'int16': np.arange(1, 3).astype('int16'), 'int32': np.arange(1, 3).astype('int32'), 'int64': np.arange(1, 3).astype('int64'), 'uint8': np.arange(1, 3).astype('uint8'), 'uint16': np.arange(1, 3).astype('uint16'), 'uint32': np.arange(1, 3).astype('uint32'), 'uint64': np.arange(1, 3).astype('uint64'), 'float16': np.arange(1, 3).astype('float16'), 'float32': np.arange(1, 3).astype('float32'), 'float64': np.arange(1, 3).astype('float64'), 'float128': np.arange(1, 3).astype('float128'), 'complex64': np.arange(1, 3).astype('complex64'), 'complex128': np.arange(1, 3).astype('complex128'), 'string': list('ab'), 'array': pd.Series([np.array([1, 2, 3], dtype=np.int32), np.array([1, 2, 3], dtype=np.int32)]), 'decimal': pd.Series([decimal.Decimal('1'), decimal.Decimal('2')]), 'date': pd.date_range('19700101', periods=2).values, 'category': pd.Series(list("AB")).astype('category')}) pdf['tdeltas'] = [pdf.date.diff()[1], pdf.date.diff()[0]] pdf['tz_aware_dates'] = pd.date_range('19700101', periods=2, tz='US/Eastern') return pdf types = [ BooleanType(), ByteType(), ShortType(), IntegerType(), LongType(), FloatType(), DoubleType(), DateType(), TimestampType(), StringType(), DecimalType(10, 0), ArrayType(IntegerType()), MapType(StringType(), IntegerType()), StructType([StructField("_1", IntegerType())]), BinaryType(), ] df = spark.range(2).repartition(1) results = [] count = 0 total = len(types) * len(columns) values = [] spark.sparkContext.setLogLevel("FATAL") for t in types: result = [] for column, pandas_t in columns: v = create_dataframe()[column][0] values.append(v) try: row = df.select(pandas_udf(lambda _: create_dataframe()[column], t)(df.id)).first() ret_str = repr(row[0]) except Exception: ret_str = "X" result.append(ret_str) progress = "SQL Type: [%s]\n Pandas Value(Type): %s(%s)]\n Result Python Value: [%s]" % ( t.simpleString(), v, pandas_t, ret_str) count += 1 print("%s/%s:\n %s" % (count, total, progress)) results.append([t.simpleString()] + list(map(str, result))) schema = ["SQL Type \\ Pandas Value(Type)"] + list(map(lambda values_column: "%s(%s)" % (values_column[0], values_column[1][1]), zip(values, columns))) strings = spark.createDataFrame(results, schema=schema)._jdf.showString(20, 20, False) print("\n".join(map(lambda line: " # %s # noqa" % line, strings.strip().split("\n")))) ``` This code is compatible with both Python 2 and 3 but the table was generated under Python 2. ## How was this patch tested? Manually tested and lint check. Closes #22795 from HyukjinKwon/SPARK-25798. Authored-by: hyukjinkwon <gurwls223@apache.org> Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2018-10-24 13:04:17 -04:00
# The following table shows most of Pandas data and SQL type conversions in Pandas UDFs that
# are not yet visible to the user. Some of behaviors are buggy and might be changed in the near
# future. The table might have to be eventually documented externally.
# Please see SPARK-25798's PR to see the codes in order to generate the table below.
#
# +-----------------------------+----------------------+----------+-------+--------+--------------------+--------------------+--------+---------+---------+---------+------------+------------+------------+-----------------------------------+-----------------------------------------------------+-----------------+--------------------+-----------------------------+-------------+-----------------+------------------+-----------+--------------------------------+ # noqa
# |SQL Type \ Pandas Value(Type)|None(object(NoneType))|True(bool)|1(int8)|1(int16)| 1(int32)| 1(int64)|1(uint8)|1(uint16)|1(uint32)|1(uint64)|1.0(float16)|1.0(float32)|1.0(float64)|1970-01-01 00:00:00(datetime64[ns])|1970-01-01 00:00:00-05:00(datetime64[ns, US/Eastern])|a(object(string))| 1(object(Decimal))|[1 2 3](object(array[int32]))|1.0(float128)|(1+0j)(complex64)|(1+0j)(complex128)|A(category)|1 days 00:00:00(timedelta64[ns])| # noqa
# +-----------------------------+----------------------+----------+-------+--------+--------------------+--------------------+--------+---------+---------+---------+------------+------------+------------+-----------------------------------+-----------------------------------------------------+-----------------+--------------------+-----------------------------+-------------+-----------------+------------------+-----------+--------------------------------+ # noqa
# | boolean| None| True| True| True| True| True| True| True| True| True| False| False| False| False| False| X| X| X| False| False| False| X| False| # noqa
# | tinyint| None| 1| 1| 1| 1| 1| X| X| X| X| 1| 1| 1| X| X| X| X| X| X| X| X| 0| X| # noqa
# | smallint| None| 1| 1| 1| 1| 1| 1| X| X| X| 1| 1| 1| X| X| X| X| X| X| X| X| X| X| # noqa
# | int| None| 1| 1| 1| 1| 1| 1| 1| X| X| 1| 1| 1| X| X| X| X| X| X| X| X| X| X| # noqa
# | bigint| None| 1| 1| 1| 1| 1| 1| 1| 1| X| 1| 1| 1| 0| 18000000000000| X| X| X| X| X| X| X| X| # noqa
# | float| None| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| X| X| X|1.401298464324817...| X| X| X| X| X| X| # noqa
# | double| None| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| 1.0| X| X| X| X| X| X| X| X| X| X| # noqa
# | date| None| X| X| X|datetime.date(197...| X| X| X| X| X| X| X| X| datetime.date(197...| X| X| X| X| X| X| X| X| X| # noqa
# | timestamp| None| X| X| X| X|datetime.datetime...| X| X| X| X| X| X| X| datetime.datetime...| datetime.datetime...| X| X| X| X| X| X| X| X| # noqa
# | string| None| u''|u'\x01'| u'\x01'| u'\x01'| u'\x01'| u'\x01'| u'\x01'| u'\x01'| u'\x01'| u''| u''| u''| X| X| u'a'| X| X| u''| u''| u''| X| X| # noqa
# | decimal(10,0)| None| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| Decimal('1')| X| X| X| X| X| X| # noqa
# | array<int>| None| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| [1, 2, 3]| X| X| X| X| X| # noqa
# | map<string,int>| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| # noqa
# | struct<_1:int>| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| # noqa
# | binary| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| X| # noqa
# +-----------------------------+----------------------+----------+-------+--------+--------------------+--------------------+--------+---------+---------+---------+------------+------------+------------+-----------------------------------+-----------------------------------------------------+-----------------+--------------------+-----------------------------+-------------+-----------------+------------------+-----------+--------------------------------+ # noqa
#
# Note: DDL formatted string is used for 'SQL Type' for simplicity. This string can be
# used in `returnType`.
# Note: The values inside of the table are generated by `repr`.
# Note: Python 2 is used to generate this table since it is used to check the backward
# compatibility often in practice.
# Note: Pandas 0.19.2 and PyArrow 0.9.0 are used.
# Note: Timezone is Singapore timezone.
# Note: 'X' means it throws an exception during the conversion.
# Note: 'binary' type is only supported with PyArrow 0.10.0+ (SPARK-23555).
# decorator @pandas_udf(returnType, functionType)
is_decorator = f is None or isinstance(f, (str, DataType))
if is_decorator:
# If DataType has been passed as a positional argument
# for decorator use it as a returnType
return_type = f or returnType
if functionType is not None:
# @pandas_udf(dataType, functionType=functionType)
# @pandas_udf(returnType=dataType, functionType=functionType)
eval_type = functionType
elif returnType is not None and isinstance(returnType, int):
# @pandas_udf(dataType, functionType)
eval_type = returnType
else:
# @pandas_udf(dataType) or @pandas_udf(returnType=dataType)
eval_type = PythonEvalType.SQL_SCALAR_PANDAS_UDF
else:
return_type = returnType
if functionType is not None:
eval_type = functionType
else:
eval_type = PythonEvalType.SQL_SCALAR_PANDAS_UDF
if return_type is None:
raise ValueError("Invalid returnType: returnType can not be None")
if eval_type not in [PythonEvalType.SQL_SCALAR_PANDAS_UDF,
[SPARK-26412][PYSPARK][SQL] Allow Pandas UDF to take an iterator of pd.Series or an iterator of tuple of pd.Series ## What changes were proposed in this pull request? Allow Pandas UDF to take an iterator of pd.Series or an iterator of tuple of pd.Series. Note the UDF input args will be always one iterator: * if the udf take only column as input, the iterator's element will be pd.Series (corresponding to the column values batch) * if the udf take multiple columns as inputs, the iterator's element will be a tuple composed of multiple `pd.Series`s, each one corresponding to the multiple columns as inputs (keep the same order). For example: ``` pandas_udf("int", PandasUDFType.SCALAR_ITER) def the_udf(iterator): for col1_batch, col2_batch in iterator: yield col1_batch + col2_batch df.select(the_udf("col1", "col2")) ``` The udf above will add col1 and col2. I haven't add unit tests, but manually tests show it works fine. So it is ready for first pass review. We can test several typical cases: ``` from pyspark.sql import SparkSession from pyspark.sql.functions import pandas_udf, PandasUDFType from pyspark.sql.functions import udf from pyspark.taskcontext import TaskContext df = spark.createDataFrame([(1, 20), (3, 40)], ["a", "b"]) pandas_udf("int", PandasUDFType.SCALAR_ITER) def fi1(it): pid = TaskContext.get().partitionId() print("DBG: fi1: do init stuff, partitionId=" + str(pid)) for batch in it: yield batch + 100 print("DBG: fi1: do close stuff, partitionId=" + str(pid)) pandas_udf("int", PandasUDFType.SCALAR_ITER) def fi2(it): pid = TaskContext.get().partitionId() print("DBG: fi2: do init stuff, partitionId=" + str(pid)) for batch in it: yield batch + 10000 print("DBG: fi2: do close stuff, partitionId=" + str(pid)) pandas_udf("int", PandasUDFType.SCALAR_ITER) def fi3(it): pid = TaskContext.get().partitionId() print("DBG: fi3: do init stuff, partitionId=" + str(pid)) for x, y in it: yield x + y * 10 + 100000 print("DBG: fi3: do close stuff, partitionId=" + str(pid)) pandas_udf("int", PandasUDFType.SCALAR) def fp1(x): return x + 1000 udf("int") def fu1(x): return x + 10 # test select "pandas iter udf/pandas udf/sql udf" expressions at the same time. # Note this case the `fi1("a"), fi2("b"), fi3("a", "b")` will generate only one plan, # and `fu1("a")`, `fp1("a")` will generate another two separate plans. df.select(fi1("a"), fi2("b"), fi3("a", "b"), fu1("a"), fp1("a")).show() # test chain two pandas iter udf together # Note this case `fi2(fi1("a"))` will generate only one plan # Also note the init stuff/close stuff call order will be like: # (debug output following) # DBG: fi2: do init stuff, partitionId=0 # DBG: fi1: do init stuff, partitionId=0 # DBG: fi1: do close stuff, partitionId=0 # DBG: fi2: do close stuff, partitionId=0 df.select(fi2(fi1("a"))).show() # test more complex chain # Note this case `fi1("a"), fi2("a")` will generate one plan, # and `fi3(fi1_output, fi2_output)` will generate another plan df.select(fi3(fi1("a"), fi2("a"))).show() ``` ## How was this patch tested? To be added. Please review http://spark.apache.org/contributing.html before opening a pull request. Closes #24643 from WeichenXu123/pandas_udf_iter. Lead-authored-by: WeichenXu <weichen.xu@databricks.com> Co-authored-by: Xiangrui Meng <meng@databricks.com> Signed-off-by: Xiangrui Meng <meng@databricks.com>
2019-06-15 11:29:20 -04:00
PythonEvalType.SQL_SCALAR_PANDAS_ITER_UDF,
PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF,
PythonEvalType.SQL_GROUPED_AGG_PANDAS_UDF]:
raise ValueError("Invalid functionType: "
"functionType must be one the values from PandasUDFType")
if is_decorator:
return functools.partial(_create_udf, returnType=return_type, evalType=eval_type)
else:
return _create_udf(f=f, returnType=return_type, evalType=eval_type)
[SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames - The old implicit would convert RDDs directly to DataFrames, and that added too many methods. - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed Python changes: - toDataFrame -> toDF - Dsl -> functions package - addColumn -> withColumn - renameColumn -> withColumnRenamed - add toDF functions to RDD on SQLContext init - add flatMap to DataFrame Author: Reynold Xin <rxin@databricks.com> Author: Davies Liu <davies@databricks.com> Closes #4556 from rxin/SPARK-5752 and squashes the following commits: 5ef9910 [Reynold Xin] More fix 61d3fca [Reynold Xin] Merge branch 'df5' of github.com:davies/spark into SPARK-5752 ff5832c [Reynold Xin] Fix python 749c675 [Reynold Xin] count(*) fixes. 5806df0 [Reynold Xin] Fix build break again. d941f3d [Reynold Xin] Fixed explode compilation break. fe1267a [Davies Liu] flatMap c4afb8e [Reynold Xin] style d9de47f [Davies Liu] add comment b783994 [Davies Liu] add comment for toDF e2154e5 [Davies Liu] schema() -> schema 3a1004f [Davies Liu] Dsl -> functions, toDF() fb256af [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 0dd74eb [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames 97dd47c [Davies Liu] fix mistake 6168f74 [Davies Liu] fix test 1fc0199 [Davies Liu] fix test a075cd5 [Davies Liu] clean up, toPandas 663d314 [Davies Liu] add test for agg('*') 9e214d5 [Reynold Xin] count(*) fixes. 1ed7136 [Reynold Xin] Fix build break again. 921b2e3 [Reynold Xin] Fixed explode compilation break. 14698d4 [Davies Liu] flatMap ba3e12d [Reynold Xin] style d08c92d [Davies Liu] add comment 5c8b524 [Davies Liu] add comment for toDF a4e5e66 [Davies Liu] schema() -> schema d377fc9 [Davies Liu] Dsl -> functions, toDF() 6b3086c [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 807e8b1 [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames
2015-02-14 02:03:22 -05:00
blacklist = ['map', 'since', 'ignore_unicode_prefix']
__all__ = [k for k, v in globals().items()
if not k.startswith('_') and k[0].islower() and callable(v) and k not in blacklist]
__all__ += ["PandasUDFType"]
__all__.sort()
[SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames - The old implicit would convert RDDs directly to DataFrames, and that added too many methods. - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed Python changes: - toDataFrame -> toDF - Dsl -> functions package - addColumn -> withColumn - renameColumn -> withColumnRenamed - add toDF functions to RDD on SQLContext init - add flatMap to DataFrame Author: Reynold Xin <rxin@databricks.com> Author: Davies Liu <davies@databricks.com> Closes #4556 from rxin/SPARK-5752 and squashes the following commits: 5ef9910 [Reynold Xin] More fix 61d3fca [Reynold Xin] Merge branch 'df5' of github.com:davies/spark into SPARK-5752 ff5832c [Reynold Xin] Fix python 749c675 [Reynold Xin] count(*) fixes. 5806df0 [Reynold Xin] Fix build break again. d941f3d [Reynold Xin] Fixed explode compilation break. fe1267a [Davies Liu] flatMap c4afb8e [Reynold Xin] style d9de47f [Davies Liu] add comment b783994 [Davies Liu] add comment for toDF e2154e5 [Davies Liu] schema() -> schema 3a1004f [Davies Liu] Dsl -> functions, toDF() fb256af [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 0dd74eb [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames 97dd47c [Davies Liu] fix mistake 6168f74 [Davies Liu] fix test 1fc0199 [Davies Liu] fix test a075cd5 [Davies Liu] clean up, toPandas 663d314 [Davies Liu] add test for agg('*') 9e214d5 [Reynold Xin] count(*) fixes. 1ed7136 [Reynold Xin] Fix build break again. 921b2e3 [Reynold Xin] Fixed explode compilation break. 14698d4 [Davies Liu] flatMap ba3e12d [Reynold Xin] style d08c92d [Davies Liu] add comment 5c8b524 [Davies Liu] add comment for toDF a4e5e66 [Davies Liu] schema() -> schema d377fc9 [Davies Liu] Dsl -> functions, toDF() 6b3086c [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 807e8b1 [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames
2015-02-14 02:03:22 -05:00
def _test():
import doctest
from pyspark.sql import Row, SparkSession
import pyspark.sql.functions
globs = pyspark.sql.functions.__dict__.copy()
spark = SparkSession.builder\
.master("local[4]")\
.appName("sql.functions tests")\
.getOrCreate()
sc = spark.sparkContext
[SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames - The old implicit would convert RDDs directly to DataFrames, and that added too many methods. - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed Python changes: - toDataFrame -> toDF - Dsl -> functions package - addColumn -> withColumn - renameColumn -> withColumnRenamed - add toDF functions to RDD on SQLContext init - add flatMap to DataFrame Author: Reynold Xin <rxin@databricks.com> Author: Davies Liu <davies@databricks.com> Closes #4556 from rxin/SPARK-5752 and squashes the following commits: 5ef9910 [Reynold Xin] More fix 61d3fca [Reynold Xin] Merge branch 'df5' of github.com:davies/spark into SPARK-5752 ff5832c [Reynold Xin] Fix python 749c675 [Reynold Xin] count(*) fixes. 5806df0 [Reynold Xin] Fix build break again. d941f3d [Reynold Xin] Fixed explode compilation break. fe1267a [Davies Liu] flatMap c4afb8e [Reynold Xin] style d9de47f [Davies Liu] add comment b783994 [Davies Liu] add comment for toDF e2154e5 [Davies Liu] schema() -> schema 3a1004f [Davies Liu] Dsl -> functions, toDF() fb256af [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 0dd74eb [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames 97dd47c [Davies Liu] fix mistake 6168f74 [Davies Liu] fix test 1fc0199 [Davies Liu] fix test a075cd5 [Davies Liu] clean up, toPandas 663d314 [Davies Liu] add test for agg('*') 9e214d5 [Reynold Xin] count(*) fixes. 1ed7136 [Reynold Xin] Fix build break again. 921b2e3 [Reynold Xin] Fixed explode compilation break. 14698d4 [Davies Liu] flatMap ba3e12d [Reynold Xin] style d08c92d [Davies Liu] add comment 5c8b524 [Davies Liu] add comment for toDF a4e5e66 [Davies Liu] schema() -> schema d377fc9 [Davies Liu] Dsl -> functions, toDF() 6b3086c [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 807e8b1 [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames
2015-02-14 02:03:22 -05:00
globs['sc'] = sc
globs['spark'] = spark
globs['df'] = spark.createDataFrame([Row(name='Alice', age=2), Row(name='Bob', age=5)])
spark.conf.set("spark.sql.legacy.utcTimestampFunc.enabled", "true")
[SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames - The old implicit would convert RDDs directly to DataFrames, and that added too many methods. - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed Python changes: - toDataFrame -> toDF - Dsl -> functions package - addColumn -> withColumn - renameColumn -> withColumnRenamed - add toDF functions to RDD on SQLContext init - add flatMap to DataFrame Author: Reynold Xin <rxin@databricks.com> Author: Davies Liu <davies@databricks.com> Closes #4556 from rxin/SPARK-5752 and squashes the following commits: 5ef9910 [Reynold Xin] More fix 61d3fca [Reynold Xin] Merge branch 'df5' of github.com:davies/spark into SPARK-5752 ff5832c [Reynold Xin] Fix python 749c675 [Reynold Xin] count(*) fixes. 5806df0 [Reynold Xin] Fix build break again. d941f3d [Reynold Xin] Fixed explode compilation break. fe1267a [Davies Liu] flatMap c4afb8e [Reynold Xin] style d9de47f [Davies Liu] add comment b783994 [Davies Liu] add comment for toDF e2154e5 [Davies Liu] schema() -> schema 3a1004f [Davies Liu] Dsl -> functions, toDF() fb256af [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 0dd74eb [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames 97dd47c [Davies Liu] fix mistake 6168f74 [Davies Liu] fix test 1fc0199 [Davies Liu] fix test a075cd5 [Davies Liu] clean up, toPandas 663d314 [Davies Liu] add test for agg('*') 9e214d5 [Reynold Xin] count(*) fixes. 1ed7136 [Reynold Xin] Fix build break again. 921b2e3 [Reynold Xin] Fixed explode compilation break. 14698d4 [Davies Liu] flatMap ba3e12d [Reynold Xin] style d08c92d [Davies Liu] add comment 5c8b524 [Davies Liu] add comment for toDF a4e5e66 [Davies Liu] schema() -> schema d377fc9 [Davies Liu] Dsl -> functions, toDF() 6b3086c [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 807e8b1 [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames
2015-02-14 02:03:22 -05:00
(failure_count, test_count) = doctest.testmod(
pyspark.sql.functions, globs=globs,
[SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames - The old implicit would convert RDDs directly to DataFrames, and that added too many methods. - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed Python changes: - toDataFrame -> toDF - Dsl -> functions package - addColumn -> withColumn - renameColumn -> withColumnRenamed - add toDF functions to RDD on SQLContext init - add flatMap to DataFrame Author: Reynold Xin <rxin@databricks.com> Author: Davies Liu <davies@databricks.com> Closes #4556 from rxin/SPARK-5752 and squashes the following commits: 5ef9910 [Reynold Xin] More fix 61d3fca [Reynold Xin] Merge branch 'df5' of github.com:davies/spark into SPARK-5752 ff5832c [Reynold Xin] Fix python 749c675 [Reynold Xin] count(*) fixes. 5806df0 [Reynold Xin] Fix build break again. d941f3d [Reynold Xin] Fixed explode compilation break. fe1267a [Davies Liu] flatMap c4afb8e [Reynold Xin] style d9de47f [Davies Liu] add comment b783994 [Davies Liu] add comment for toDF e2154e5 [Davies Liu] schema() -> schema 3a1004f [Davies Liu] Dsl -> functions, toDF() fb256af [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 0dd74eb [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames 97dd47c [Davies Liu] fix mistake 6168f74 [Davies Liu] fix test 1fc0199 [Davies Liu] fix test a075cd5 [Davies Liu] clean up, toPandas 663d314 [Davies Liu] add test for agg('*') 9e214d5 [Reynold Xin] count(*) fixes. 1ed7136 [Reynold Xin] Fix build break again. 921b2e3 [Reynold Xin] Fixed explode compilation break. 14698d4 [Davies Liu] flatMap ba3e12d [Reynold Xin] style d08c92d [Davies Liu] add comment 5c8b524 [Davies Liu] add comment for toDF a4e5e66 [Davies Liu] schema() -> schema d377fc9 [Davies Liu] Dsl -> functions, toDF() 6b3086c [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 807e8b1 [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames
2015-02-14 02:03:22 -05:00
optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE)
spark.conf.unset("spark.sql.legacy.utcTimestampFunc.enabled")
spark.stop()
[SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames - The old implicit would convert RDDs directly to DataFrames, and that added too many methods. - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed Python changes: - toDataFrame -> toDF - Dsl -> functions package - addColumn -> withColumn - renameColumn -> withColumnRenamed - add toDF functions to RDD on SQLContext init - add flatMap to DataFrame Author: Reynold Xin <rxin@databricks.com> Author: Davies Liu <davies@databricks.com> Closes #4556 from rxin/SPARK-5752 and squashes the following commits: 5ef9910 [Reynold Xin] More fix 61d3fca [Reynold Xin] Merge branch 'df5' of github.com:davies/spark into SPARK-5752 ff5832c [Reynold Xin] Fix python 749c675 [Reynold Xin] count(*) fixes. 5806df0 [Reynold Xin] Fix build break again. d941f3d [Reynold Xin] Fixed explode compilation break. fe1267a [Davies Liu] flatMap c4afb8e [Reynold Xin] style d9de47f [Davies Liu] add comment b783994 [Davies Liu] add comment for toDF e2154e5 [Davies Liu] schema() -> schema 3a1004f [Davies Liu] Dsl -> functions, toDF() fb256af [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 0dd74eb [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames 97dd47c [Davies Liu] fix mistake 6168f74 [Davies Liu] fix test 1fc0199 [Davies Liu] fix test a075cd5 [Davies Liu] clean up, toPandas 663d314 [Davies Liu] add test for agg('*') 9e214d5 [Reynold Xin] count(*) fixes. 1ed7136 [Reynold Xin] Fix build break again. 921b2e3 [Reynold Xin] Fixed explode compilation break. 14698d4 [Davies Liu] flatMap ba3e12d [Reynold Xin] style d08c92d [Davies Liu] add comment 5c8b524 [Davies Liu] add comment for toDF a4e5e66 [Davies Liu] schema() -> schema d377fc9 [Davies Liu] Dsl -> functions, toDF() 6b3086c [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 807e8b1 [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames
2015-02-14 02:03:22 -05:00
if failure_count:
sys.exit(-1)
[SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames - The old implicit would convert RDDs directly to DataFrames, and that added too many methods. - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed Python changes: - toDataFrame -> toDF - Dsl -> functions package - addColumn -> withColumn - renameColumn -> withColumnRenamed - add toDF functions to RDD on SQLContext init - add flatMap to DataFrame Author: Reynold Xin <rxin@databricks.com> Author: Davies Liu <davies@databricks.com> Closes #4556 from rxin/SPARK-5752 and squashes the following commits: 5ef9910 [Reynold Xin] More fix 61d3fca [Reynold Xin] Merge branch 'df5' of github.com:davies/spark into SPARK-5752 ff5832c [Reynold Xin] Fix python 749c675 [Reynold Xin] count(*) fixes. 5806df0 [Reynold Xin] Fix build break again. d941f3d [Reynold Xin] Fixed explode compilation break. fe1267a [Davies Liu] flatMap c4afb8e [Reynold Xin] style d9de47f [Davies Liu] add comment b783994 [Davies Liu] add comment for toDF e2154e5 [Davies Liu] schema() -> schema 3a1004f [Davies Liu] Dsl -> functions, toDF() fb256af [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 0dd74eb [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames 97dd47c [Davies Liu] fix mistake 6168f74 [Davies Liu] fix test 1fc0199 [Davies Liu] fix test a075cd5 [Davies Liu] clean up, toPandas 663d314 [Davies Liu] add test for agg('*') 9e214d5 [Reynold Xin] count(*) fixes. 1ed7136 [Reynold Xin] Fix build break again. 921b2e3 [Reynold Xin] Fixed explode compilation break. 14698d4 [Davies Liu] flatMap ba3e12d [Reynold Xin] style d08c92d [Davies Liu] add comment 5c8b524 [Davies Liu] add comment for toDF a4e5e66 [Davies Liu] schema() -> schema d377fc9 [Davies Liu] Dsl -> functions, toDF() 6b3086c [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 807e8b1 [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames
2015-02-14 02:03:22 -05:00
if __name__ == "__main__":
_test()