You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by hv...@apache.org on 2016/09/25 23:46:17 UTC

spark git commit: [SPARK-17551][SQL] Add DataFrame API for null ordering

Repository: spark
Updated Branches:
  refs/heads/master 7945daed1 -> de333d121


[SPARK-17551][SQL] Add DataFrame API for null ordering

## What changes were proposed in this pull request?
This pull request adds Scala/Java DataFrame API for null ordering (NULLS FIRST | LAST).

Also did some minor clean up for related code (e.g. incorrect indentation), and renamed "orderby-nulls-ordering.sql" to be consistent with existing test files.

## How was this patch tested?
Added a new test case in DataFrameSuite.

Author: petermaxlee <pe...@gmail.com>
Author: Xin Wu <xi...@us.ibm.com>

Closes #15123 from petermaxlee/SPARK-17551.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/de333d12
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/de333d12
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/de333d12

Branch: refs/heads/master
Commit: de333d121da4cb80d45819cbcf8b4246e48ec4d0
Parents: 7945dae
Author: xin wu <xi...@us.ibm.com>
Authored: Sun Sep 25 16:46:12 2016 -0700
Committer: Herman van Hovell <hv...@databricks.com>
Committed: Sun Sep 25 16:46:12 2016 -0700

----------------------------------------------------------------------
 .../sql/catalyst/expressions/SortOrder.scala    |  28 +-
 .../expressions/codegen/GenerateOrdering.scala  |  16 +-
 .../scala/org/apache/spark/sql/Column.scala     |  64 ++++-
 .../scala/org/apache/spark/sql/functions.scala  |  51 +++-
 .../inputs/order-by-nulls-ordering.sql          |  83 ++++++
 .../sql-tests/inputs/orderby-nulls-ordering.sql |  83 ------
 .../results/order-by-nulls-ordering.sql.out     | 254 +++++++++++++++++++
 .../results/orderby-nulls-ordering.sql.out      | 254 -------------------
 .../org/apache/spark/sql/DataFrameSuite.scala   |  18 ++
 9 files changed, 481 insertions(+), 370 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/de333d12/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala
