diff options
author | Cheng Lian <lian@databricks.com> | 2016-02-21 22:53:15 +0800 |
---|---|---|
committer | Cheng Lian <lian@databricks.com> | 2016-02-21 22:53:15 +0800 |
commit | d9efe63ecdc60a9955f1924de0e8a00bcb6a559d (patch) | |
tree | 218879fda9e2285db67a72ed9fc42ec3ab61afa5 /python/pyspark | |
parent | d806ed34365aa27895547297fff4cc48ecbeacdf (diff) | |
download | spark-d9efe63ecdc60a9955f1924de0e8a00bcb6a559d.tar.gz spark-d9efe63ecdc60a9955f1924de0e8a00bcb6a559d.tar.bz2 spark-d9efe63ecdc60a9955f1924de0e8a00bcb6a559d.zip |
[SPARK-12799] Simplify various string output for expressions
This PR introduces several major changes:
1. Replacing `Expression.prettyString` with `Expression.sql`
The `prettyString` method is mostly an internal, developer faced facility for debugging purposes, and shouldn't be exposed to users.
1. Using SQL-like representation as column names for selected fields that are not named expression (back-ticks and double quotes should be removed)
Before, we were using `prettyString` as column names when possible, and sometimes the result column names can be weird. Here are several examples:
Expression | `prettyString` | `sql` | Note
------------------ | -------------- | ---------- | ---------------
`a && b` | `a && b` | `a AND b` |
`a.getField("f")` | `a[f]` | `a.f` | `a` is a struct
1. Adding trait `NonSQLExpression` extending from `Expression` for expressions that don't have a SQL representation (e.g. Scala UDF/UDAF and Java/Scala object expressions used for encoders)
`NonSQLExpression.sql` may return an arbitrary user facing string representation of the expression.
Author: Cheng Lian <lian@databricks.com>
Closes #10757 from liancheng/spark-12799.simplify-expression-string-methods.
Diffstat (limited to 'python/pyspark')
-rw-r--r-- | python/pyspark/sql/column.py | 32 | ||||
-rw-r--r-- | python/pyspark/sql/context.py | 10 | ||||
-rw-r--r-- | python/pyspark/sql/functions.py | 30 |
3 files changed, 36 insertions, 36 deletions
diff --git a/python/pyspark/sql/column.py b/python/pyspark/sql/column.py index 320451c52c..3866a49c0b 100644 --- a/python/pyspark/sql/column.py +++ b/python/pyspark/sql/column.py @@ -219,17 +219,17 @@ class Column(object): >>> from pyspark.sql import Row >>> df = sc.parallelize([Row(r=Row(a=1, b="b"))]).toDF() >>> df.select(df.r.getField("b")).show() - +----+ - |r[b]| - +----+ - | b| - +----+ + +---+ + |r.b| + +---+ + | b| + +---+ >>> df.select(df.r.a).show() - +----+ - |r[a]| - +----+ - | 1| - +----+ + +---+ + |r.a| + +---+ + | 1| + +---+ """ return self[name] @@ -346,12 +346,12 @@ class Column(object): expression is between the given columns. >>> df.select(df.name, df.age.between(2, 4)).show() - +-----+--------------------------+ - | name|((age >= 2) && (age <= 4))| - +-----+--------------------------+ - |Alice| true| - | Bob| false| - +-----+--------------------------+ + +-----+---------------------------+ + | name|((age >= 2) AND (age <= 4))| + +-----+---------------------------+ + |Alice| true| + | Bob| false| + +-----+---------------------------+ """ return (self >= lowerBound) & (self <= upperBound) diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index 7f6fb410ab..89bf1443a6 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -92,8 +92,8 @@ class SQLContext(object): >>> df.registerTempTable("allTypes") >>> sqlContext.sql('select i+1, d+1, not b, list[1], dict["s"], time, row.a ' ... 'from allTypes where b and i > 0').collect() - [Row(_c0=2, _c1=2.0, _c2=False, _c3=2, _c4=0, \ - time=datetime.datetime(2014, 8, 1, 14, 1, 5), a=1)] + [Row((i + CAST(1 AS BIGINT))=2, (d + CAST(1 AS DOUBLE))=2.0, (NOT b)=False, list[1]=2, \ + dict[s]=0, time=datetime.datetime(2014, 8, 1, 14, 1, 5), a=1)] >>> df.map(lambda x: (x.i, x.s, x.d, x.l, x.b, x.time, x.row.a, x.list)).collect() [(1, u'string', 1.0, 1, True, datetime.datetime(2014, 8, 1, 14, 1, 5), 1, [1, 2, 3])] """ @@ -210,17 +210,17 @@ class SQLContext(object): >>> sqlContext.registerFunction("stringLengthString", lambda x: len(x)) >>> sqlContext.sql("SELECT stringLengthString('test')").collect() - [Row(_c0=u'4')] + [Row(stringLengthString(test)=u'4')] >>> from pyspark.sql.types import IntegerType >>> sqlContext.registerFunction("stringLengthInt", lambda x: len(x), IntegerType()) >>> sqlContext.sql("SELECT stringLengthInt('test')").collect() - [Row(_c0=4)] + [Row(stringLengthInt(test)=4)] >>> from pyspark.sql.types import IntegerType >>> sqlContext.udf.register("stringLengthInt", lambda x: len(x), IntegerType()) >>> sqlContext.sql("SELECT stringLengthInt('test')").collect() - [Row(_c0=4)] + [Row(stringLengthInt(test)=4)] """ udf = UserDefinedFunction(f, returnType, name) self._ssql_ctx.udf().registerPython(name, udf._judf) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 5fc1cc2cae..fdae05d98c 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -223,22 +223,22 @@ def coalesce(*cols): +----+----+ >>> cDf.select(coalesce(cDf["a"], cDf["b"])).show() - +-------------+ - |coalesce(a,b)| - +-------------+ - | null| - | 1| - | 2| - +-------------+ + +--------------+ + |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| - +----+----+---------------+ + +----+----+----------------+ + | 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)) @@ -1528,7 +1528,7 @@ def array_contains(col, value): >>> df = sqlContext.createDataFrame([(["a", "b", "c"],), ([],)], ['data']) >>> df.select(array_contains(df.data, "a")).collect() - [Row(array_contains(data,a)=True), Row(array_contains(data,a)=False)] + [Row(array_contains(data, a)=True), Row(array_contains(data, a)=False)] """ sc = SparkContext._active_spark_context return Column(sc._jvm.functions.array_contains(_to_java_column(col), value)) |