index d015125..3bebd55 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala
@@ -54,10 +54,7 @@ case object NullsLast extends NullOrdering{
  * An expression that can be used to sort a tuple.  This class extends expression primarily so that
  * transformations over expression will descend into its child.
  */
-case class SortOrder(
-  child: Expression,
-  direction: SortDirection,
-  nullOrdering: NullOrdering)
+case class SortOrder(child: Expression, direction: SortDirection, nullOrdering: NullOrdering)
   extends UnaryExpression with Unevaluable {
 
   /** Sort order is not foldable because we don't have an eval for it. */
@@ -94,17 +91,9 @@ case class SortPrefix(child: SortOrder) extends UnaryExpression {
 
   val nullValue = child.child.dataType match {
     case BooleanType | DateType | TimestampType | _: IntegralType =>
-      if (nullAsSmallest) {
-        Long.MinValue
-      } else {
-        Long.MaxValue
-      }
+      if (nullAsSmallest) Long.MinValue else Long.MaxValue
     case dt: DecimalType if dt.precision - dt.scale <= Decimal.MAX_LONG_DIGITS =>
-      if (nullAsSmallest) {
-        Long.MinValue
-      } else {
-        Long.MaxValue
-      }
+      if (nullAsSmallest) Long.MinValue else Long.MaxValue
     case _: DecimalType =>
       if (nullAsSmallest) {
         DoublePrefixComparator.computePrefix(Double.NegativeInfinity)
@@ -112,16 +101,13 @@ case class SortPrefix(child: SortOrder) extends UnaryExpression {
         DoublePrefixComparator.computePrefix(Double.NaN)
       }
     case _ =>
-      if (nullAsSmallest) {
-        0L
-      } else {
-        -1L
-      }
+      if (nullAsSmallest) 0L else -1L
   }
 
-  private def nullAsSmallest: Boolean = (child.isAscending && child.nullOrdering == NullsFirst) ||
+  private def nullAsSmallest: Boolean = {
+    (child.isAscending && child.nullOrdering == NullsFirst) ||
       (!child.isAscending && child.nullOrdering == NullsLast)
-
+  }
 
   override def eval(input: InternalRow): Any = throw new UnsupportedOperationException
 

http://git-wip-us.apache.org/repos/asf/spark/blob/de333d12/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala
index e7df95e..f1c30ef 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala
@@ -100,16 +100,16 @@ object GenerateOrdering extends CodeGenerator[Seq[SortOrder], Ordering[InternalR
             // Nothing
           } else if ($isNullA) {
             return ${
-        order.nullOrdering match {
-          case NullsFirst => "-1"
-          case NullsLast => "1"
-        }};
+              order.nullOrdering match {
+                case NullsFirst => "-1"
+                case NullsLast => "1"
+              }};
           } else if ($isNullB) {
             return ${
-        order.nullOrdering match {
-          case NullsFirst => "1"
-          case NullsLast => "-1"
-        }};
+              order.nullOrdering match {
+                case NullsFirst => "1"
+                case NullsLast => "-1"
+              }};
           } else {
             int comp = ${ctx.genComp(order.child.dataType, primitiveA, primitiveB)};
             if (comp != 0) {

http://git-wip-us.apache.org/repos/asf/spark/blob/de333d12/sql/core/src/main/scala/org/apache/spark/sql/Column.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala
index 844ca7a..63da501 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala
@@ -1007,7 +1007,7 @@ class Column(protected[sql] val expr: Expression) extends Logging {
   /**
    * Returns an ordering used in sorting.
    * {{{
-   *   // Scala: sort a DataFrame by age column in descending order.
+   *   // Scala
    *   df.sort(df("age").desc)
    *
    *   // Java
@@ -1020,7 +1020,37 @@ class Column(protected[sql] val expr: Expression) extends Logging {
   def desc: Column = withExpr { SortOrder(expr, Descending) }
 
   /**
-   * Returns an ordering used in sorting.
+   * Returns a descending ordering used in sorting, where null values appear before non-null values.
+   * {{{
+   *   // Scala: sort a DataFrame by age column in descending order and null values appearing first.
+   *   df.sort(df("age").desc_nulls_first)
+   *
+   *   // Java
+   *   df.sort(df.col("age").desc_nulls_first());
+   * }}}
+   *
+   * @group expr_ops
+   * @since 2.1.0
+   */
+  def desc_nulls_first: Column = withExpr { SortOrder(expr, Descending, NullsFirst) }
+
+  /**
+   * Returns a descending ordering used in sorting, where null values appear after non-null values.
+   * {{{
+   *   // Scala: sort a DataFrame by age column in descending order and null values appearing last.
+   *   df.sort(df("age").desc_nulls_last)
+   *
+   *   // Java
+   *   df.sort(df.col("age").desc_nulls_last());
+   * }}}
+   *
+   * @group expr_ops
+   * @since 2.1.0
+   */
+  def desc_nulls_last: Column = withExpr { SortOrder(expr, Descending, NullsLast) }
+
+  /**
+   * Returns an ascending ordering used in sorting.
    * {{{
    *   // Scala: sort a DataFrame by age column in ascending order.
    *   df.sort(df("age").asc)
@@ -1035,6 +1065,36 @@ class Column(protected[sql] val expr: Expression) extends Logging {
   def asc: Column = withExpr { SortOrder(expr, Ascending) }
 
   /**
+   * Returns an ascending ordering used in sorting, where null values appear before non-null values.
+   * {{{
+   *   // Scala: sort a DataFrame by age column in ascending order and null values appearing first.
+   *   df.sort(df("age").asc_nulls_last)
+   *
+   *   // Java
+   *   df.sort(df.col("age").asc_nulls_last());
+   * }}}
+   *
+   * @group expr_ops
+   * @since 2.1.0
+   */
+  def asc_nulls_first: Column = withExpr { SortOrder(expr, Ascending, NullsFirst) }
+
+  /**
+   * Returns an ordering used in sorting, where null values appear after non-null values.
+   * {{{
+   *   // Scala: sort a DataFrame by age column in ascending order and null values appearing last.
+   *   df.sort(df("age").asc_nulls_last)
+   *
+   *   // Java
+   *   df.sort(df.col("age").asc_nulls_last());
+   * }}}
+   *
+   * @group expr_ops
+   * @since 2.1.0
+   */
+  def asc_nulls_last: Column = withExpr { SortOrder(expr, Ascending, NullsLast) }
+
+  /**
    * Prints the expression to the console for debugging purpose.
    *
    * @group df_ops

http://git-wip-us.apache.org/repos/asf/spark/blob/de333d12/sql/core/src/main/scala/org/apache/spark/sql/functions.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala
index 960c87f..47bf41a 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala
@@ -109,7 +109,6 @@ object functions {
   /**
    * Returns a sort expression based on ascending order of the column.
    * {{{
-   *   // Sort by dept in ascending order, and then age in descending order.
    *   df.sort(asc("dept"), desc("age"))
    * }}}
    *
@@ -119,9 +118,32 @@ object functions {
   def asc(columnName: String): Column = Column(columnName).asc
 
   /**
+   * Returns a sort expression based on ascending order of the column,
+   * and null values return before non-null values.
+   * {{{
+   *   df.sort(asc_nulls_last("dept"), desc("age"))
+   * }}}
+   *
+   * @group sort_funcs
+   * @since 2.1.0
+   */
+  def asc_nulls_first(columnName: String): Column = Column(columnName).asc_nulls_first
+
+  /**
+   * Returns a sort expression based on ascending order of the column,
+   * and null values appear after non-null values.
+   * {{{
+   *   df.sort(asc_nulls_last("dept"), desc("age"))
+   * }}}
+   *
+   * @group sort_funcs
+   * @since 2.1.0
+   */
+  def asc_nulls_last(columnName: String): Column = Column(columnName).asc_nulls_last
+
+  /**
    * Returns a sort expression based on the descending order of the column.
    * {{{
-   *   // Sort by dept in ascending order, and then age in descending order.
    *   df.sort(asc("dept"), desc("age"))
    * }}}
    *
@@ -130,6 +152,31 @@ object functions {
    */
   def desc(columnName: String): Column = Column(columnName).desc
 
+  /**
+   * Returns a sort expression based on the descending order of the column,
+   * and null values appear before non-null values.
+   * {{{
+   *   df.sort(asc("dept"), desc_nulls_first("age"))
+   * }}}
+   *
+   * @group sort_funcs
+   * @since 2.1.0
+   */
+  def desc_nulls_first(columnName: String): Column = Column(columnName).desc_nulls_first
+
+  /**
+   * Returns a sort expression based on the descending order of the column,
+   * and null values appear after non-null values.
+   * {{{
+   *   df.sort(asc("dept"), desc_nulls_last("age"))
+   * }}}
+   *
+   * @group sort_funcs
+   * @since 2.1.0
+   */
+  def desc_nulls_last(columnName: String): Column = Column(columnName).desc_nulls_last
+
+
   //////////////////////////////////////////////////////////////////////////////////////////////
   // Aggregate functions
   //////////////////////////////////////////////////////////////////////////////////////////////

http://git-wip-us.apache.org/repos/asf/spark/blob/de333d12/sql/core/src/test/resources/sql-tests/inputs/order-by-nulls-ordering.sql
----------------------------------------------------------------------
diff --git a/sql/core/src/test/resources/sql-tests/inputs/order-by-nulls-ordering.sql b/sql/core/src/test/resources/sql-tests/inputs/order-by-nulls-ordering.sql
new file mode 100644
index 0000000..f7637b4
--- /dev/null
+++ b/sql/core/src/test/resources/sql-tests/inputs/order-by-nulls-ordering.sql
@@ -0,0 +1,83 @@
+-- Q1. testing window functions with order by
+create table spark_10747(col1 int, col2 int, col3 int) using parquet;
+
+-- Q2. insert to tables
+INSERT INTO spark_10747 VALUES (6, 12, 10), (6, 11, 4), (6, 9, 10), (6, 15, 8),
+(6, 15, 8), (6, 7, 4), (6, 7, 8), (6, 13, null), (6, 10, null);
+
+-- Q3. windowing with order by DESC NULLS LAST
+select col1, col2, col3, sum(col2)
+    over (partition by col1
+       order by col3 desc nulls last, col2
+       rows between 2 preceding and 2 following ) as sum_col2
+from spark_10747 where col1 = 6 order by sum_col2;
+
+-- Q4. windowing with order by DESC NULLS FIRST
+select col1, col2, col3, sum(col2)
+    over (partition by col1
+       order by col3 desc nulls first, col2
+       rows between 2 preceding and 2 following ) as sum_col2
+from spark_10747 where col1 = 6 order by sum_col2;
+
+-- Q5. windowing with order by ASC NULLS LAST
+select col1, col2, col3, sum(col2)
+    over (partition by col1
+       order by col3 asc nulls last, col2
+       rows between 2 preceding and 2 following ) as sum_col2
+from spark_10747 where col1 = 6 order by sum_col2;
+
+-- Q6. windowing with order by ASC NULLS FIRST
+select col1, col2, col3, sum(col2)
+    over (partition by col1
+       order by col3 asc nulls first, col2
+       rows between 2 preceding and 2 following ) as sum_col2
+from spark_10747 where col1 = 6 order by sum_col2;
+
+-- Q7. Regular query with ORDER BY ASC NULLS FIRST
+SELECT COL1, COL2, COL3 FROM spark_10747 ORDER BY COL3 ASC NULLS FIRST, COL2;
+
+-- Q8. Regular query with ORDER BY ASC NULLS LAST
+SELECT COL1, COL2, COL3 FROM spark_10747 ORDER BY COL3 NULLS LAST, COL2;
+
+-- Q9. Regular query with ORDER BY DESC NULLS FIRST
+SELECT COL1, COL2, COL3 FROM spark_10747 ORDER BY COL3 DESC NULLS FIRST, COL2;
+
+-- Q10. Regular query with ORDER BY DESC NULLS LAST
+SELECT COL1, COL2, COL3 FROM spark_10747 ORDER BY COL3 DESC NULLS LAST, COL2;
+
+-- drop the test table
+drop table spark_10747;
+
+-- Q11. mix datatype for ORDER BY NULLS FIRST|LAST
+create table spark_10747_mix(
+col1 string,
+col2 int,
+col3 double,
+col4 decimal(10,2),
+col5 decimal(20,1))
+using parquet;
+
+-- Q12. Insert to the table
+INSERT INTO spark_10747_mix VALUES
+('b', 2, 1.0, 1.00, 10.0),
+('d', 3, 2.0, 3.00, 0.0),
+('c', 3, 2.0, 2.00, 15.1),
+('d', 3, 0.0, 3.00, 1.0),
+(null, 3, 0.0, 3.00, 1.0),
+('d', 3, null, 4.00, 1.0),
+('a', 1, 1.0, 1.00, null),
+('c', 3, 2.0, 2.00, null);
+
+-- Q13. Regular query with 2 NULLS LAST columns
+select * from spark_10747_mix order by col1 nulls last, col5 nulls last;
+
+-- Q14. Regular query with 2 NULLS FIRST columns
+select * from spark_10747_mix order by col1 desc nulls first, col5 desc nulls first;
+
+-- Q15. Regular query with mixed NULLS FIRST|LAST
+select * from spark_10747_mix order by col5 desc nulls first, col3 desc nulls last;
+
+-- drop the test table
+drop table spark_10747_mix;
+
+

http://git-wip-us.apache.org/repos/asf/spark/blob/de333d12/sql/core/src/test/resources/sql-tests/inputs/orderby-nulls-ordering.sql
----------------------------------------------------------------------
diff --git a/sql/core/src/test/resources/sql-tests/inputs/orderby-nulls-ordering.sql b/sql/core/src/test/resources/sql-tests/inputs/orderby-nulls-ordering.sql
deleted file mode 100644
index f7637b4..0000000
--- a/sql/core/src/test/resources/sql-tests/inputs/orderby-nulls-ordering.sql
+++ /dev/null
@@ -1,83 +0,0 @@
--- Q1. testing window functions with order by
-create table spark_10747(col1 int, col2 int, col3 int) using parquet;
-
--- Q2. insert to tables
-INSERT INTO spark_10747 VALUES (6, 12, 10), (6, 11, 4), (6, 9, 10), (6, 15, 8),
-(6, 15, 8), (6, 7, 4), (6, 7, 8), (6, 13, null), (6, 10, null);
-
--- Q3. windowing with order by DESC NULLS LAST
-select col1, col2, col3, sum(col2)
-    over (partition by col1
-       order by col3 desc nulls last, col2
-       rows between 2 preceding and 2 following ) as sum_col2
-from spark_10747 where col1 = 6 order by sum_col2;
-
--- Q4. windowing with order by DESC NULLS FIRST
-select col1, col2, col3, sum(col2)
-    over (partition by col1
-       order by col3 desc nulls first, col2
-       rows between 2 preceding and 2 following ) as sum_col2
-from spark_10747 where col1 = 6 order by sum_col2;
-
--- Q5. windowing with order by ASC NULLS LAST
-select col1, col2, col3, sum(col2)
-    over (partition by col1
-       order by col3 asc nulls last, col2
-       rows between 2 preceding and 2 following ) as sum_col2
-from spark_10747 where col1 = 6 order by sum_col2;
-
--- Q6. windowing with order by ASC NULLS FIRST
-select col1, col2, col3, sum(col2)
-    over (partition by col1
-       order by col3 asc nulls first, col2
-       rows between 2 preceding and 2 following ) as sum_col2
-from spark_10747 where col1 = 6 order by sum_col2;
-
--- Q7. Regular query with ORDER BY ASC NULLS FIRST
-SELECT COL1, COL2, COL3 FROM spark_10747 ORDER BY COL3 ASC NULLS FIRST, COL2;
-
--- Q8. Regular query with ORDER BY ASC NULLS LAST
-SELECT COL1, COL2, COL3 FROM spark_10747 ORDER BY COL3 NULLS LAST, COL2;
-
--- Q9. Regular query with ORDER BY DESC NULLS FIRST
-SELECT COL1, COL2, COL3 FROM spark_10747 ORDER BY COL3 DESC NULLS FIRST, COL2;
-
--- Q10. Regular query with ORDER BY DESC NULLS LAST
-SELECT COL1, COL2, COL3 FROM spark_10747 ORDER BY COL3 DESC NULLS LAST, COL2;
-
--- drop the test table
-drop table spark_10747;
-
--- Q11. mix datatype for ORDER BY NULLS FIRST|LAST
-create table spark_10747_mix(
-col1 string,
-col2 int,
-col3 double,
-col4 decimal(10,2),
-col5 decimal(20,1))
-using parquet;
-
--- Q12. Insert to the table
-INSERT INTO spark_10747_mix VALUES
-('b', 2, 1.0, 1.00, 10.0),
-('d', 3, 2.0, 3.00, 0.0),
-('c', 3, 2.0, 2.00, 15.1),
-('d', 3, 0.0, 3.00, 1.0),
-(null, 3, 0.0, 3.00, 1.0),
-('d', 3, null, 4.00, 1.0),
-('a', 1, 1.0, 1.00, null),
-('c', 3, 2.0, 2.00, null);
-
--- Q13. Regular query with 2 NULLS LAST columns
-select * from spark_10747_mix order by col1 nulls last, col5 nulls last;
-
--- Q14. Regular query with 2 NULLS FIRST columns
-select * from spark_10747_mix order by col1 desc nulls first, col5 desc nulls first;
-
--- Q15. Regular query with mixed NULLS FIRST|LAST
-select * from spark_10747_mix order by col5 desc nulls first, col3 desc nulls last;
-
--- drop the test table
-drop table spark_10747_mix;
-
-

http://git-wip-us.apache.org/repos/asf/spark/blob/de333d12/sql/core/src/test/resources/sql-tests/results/order-by-nulls-ordering.sql.out
----------------------------------------------------------------------
diff --git a/sql/core/src/test/resources/sql-tests/results/order-by-nulls-ordering.sql.out b/sql/core/src/test/resources/sql-tests/results/order-by-nulls-ordering.sql.out
new file mode 100644
index 0000000..c1b63df
--- /dev/null
+++ b/sql/core/src/test/resources/sql-tests/results/order-by-nulls-ordering.sql.out
@@ -0,0 +1,254 @@
+-- Automatically generated by SQLQueryTestSuite
+-- Number of queries: 17
+
+
+-- !query 0
+create table spark_10747(col1 int, col2 int, col3 int) using parquet
+-- !query 0 schema
+struct<>
+-- !query 0 output
+
+
+
+-- !query 1
+INSERT INTO spark_10747 VALUES (6, 12, 10), (6, 11, 4), (6, 9, 10), (6, 15, 8),
+(6, 15, 8), (6, 7, 4), (6, 7, 8), (6, 13, null), (6, 10, null)
+-- !query 1 schema
+struct<>
+-- !query 1 output
+
+
+
+-- !query 2
+select col1, col2, col3, sum(col2)
+    over (partition by col1
+       order by col3 desc nulls last, col2
+       rows between 2 preceding and 2 following ) as sum_col2
+from spark_10747 where col1 = 6 order by sum_col2
+-- !query 2 schema
+struct<col1:int,col2:int,col3:int,sum_col2:bigint>
+-- !query 2 output
+6	9	10	28
+6	13	NULL	34
+6	10	NULL	41
+6	12	10	43
+6	15	8	55
+6	15	8	56
+6	11	4	56
+6	7	8	58
+6	7	4	58
+
+
+-- !query 3
+select col1, col2, col3, sum(col2)
+    over (partition by col1
+       order by col3 desc nulls first, col2
+       rows between 2 preceding and 2 following ) as sum_col2
+from spark_10747 where col1 = 6 order by sum_col2
+-- !query 3 schema
+struct<col1:int,col2:int,col3:int,sum_col2:bigint>
+-- !query 3 output
+6	10	NULL	32
+6	11	4	33
+6	13	NULL	44
+6	7	4	48
+6	9	10	51
+6	15	8	55
+6	12	10	56
+6	15	8	56
+6	7	8	58
+
+
+-- !query 4
+select col1, col2, col3, sum(col2)
+    over (partition by col1
+       order by col3 asc nulls last, col2
+       rows between 2 preceding and 2 following ) as sum_col2
+from spark_10747 where col1 = 6 order by sum_col2
+-- !query 4 schema
+struct<col1:int,col2:int,col3:int,sum_col2:bigint>
+-- !query 4 output
+6	7	4	25
+6	13	NULL	35
+6	11	4	40
+6	10	NULL	44
+6	7	8	55
+6	15	8	57
+6	15	8	58
+6	12	10	59
+6	9	10	61
+
+
+-- !query 5
+select col1, col2, col3, sum(col2)
+    over (partition by col1
+       order by col3 asc nulls first, col2
+       rows between 2 preceding and 2 following ) as sum_col2
+from spark_10747 where col1 = 6 order by sum_col2
+-- !query 5 schema
+struct<col1:int,col2:int,col3:int,sum_col2:bigint>
+-- !query 5 output
+6	10	NULL	30
+6	12	10	36
+6	13	NULL	41
+6	7	4	48
+6	9	10	51
+6	11	4	53
+6	7	8	55
+6	15	8	57
+6	15	8	58
+
+
+-- !query 6
+SELECT COL1, COL2, COL3 FROM spark_10747 ORDER BY COL3 ASC NULLS FIRST, COL2
+-- !query 6 schema
+struct<COL1:int,COL2:int,COL3:int>
+-- !query 6 output
+6	10	NULL
+6	13	NULL
+6	7	4
+6	11	4
+6	7	8
+6	15	8
+6	15	8
+6	9	10
+6	12	10
+
+
+-- !query 7
+SELECT COL1, COL2, COL3 FROM spark_10747 ORDER BY COL3 NULLS LAST, COL2
+-- !query 7 schema
+struct<COL1:int,COL2:int,COL3:int>
+-- !query 7 output
+6	7	4
+6	11	4
+6	7	8
+6	15	8
+6	15	8
+6	9	10
+6	12	10
+6	10	NULL
+6	13	NULL
+
+
+-- !query 8
+SELECT COL1, COL2, COL3 FROM spark_10747 ORDER BY COL3 DESC NULLS FIRST, COL2
+-- !query 8 schema
+struct<COL1:int,COL2:int,COL3:int>
+-- !query 8 output
+6	10	NULL
+6	13	NULL
+6	9	10
+6	12	10
+6	7	8
+6	15	8
+6	15	8
+6	7	4
+6	11	4
+
+
+-- !query 9
+SELECT COL1, COL2, COL3 FROM spark_10747 ORDER BY COL3 DESC NULLS LAST, COL2
+-- !query 9 schema
+struct<COL1:int,COL2:int,COL3:int>
+-- !query 9 output
+6	9	10
+6	12	10
+6	7	8
+6	15	8
+6	15	8
+6	7	4
+6	11	4
+6	10	NULL
+6	13	NULL
+
+
+-- !query 10
+drop table spark_10747
+-- !query 10 schema
+struct<>
+-- !query 10 output
+
+
+
+-- !query 11
+create table spark_10747_mix(
+col1 string,
+col2 int,
+col3 double,
+col4 decimal(10,2),
+col5 decimal(20,1))
+using parquet
+-- !query 11 schema
+struct<>
+-- !query 11 output
+
+
+
+-- !query 12
+INSERT INTO spark_10747_mix VALUES
+('b', 2, 1.0, 1.00, 10.0),
+('d', 3, 2.0, 3.00, 0.0),
+('c', 3, 2.0, 2.00, 15.1),
+('d', 3, 0.0, 3.00, 1.0),
+(null, 3, 0.0, 3.00, 1.0),
+('d', 3, null, 4.00, 1.0),
+('a', 1, 1.0, 1.00, null),
+('c', 3, 2.0, 2.00, null)
+-- !query 12 schema
+struct<>
+-- !query 12 output
+
+
+
+-- !query 13
+select * from spark_10747_mix order by col1 nulls last, col5 nulls last
+-- !query 13 schema
+struct<col1:string,col2:int,col3:double,col4:decimal(10,2),col5:decimal(20,1)>
+-- !query 13 output
+a	1	1.0	1	NULL
+b	2	1.0	1	10
+c	3	2.0	2	15.1
+c	3	2.0	2	NULL
+d	3	2.0	3	0
+d	3	0.0	3	1
+d	3	NULL	4	1
+NULL	3	0.0	3	1
+
+
+-- !query 14
+select * from spark_10747_mix order by col1 desc nulls first, col5 desc nulls first
+-- !query 14 schema
+struct<col1:string,col2:int,col3:double,col4:decimal(10,2),col5:decimal(20,1)>
+-- !query 14 output
+NULL	3	0.0	3	1
+d	3	0.0	3	1
+d	3	NULL	4	1
+d	3	2.0	3	0
+c	3	2.0	2	NULL
+c	3	2.0	2	15.1
+b	2	1.0	1	10
+a	1	1.0	1	NULL
+
+
+-- !query 15
+select * from spark_10747_mix order by col5 desc nulls first, col3 desc nulls last
+-- !query 15 schema
+struct<col1:string,col2:int,col3:double,col4:decimal(10,2),col5:decimal(20,1)>
+-- !query 15 output
+c	3	2.0	2	NULL
+a	1	1.0	1	NULL
+c	3	2.0	2	15.1
+b	2	1.0	1	10
+d	3	0.0	3	1
+NULL	3	0.0	3	1
+d	3	NULL	4	1
+d	3	2.0	3	0
+
+
+-- !query 16
+drop table spark_10747_mix
+-- !query 16 schema
+struct<>
+-- !query 16 output
+

http://git-wip-us.apache.org/repos/asf/spark/blob/de333d12/sql/core/src/test/resources/sql-tests/results/orderby-nulls-ordering.sql.out
----------------------------------------------------------------------
diff --git a/sql/core/src/test/resources/sql-tests/results/orderby-nulls-ordering.sql.out b/sql/core/src/test/resources/sql-tests/results/orderby-nulls-ordering.sql.out
deleted file mode 100644
index c1b63df..0000000
--- a/sql/core/src/test/resources/sql-tests/results/orderby-nulls-ordering.sql.out
+++ /dev/null
@@ -1,254 +0,0 @@
--- Automatically generated by SQLQueryTestSuite
--- Number of queries: 17
-
-
--- !query 0
-create table spark_10747(col1 int, col2 int, col3 int) using parquet
--- !query 0 schema
-struct<>
--- !query 0 output
-
-
-
--- !query 1
-INSERT INTO spark_10747 VALUES (6, 12, 10), (6, 11, 4), (6, 9, 10), (6, 15, 8),
-(6, 15, 8), (6, 7, 4), (6, 7, 8), (6, 13, null), (6, 10, null)
--- !query 1 schema
-struct<>
--- !query 1 output
-
-
-
--- !query 2
-select col1, col2, col3, sum(col2)
-    over (partition by col1
-       order by col3 desc nulls last, col2
-       rows between 2 preceding and 2 following ) as sum_col2
-from spark_10747 where col1 = 6 order by sum_col2
--- !query 2 schema
-struct<col1:int,col2:int,col3:int,sum_col2:bigint>
--- !query 2 output
-6	9	10	28
-6	13	NULL	34
-6	10	NULL	41
-6	12	10	43
-6	15	8	55
-6	15	8	56
-6	11	4	56
-6	7	8	58
-6	7	4	58
-
-
--- !query 3
-select col1, col2, col3, sum(col2)
-    over (partition by col1
-       order by col3 desc nulls first, col2
-       rows between 2 preceding and 2 following ) as sum_col2
-from spark_10747 where col1 = 6 order by sum_col2
--- !query 3 schema
-struct<col1:int,col2:int,col3:int,sum_col2:bigint>
--- !query 3 output
-6	10	NULL	32
-6	11	4	33
-6	13	NULL	44
-6	7	4	48
-6	9	10	51
-6	15	8	55
-6	12	10	56
-6	15	8	56
-6	7	8	58
-
-
--- !query 4
-select col1, col2, col3, sum(col2)
-    over (partition by col1
-       order by col3 asc nulls last, col2
-       rows between 2 preceding and 2 following ) as sum_col2
-from spark_10747 where col1 = 6 order by sum_col2
--- !query 4 schema
-struct<col1:int,col2:int,col3:int,sum_col2:bigint>
--- !query 4 output
-6	7	4	25
-6	13	NULL	35
-6	11	4	40
-6	10	NULL	44
-6	7	8	55
-6	15	8	57
-6	15	8	58
-6	12	10	59
-6	9	10	61
-
-
--- !query 5
-select col1, col2, col3, sum(col2)
-    over (partition by col1
-       order by col3 asc nulls first, col2
-       rows between 2 preceding and 2 following ) as sum_col2
-from spark_10747 where col1 = 6 order by sum_col2
--- !query 5 schema
-struct<col1:int,col2:int,col3:int,sum_col2:bigint>
--- !query 5 output
-6	10	NULL	30
-6	12	10	36
-6	13	NULL	41
-6	7	4	48
-6	9	10	51
-6	11	4	53
-6	7	8	55
-6	15	8	57
-6	15	8	58
-
-
--- !query 6
-SELECT COL1, COL2, COL3 FROM spark_10747 ORDER BY COL3 ASC NULLS FIRST, COL2
--- !query 6 schema
-struct<COL1:int,COL2:int,COL3:int>
--- !query 6 output
-6	10	NULL
-6	13	NULL
-6	7	4
-6	11	4
-6	7	8
-6	15	8
-6	15	8
-6	9	10
-6	12	10
-
-
--- !query 7
-SELECT COL1, COL2, COL3 FROM spark_10747 ORDER BY COL3 NULLS LAST, COL2
--- !query 7 schema
-struct<COL1:int,COL2:int,COL3:int>
--- !query 7 output
-6	7	4
-6	11	4
-6	7	8
-6	15	8
-6	15	8
-6	9	10
-6	12	10
-6	10	NULL
-6	13	NULL
-
-
--- !query 8
-SELECT COL1, COL2, COL3 FROM spark_10747 ORDER BY COL3 DESC NULLS FIRST, COL2
--- !query 8 schema
-struct<COL1:int,COL2:int,COL3:int>
--- !query 8 output
-6	10	NULL
-6	13	NULL
-6	9	10
-6	12	10
-6	7	8
-6	15	8
-6	15	8
-6	7	4
-6	11	4
-
-
--- !query 9
-SELECT COL1, COL2, COL3 FROM spark_10747 ORDER BY COL3 DESC NULLS LAST, COL2
--- !query 9 schema
-struct<COL1:int,COL2:int,COL3:int>
--- !query 9 output
-6	9	10
-6	12	10
-6	7	8
-6	15	8
-6	15	8
-6	7	4
-6	11	4
-6	10	NULL
-6	13	NULL
-
-
--- !query 10
-drop table spark_10747
--- !query 10 schema
-struct<>
--- !query 10 output
-
-
-
--- !query 11
-create table spark_10747_mix(
-col1 string,
-col2 int,
-col3 double,
-col4 decimal(10,2),
-col5 decimal(20,1))
-using parquet
--- !query 11 schema
-struct<>
--- !query 11 output
-
-
-
--- !query 12
-INSERT INTO spark_10747_mix VALUES
-('b', 2, 1.0, 1.00, 10.0),
-('d', 3, 2.0, 3.00, 0.0),
-('c', 3, 2.0, 2.00, 15.1),
-('d', 3, 0.0, 3.00, 1.0),
-(null, 3, 0.0, 3.00, 1.0),
-('d', 3, null, 4.00, 1.0),
-('a', 1, 1.0, 1.00, null),
-('c', 3, 2.0, 2.00, null)
--- !query 12 schema
-struct<>
--- !query 12 output
-
-
-
--- !query 13
-select * from spark_10747_mix order by col1 nulls last, col5 nulls last
--- !query 13 schema
-struct<col1:string,col2:int,col3:double,col4:decimal(10,2),col5:decimal(20,1)>
--- !query 13 output
-a	1	1.0	1	NULL
-b	2	1.0	1	10
-c	3	2.0	2	15.1
-c	3	2.0	2	NULL
-d	3	2.0	3	0
-d	3	0.0	3	1
-d	3	NULL	4	1
-NULL	3	0.0	3	1
-
-
--- !query 14
-select * from spark_10747_mix order by col1 desc nulls first, col5 desc nulls first
--- !query 14 schema
-struct<col1:string,col2:int,col3:double,col4:decimal(10,2),col5:decimal(20,1)>
--- !query 14 output
-NULL	3	0.0	3	1
-d	3	0.0	3	1
-d	3	NULL	4	1
-d	3	2.0	3	0
-c	3	2.0	2	NULL
-c	3	2.0	2	15.1
-b	2	1.0	1	10
-a	1	1.0	1	NULL
-
-
--- !query 15
-select * from spark_10747_mix order by col5 desc nulls first, col3 desc nulls last
--- !query 15 schema
-struct<col1:string,col2:int,col3:double,col4:decimal(10,2),col5:decimal(20,1)>
--- !query 15 output
-c	3	2.0	2	NULL
-a	1	1.0	1	NULL
-c	3	2.0	2	15.1
-b	2	1.0	1	10
-d	3	0.0	3	1
-NULL	3	0.0	3	1
-d	3	NULL	4	1
-d	3	2.0	3	0
-
-
--- !query 16
-drop table spark_10747_mix
--- !query 16 schema
-struct<>
--- !query 16 output
-

http://git-wip-us.apache.org/repos/asf/spark/blob/de333d12/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
index 2c60a7d..16cc368 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
@@ -326,6 +326,24 @@ class DataFrameSuite extends QueryTest with SharedSQLContext {
       Row(6))
   }
 
+  test("sorting with null ordering") {
+    val data = Seq[java.lang.Integer](2, 1, null).toDF("key")
+
+    checkAnswer(data.orderBy('key.asc), Row(null) :: Row(1) :: Row(2) :: Nil)
+    checkAnswer(data.orderBy(asc("key")), Row(null) :: Row(1) :: Row(2) :: Nil)
+    checkAnswer(data.orderBy('key.asc_nulls_first), Row(null) :: Row(1) :: Row(2) :: Nil)
+    checkAnswer(data.orderBy(asc_nulls_first("key")), Row(null) :: Row(1) :: Row(2) :: Nil)
+    checkAnswer(data.orderBy('key.asc_nulls_last), Row(1) :: Row(2) :: Row(null) :: Nil)
+    checkAnswer(data.orderBy(asc_nulls_last("key")), Row(1) :: Row(2) :: Row(null) :: Nil)
+
+    checkAnswer(data.orderBy('key.desc), Row(2) :: Row(1) :: Row(null) :: Nil)
+    checkAnswer(data.orderBy(desc("key")), Row(2) :: Row(1) :: Row(null) :: Nil)
+    checkAnswer(data.orderBy('key.desc_nulls_first), Row(null) :: Row(2) :: Row(1) :: Nil)
+    checkAnswer(data.orderBy(desc_nulls_first("key")), Row(null) :: Row(2) :: Row(1) :: Nil)
+    checkAnswer(data.orderBy('key.desc_nulls_last), Row(2) :: Row(1) :: Row(null) :: Nil)
+    checkAnswer(data.orderBy(desc_nulls_last("key")), Row(2) :: Row(1) :: Row(null) :: Nil)
+  }
+
   test("global sorting") {
     checkAnswer(
       testData2.orderBy('a.asc, 'b.asc),


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org