You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by jc...@apache.org on 2016/03/02 12:00:12 UTC

[01/18] hive git commit: HIVE-12994: Implement support for NULLS FIRST/NULLS LAST (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Repository: hive
Updated Branches:
  refs/heads/master 01fd68443 -> 9350b6934


http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/windowing_order_null.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/windowing_order_null.q.out b/ql/src/test/results/clientpositive/windowing_order_null.q.out
new file mode 100644
index 0000000..ca18b42
--- /dev/null
+++ b/ql/src/test/results/clientpositive/windowing_order_null.q.out
@@ -0,0 +1,183 @@
+PREHOOK: query: drop table over10k
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table over10k
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: create table over10k(
+           t tinyint,
+           si smallint,
+           i int,
+           b bigint,
+           f float,
+           d double,
+           bo boolean,
+           s string,
+	   ts timestamp, 
+           dec decimal,  
+           bin binary)
+       row format delimited
+       fields terminated by '|'
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@over10k
+POSTHOOK: query: create table over10k(
+           t tinyint,
+           si smallint,
+           i int,
+           b bigint,
+           f float,
+           d double,
+           bo boolean,
+           s string,
+	   ts timestamp, 
+           dec decimal,  
+           bin binary)
+       row format delimited
+       fields terminated by '|'
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@over10k
+PREHOOK: query: load data local inpath '../../data/files/over10k' into table over10k
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: default@over10k
+POSTHOOK: query: load data local inpath '../../data/files/over10k' into table over10k
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: default@over10k
+PREHOOK: query: load data local inpath '../../data/files/over4_null' into table over10k
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: default@over10k
+POSTHOOK: query: load data local inpath '../../data/files/over4_null' into table over10k
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: default@over10k
+PREHOOK: query: select i, s, b, sum(b) over (partition by i order by s nulls last,b rows unbounded preceding) from over10k limit 10
+PREHOOK: type: QUERY
+PREHOOK: Input: default@over10k
+#### A masked pattern was here ####
+POSTHOOK: query: select i, s, b, sum(b) over (partition by i order by s nulls last,b rows unbounded preceding) from over10k limit 10
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@over10k
+#### A masked pattern was here ####
+NULL	alice ichabod	NULL	NULL
+NULL	NULL	NULL	NULL
+65534	calvin miller	NULL	NULL
+65534	NULL	NULL	NULL
+65536	alice ichabod	4294967441	4294967441
+65536	alice robinson	4294967476	8589934917
+65536	bob robinson	4294967349	12884902266
+65536	calvin thompson	4294967336	17179869602
+65536	david johnson	4294967490	21474837092
+65536	david laertes	4294967431	25769804523
+PREHOOK: query: select d, s, f, sum(f) over (partition by d order by s,f desc nulls first rows unbounded preceding) from over10k limit 10
+PREHOOK: type: QUERY
+PREHOOK: Input: default@over10k
+#### A masked pattern was here ####
+POSTHOOK: query: select d, s, f, sum(f) over (partition by d order by s,f desc nulls first rows unbounded preceding) from over10k limit 10
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@over10k
+#### A masked pattern was here ####
+NULL	alice ichabod	NULL	NULL
+NULL	calvin miller	NULL	NULL
+0.01	NULL	NULL	NULL
+0.01	NULL	NULL	NULL
+0.01	calvin miller	8.39	8.390000343322754
+0.02	NULL	NULL	NULL
+0.02	holly polk	5.29	5.289999961853027
+0.02	wendy quirinius	25.5	30.789999961853027
+0.02	yuri laertes	37.59	68.38000011444092
+0.03	nick steinbeck	79.24	79.23999786376953
+PREHOOK: query: select ts, s, f, sum(f) over (partition by ts order by f asc nulls first range between current row and unbounded following) from over10k limit 10
+PREHOOK: type: QUERY
+PREHOOK: Input: default@over10k
+#### A masked pattern was here ####
+POSTHOOK: query: select ts, s, f, sum(f) over (partition by ts order by f asc nulls first range between current row and unbounded following) from over10k limit 10
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@over10k
+#### A masked pattern was here ####
+2013-03-01 09:11:58.70307	NULL	NULL	1276.850001335144
+2013-03-01 09:11:58.70307	gabriella xylophone	3.17	1276.850001335144
+2013-03-01 09:11:58.70307	calvin brown	10.89	1273.68000125885
+2013-03-01 09:11:58.70307	jessica laertes	14.54	1262.7900009155273
+2013-03-01 09:11:58.70307	yuri allen	14.78	1248.2500009536743
+2013-03-01 09:11:58.70307	tom johnson	17.85	1233.4700012207031
+2013-03-01 09:11:58.70307	bob ovid	20.61	1215.6200008392334
+2013-03-01 09:11:58.70307	fred nixon	28.69	1195.0100002288818
+2013-03-01 09:11:58.70307	oscar brown	29.22	1166.3199996948242
+2013-03-01 09:11:58.70307	calvin laertes	31.17	1137.1000003814697
+PREHOOK: query: select t, s, d, avg(d) over (partition by t order by s,d desc nulls first rows between 5 preceding and 5 following) from over10k limit 10
+PREHOOK: type: QUERY
+PREHOOK: Input: default@over10k
+#### A masked pattern was here ####
+POSTHOOK: query: select t, s, d, avg(d) over (partition by t order by s,d desc nulls first rows between 5 preceding and 5 following) from over10k limit 10
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@over10k
+#### A masked pattern was here ####
+-3	alice allen	29.44	33.20166666666666
+-3	alice davidson	31.52	30.741428571428568
+-3	alice falkner	49.8	27.742499999999996
+-3	alice king	41.5	26.706666666666663
+-3	alice king	30.76	26.306999999999995
+-3	alice xylophone	16.19	24.458181818181814
+-3	bob ellison	15.98	25.029090909090908
+-3	bob falkner	6.75	24.216363636363635
+-3	bob ichabod	18.42	20.173636363636362
+-3	bob johnson	22.71	16.431818181818176
+PREHOOK: query: select ts, s, sum(i) over(partition by ts order by s nulls last) from over10k limit 10 offset 3
+PREHOOK: type: QUERY
+PREHOOK: Input: default@over10k
+#### A masked pattern was here ####
+POSTHOOK: query: select ts, s, sum(i) over(partition by ts order by s nulls last) from over10k limit 10 offset 3
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@over10k
+#### A masked pattern was here ####
+2013-03-01 09:11:58.70307	calvin steinbeck	262874
+2013-03-01 09:11:58.70307	david falkner	328506
+2013-03-01 09:11:58.70307	fred nixon	394118
+2013-03-01 09:11:58.70307	fred zipper	459719
+2013-03-01 09:11:58.70307	gabriella van buren	525334
+2013-03-01 09:11:58.70307	gabriella xylophone	591058
+2013-03-01 09:11:58.70307	jessica laertes	656771
+2013-03-01 09:11:58.70307	jessica polk	722558
+2013-03-01 09:11:58.70307	katie king	788310
+2013-03-01 09:11:58.70307	katie white	853920
+PREHOOK: query: select s, i, round(sum(d) over (partition by s order by i desc nulls last) , 3) from over10k limit 5
+PREHOOK: type: QUERY
+PREHOOK: Input: default@over10k
+#### A masked pattern was here ####
+POSTHOOK: query: select s, i, round(sum(d) over (partition by s order by i desc nulls last) , 3) from over10k limit 5
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@over10k
+#### A masked pattern was here ####
+NULL	65536	0.02
+NULL	65534	0.03
+NULL	NULL	0.04
+alice allen	65758	23.59
+alice allen	65720	43.98
+PREHOOK: query: select s, i, round(avg(d) over (partition by s order by i desc nulls last) / 10.0 , 3) from over10k limit 5
+PREHOOK: type: QUERY
+PREHOOK: Input: default@over10k
+#### A masked pattern was here ####
+POSTHOOK: query: select s, i, round(avg(d) over (partition by s order by i desc nulls last) / 10.0 , 3) from over10k limit 5
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@over10k
+#### A masked pattern was here ####
+NULL	65536	0.002
+NULL	65534	0.002
+NULL	NULL	0.001
+alice allen	65758	2.359
+alice allen	65720	2.199
+PREHOOK: query: select s, i, round((avg(d) over  w1 + 10.0) - (avg(d) over w1 - 10.0),3) from over10k window w1 as (partition by s order by i nulls last) limit 5
+PREHOOK: type: QUERY
+PREHOOK: Input: default@over10k
+#### A masked pattern was here ####
+POSTHOOK: query: select s, i, round((avg(d) over  w1 + 10.0) - (avg(d) over w1 - 10.0),3) from over10k window w1 as (partition by s order by i nulls last) limit 5
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@over10k
+#### A masked pattern was here ####
+NULL	65534	20.0
+NULL	65536	20.0
+NULL	NULL	20.0
+alice allen	65545	20.0
+alice allen	65557	20.0

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/windowing_streaming.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/windowing_streaming.q.out b/ql/src/test/results/clientpositive/windowing_streaming.q.out
index d3226a1..27dd96f 100644
--- a/ql/src/test/results/clientpositive/windowing_streaming.q.out
+++ b/ql/src/test/results/clientpositive/windowing_streaming.q.out
@@ -84,7 +84,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:
@@ -155,7 +155,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:
@@ -324,7 +324,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col5
+                  order by: _col5 ASC NULLS FIRST
                   partition by: _col0
                   raw input shape:
                   window functions:

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/serde/if/serde.thrift
----------------------------------------------------------------------
diff --git a/serde/if/serde.thrift b/serde/if/serde.thrift
index 40d5f47..0b3804d 100644
--- a/serde/if/serde.thrift
+++ b/serde/if/serde.thrift
@@ -30,6 +30,7 @@ const string SERIALIZATION_NULL_FORMAT = "serialization.null.format"
 const string SERIALIZATION_ESCAPE_CRLF = "serialization.escape.crlf"
 const string SERIALIZATION_LAST_COLUMN_TAKES_REST = "serialization.last.column.takes.rest"
 const string SERIALIZATION_SORT_ORDER = "serialization.sort.order"
+const string SERIALIZATION_NULL_POSITION = "serialization.null.position";
 const string SERIALIZATION_USE_JSON_OBJECTS = "serialization.use.json.object"
 const string SERIALIZATION_ENCODING = "serialization.encoding"
 

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/serde/src/gen/thrift/gen-cpp/serde_constants.cpp
----------------------------------------------------------------------
diff --git a/serde/src/gen/thrift/gen-cpp/serde_constants.cpp b/serde/src/gen/thrift/gen-cpp/serde_constants.cpp
index 243d3b8..75701e2 100644
--- a/serde/src/gen/thrift/gen-cpp/serde_constants.cpp
+++ b/serde/src/gen/thrift/gen-cpp/serde_constants.cpp
@@ -27,6 +27,8 @@ serdeConstants::serdeConstants() {
 
   SERIALIZATION_SORT_ORDER = "serialization.sort.order";
 
+  SERIALIZATION_NULL_POSITION = "serialization.null.position";
+
   SERIALIZATION_USE_JSON_OBJECTS = "serialization.use.json.object";
 
   SERIALIZATION_ENCODING = "serialization.encoding";

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/serde/src/gen/thrift/gen-cpp/serde_constants.h
----------------------------------------------------------------------
diff --git a/serde/src/gen/thrift/gen-cpp/serde_constants.h b/serde/src/gen/thrift/gen-cpp/serde_constants.h
index 3566ead..6d85928 100644
--- a/serde/src/gen/thrift/gen-cpp/serde_constants.h
+++ b/serde/src/gen/thrift/gen-cpp/serde_constants.h
@@ -23,6 +23,7 @@ class serdeConstants {
   std::string SERIALIZATION_ESCAPE_CRLF;
   std::string SERIALIZATION_LAST_COLUMN_TAKES_REST;
   std::string SERIALIZATION_SORT_ORDER;
+  std::string SERIALIZATION_NULL_POSITION;
   std::string SERIALIZATION_USE_JSON_OBJECTS;
   std::string SERIALIZATION_ENCODING;
   std::string FIELD_DELIM;

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde/serdeConstants.java
----------------------------------------------------------------------
diff --git a/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde/serdeConstants.java b/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde/serdeConstants.java
index 8b3eeb7..04ed8f5 100644
--- a/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde/serdeConstants.java
+++ b/serde/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/serde/serdeConstants.java
@@ -52,6 +52,8 @@ public class serdeConstants {
 
   public static final String SERIALIZATION_SORT_ORDER = "serialization.sort.order";
 
+  public static final String SERIALIZATION_NULL_SORT_ORDER = "serialization.sort.order.null";
+
   public static final String SERIALIZATION_USE_JSON_OBJECTS = "serialization.use.json.object";
 
   public static final String SERIALIZATION_ENCODING = "serialization.encoding";

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/serde/src/gen/thrift/gen-php/org/apache/hadoop/hive/serde/Types.php
----------------------------------------------------------------------
diff --git a/serde/src/gen/thrift/gen-php/org/apache/hadoop/hive/serde/Types.php b/serde/src/gen/thrift/gen-php/org/apache/hadoop/hive/serde/Types.php
index 8370698..0bc6dd7 100644
--- a/serde/src/gen/thrift/gen-php/org/apache/hadoop/hive/serde/Types.php
+++ b/serde/src/gen/thrift/gen-php/org/apache/hadoop/hive/serde/Types.php
@@ -26,6 +26,7 @@ final class Constant extends \Thrift\Type\TConstant {
   static protected $SERIALIZATION_ESCAPE_CRLF;
   static protected $SERIALIZATION_LAST_COLUMN_TAKES_REST;
   static protected $SERIALIZATION_SORT_ORDER;
+  static protected $SERIALIZATION_NULL_POSITION;
   static protected $SERIALIZATION_USE_JSON_OBJECTS;
   static protected $SERIALIZATION_ENCODING;
   static protected $FIELD_DELIM;
@@ -97,6 +98,10 @@ final class Constant extends \Thrift\Type\TConstant {
     return "serialization.sort.order";
   }
 
+  static protected function init_SERIALIZATION_NULL_POSITION() {
+    return "serialization.null.position";
+  }
+
   static protected function init_SERIALIZATION_USE_JSON_OBJECTS() {
     return "serialization.use.json.object";
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/serde/src/gen/thrift/gen-py/org_apache_hadoop_hive_serde/constants.py
----------------------------------------------------------------------
diff --git a/serde/src/gen/thrift/gen-py/org_apache_hadoop_hive_serde/constants.py b/serde/src/gen/thrift/gen-py/org_apache_hadoop_hive_serde/constants.py
index 6ef3bcf..7939791 100644
--- a/serde/src/gen/thrift/gen-py/org_apache_hadoop_hive_serde/constants.py
+++ b/serde/src/gen/thrift/gen-py/org_apache_hadoop_hive_serde/constants.py
@@ -17,6 +17,7 @@ SERIALIZATION_NULL_FORMAT = "serialization.null.format"
 SERIALIZATION_ESCAPE_CRLF = "serialization.escape.crlf"
 SERIALIZATION_LAST_COLUMN_TAKES_REST = "serialization.last.column.takes.rest"
 SERIALIZATION_SORT_ORDER = "serialization.sort.order"
+SERIALIZATION_NULL_POSITION = "serialization.null.position"
 SERIALIZATION_USE_JSON_OBJECTS = "serialization.use.json.object"
 SERIALIZATION_ENCODING = "serialization.encoding"
 FIELD_DELIM = "field.delim"

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/serde/src/gen/thrift/gen-rb/serde_constants.rb
----------------------------------------------------------------------
diff --git a/serde/src/gen/thrift/gen-rb/serde_constants.rb b/serde/src/gen/thrift/gen-rb/serde_constants.rb
index f98441b..d09e3c2 100644
--- a/serde/src/gen/thrift/gen-rb/serde_constants.rb
+++ b/serde/src/gen/thrift/gen-rb/serde_constants.rb
@@ -23,6 +23,8 @@ SERIALIZATION_LAST_COLUMN_TAKES_REST = %q"serialization.last.column.takes.rest"
 
 SERIALIZATION_SORT_ORDER = %q"serialization.sort.order"
 
+SERIALIZATION_NULL_POSITION = %q"serialization.null.position"
+
 SERIALIZATION_USE_JSON_OBJECTS = %q"serialization.use.json.object"
 
 SERIALIZATION_ENCODING = %q"serialization.encoding"

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/serde/src/java/org/apache/hadoop/hive/serde2/binarysortable/BinarySortableSerDe.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/binarysortable/BinarySortableSerDe.java b/serde/src/java/org/apache/hadoop/hive/serde2/binarysortable/BinarySortableSerDe.java
index 144ea5a..5e119d7 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/binarysortable/BinarySortableSerDe.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/binarysortable/BinarySortableSerDe.java
@@ -28,8 +28,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.common.type.HiveIntervalDayTime;
@@ -92,16 +90,18 @@ import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * BinarySortableSerDe can be used to write data in a way that the data can be
  * compared byte-by-byte with the same order.
  *
- * The data format: NULL: a single byte \0 NON-NULL Primitives: ALWAYS prepend a
- * single byte \1, and then: Boolean: FALSE = \1, TRUE = \2 Byte: flip the
- * sign-bit to make sure negative comes before positive Short: flip the sign-bit
- * to make sure negative comes before positive Int: flip the sign-bit to make
- * sure negative comes before positive Long: flip the sign-bit to make sure
+ * The data format: NULL: a single byte (\0 or \1, check below) NON-NULL Primitives:
+ * ALWAYS prepend a single byte (\0 or \1), and then: Boolean: FALSE = \1, TRUE = \2
+ * Byte: flip the sign-bit to make sure negative comes before positive Short: flip the
+ * sign-bit to make sure negative comes before positive Int: flip the sign-bit to
+ * make sure negative comes before positive Long: flip the sign-bit to make sure
  * negative comes before positive Double: flip the sign-bit for positive double,
  * and all bits for negative double values String: NULL-terminated UTF-8 string,
  * with NULL escaped to \1 \1, and \1 escaped to \1 \2 NON-NULL Complex Types:
@@ -115,14 +115,23 @@ import org.apache.hadoop.io.Writable;
  * field should be sorted ascendingly, and "-" means descendingly. The sub
  * fields in the same top-level field will have the same sort order.
  *
+ * This SerDe takes an additional parameter SERIALIZATION_NULL_SORT_ORDER which is a
+ * string containing only "a" and "z". The length of the string should equal to
+ * the number of fields in the top-level struct for serialization. "a" means that
+ * NULL should come first (thus, single byte is \0 for ascending order, \1
+ * for descending order), while "z" means that NULL should come last (thus, single
+ * byte is \1 for ascending order, \0 for descending order).
  */
 @SerDeSpec(schemaProps = {
     serdeConstants.LIST_COLUMNS, serdeConstants.LIST_COLUMN_TYPES,
-    serdeConstants.SERIALIZATION_SORT_ORDER})
+    serdeConstants.SERIALIZATION_SORT_ORDER, serdeConstants.SERIALIZATION_NULL_SORT_ORDER})
 public class BinarySortableSerDe extends AbstractSerDe {
 
   public static final Logger LOG = LoggerFactory.getLogger(BinarySortableSerDe.class.getName());
 
+  public static final byte ZERO = (byte) 0;
+  public static final byte ONE = (byte) 1;
+
   List<String> columnNames;
   List<TypeInfo> columnTypes;
 
@@ -130,6 +139,8 @@ public class BinarySortableSerDe extends AbstractSerDe {
   StructObjectInspector rowObjectInspector;
 
   boolean[] columnSortOrderIsDesc;
+  byte[] columnNullMarker;
+  byte[] columnNotNullMarker;
 
   public static Charset decimalCharSet = Charset.forName("US-ASCII");
 
@@ -170,6 +181,37 @@ public class BinarySortableSerDe extends AbstractSerDe {
       columnSortOrderIsDesc[i] = (columnSortOrder != null && columnSortOrder
           .charAt(i) == '-');
     }
+
+    // Null first/last
+    String columnNullOrder = tbl
+        .getProperty(serdeConstants.SERIALIZATION_NULL_SORT_ORDER);
+    columnNullMarker = new byte[columnNames.size()];
+    columnNotNullMarker = new byte[columnNames.size()];
+    for (int i = 0; i < columnSortOrderIsDesc.length; i++) {
+      if (columnSortOrderIsDesc[i]) {
+        // Descending
+        if (columnNullOrder != null && columnNullOrder.charAt(i) == 'a') {
+          // Null first
+          columnNullMarker[i] = ONE;
+          columnNotNullMarker[i] = ZERO;
+        } else {
+          // Null last (default for descending order)
+          columnNullMarker[i] = ZERO;
+          columnNotNullMarker[i] = ONE;
+        }
+      } else {
+        // Ascending
+        if (columnNullOrder != null && columnNullOrder.charAt(i) == 'z') {
+          // Null last
+          columnNullMarker[i] = ONE;
+          columnNotNullMarker[i] = ZERO;
+        } else {
+          // Null first (default for ascending order)
+          columnNullMarker[i] = ZERO;
+          columnNotNullMarker[i] = ONE;
+        }
+      }
+    }
   }
 
   @Override
@@ -193,7 +235,7 @@ public class BinarySortableSerDe extends AbstractSerDe {
     try {
       for (int i = 0; i < columnNames.size(); i++) {
         row.set(i, deserialize(inputByteBuffer, columnTypes.get(i),
-            columnSortOrderIsDesc[i], row.get(i)));
+            columnSortOrderIsDesc[i], columnNullMarker[i], columnNotNullMarker[i], row.get(i)));
       }
     } catch (IOException e) {
       throw new SerDeException(e);
@@ -203,14 +245,14 @@ public class BinarySortableSerDe extends AbstractSerDe {
   }
 
   static Object deserialize(InputByteBuffer buffer, TypeInfo type,
-      boolean invert, Object reuse) throws IOException {
+      boolean invert, byte nullMarker, byte notNullMarker, Object reuse) throws IOException {
 
     // Is this field a null?
     byte isNull = buffer.read(invert);
-    if (isNull == 0) {
+    if (isNull == nullMarker) {
       return null;
     }
-    assert (isNull == 1);
+    assert (isNull == notNullMarker);
 
     switch (type.getCategory()) {
     case PRIMITIVE: {
@@ -475,7 +517,7 @@ public class BinarySortableSerDe extends AbstractSerDe {
         if (size == r.size()) {
           r.add(null);
         }
-        r.set(size, deserialize(buffer, etype, invert, r.get(size)));
+        r.set(size, deserialize(buffer, etype, invert, nullMarker, notNullMarker, r.get(size)));
         size++;
       }
       // Remove additional elements if the list is reused
@@ -506,8 +548,8 @@ public class BinarySortableSerDe extends AbstractSerDe {
         }
         // \1 followed by each key and then each value
         assert (more == 1);
-        Object k = deserialize(buffer, ktype, invert, null);
-        Object v = deserialize(buffer, vtype, invert, null);
+        Object k = deserialize(buffer, ktype, invert, nullMarker, notNullMarker, null);
+        Object v = deserialize(buffer, vtype, invert, nullMarker, notNullMarker, null);
         r.put(k, v);
       }
       return r;
@@ -527,7 +569,7 @@ public class BinarySortableSerDe extends AbstractSerDe {
       // Read one field by one field
       for (int eid = 0; eid < size; eid++) {
         r
-            .set(eid, deserialize(buffer, fieldTypes.get(eid), invert, r
+            .set(eid, deserialize(buffer, fieldTypes.get(eid), invert, nullMarker, notNullMarker, r
             .get(eid)));
       }
       return r;
@@ -540,7 +582,7 @@ public class BinarySortableSerDe extends AbstractSerDe {
       byte tag = buffer.read(invert);
       r.setTag(tag);
       r.setObject(deserialize(buffer, utype.getAllUnionObjectTypeInfos().get(tag),
-          invert, null));
+          invert, nullMarker, notNullMarker, null));
       return r;
     }
     default: {
@@ -626,7 +668,8 @@ public class BinarySortableSerDe extends AbstractSerDe {
 
     for (int i = 0; i < columnNames.size(); i++) {
       serialize(output, soi.getStructFieldData(obj, fields.get(i)),
-          fields.get(i).getFieldObjectInspector(), columnSortOrderIsDesc[i]);
+          fields.get(i).getFieldObjectInspector(), columnSortOrderIsDesc[i],
+          columnNullMarker[i], columnNotNullMarker[i]);
     }
 
     serializeBytesWritable.set(output.getData(), 0, output.getLength());
@@ -641,14 +684,14 @@ public class BinarySortableSerDe extends AbstractSerDe {
   }
 
   static void serialize(ByteStream.Output buffer, Object o, ObjectInspector oi,
-      boolean invert) throws SerDeException {
+      boolean invert, byte nullMarker, byte notNullMarker) throws SerDeException {
     // Is this field a null?
     if (o == null) {
-      writeByte(buffer, (byte) 0, invert);
+      writeByte(buffer, nullMarker, invert);
       return;
     }
     // This field is not a null.
-    writeByte(buffer, (byte) 1, invert);
+    writeByte(buffer, notNullMarker, invert);
 
     switch (oi.getCategory()) {
     case PRIMITIVE: {
@@ -786,7 +829,7 @@ public class BinarySortableSerDe extends AbstractSerDe {
       int size = loi.getListLength(o);
       for (int eid = 0; eid < size; eid++) {
         writeByte(buffer, (byte) 1, invert);
-        serialize(buffer, loi.getListElement(o, eid), eoi, invert);
+        serialize(buffer, loi.getListElement(o, eid), eoi, invert, nullMarker, notNullMarker);
       }
       // and \0 to terminate
       writeByte(buffer, (byte) 0, invert);
@@ -801,8 +844,8 @@ public class BinarySortableSerDe extends AbstractSerDe {
       Map<?, ?> map = moi.getMap(o);
       for (Map.Entry<?, ?> entry : map.entrySet()) {
         writeByte(buffer, (byte) 1, invert);
-        serialize(buffer, entry.getKey(), koi, invert);
-        serialize(buffer, entry.getValue(), voi, invert);
+        serialize(buffer, entry.getKey(), koi, invert, nullMarker, notNullMarker);
+        serialize(buffer, entry.getValue(), voi, invert, nullMarker, notNullMarker);
       }
       // and \0 to terminate
       writeByte(buffer, (byte) 0, invert);
@@ -814,7 +857,7 @@ public class BinarySortableSerDe extends AbstractSerDe {
 
       for (int i = 0; i < fields.size(); i++) {
         serialize(buffer, soi.getStructFieldData(o, fields.get(i)), fields.get(
-            i).getFieldObjectInspector(), invert);
+            i).getFieldObjectInspector(), invert, nullMarker, notNullMarker);
       }
       return;
     }
@@ -823,7 +866,7 @@ public class BinarySortableSerDe extends AbstractSerDe {
       byte tag = uoi.getTag(o);
       writeByte(buffer, tag, invert);
       serialize(buffer, uoi.getField(o), uoi.getObjectInspectors().get(tag),
-          invert);
+          invert, nullMarker, notNullMarker);
       return;
     }
     default: {
@@ -971,13 +1014,24 @@ public class BinarySortableSerDe extends AbstractSerDe {
   }
 
   public static void serializeStruct(Output byteStream, Object[] fieldData,
-      List<ObjectInspector> fieldOis, boolean[] sortableSortOrders) throws SerDeException {
+      List<ObjectInspector> fieldOis, boolean[] sortableSortOrders,
+      byte[] nullMarkers, byte[] notNullMarkers) throws SerDeException {
     for (int i = 0; i < fieldData.length; i++) {
-      serialize(byteStream, fieldData[i], fieldOis.get(i), sortableSortOrders[i]);
+      serialize(byteStream, fieldData[i], fieldOis.get(i), sortableSortOrders[i],
+              nullMarkers[i], notNullMarkers[i]);
     }
   }
 
   public boolean[] getSortOrders() {
     return columnSortOrderIsDesc;
   }
+
+  public byte[] getNullMarkers() {
+    return columnNullMarker;
+  }
+
+  public byte[] getNotNullMarkers() {
+    return columnNotNullMarker;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/serde/src/java/org/apache/hadoop/hive/serde2/binarysortable/BinarySortableSerDeWithEndPrefix.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/binarysortable/BinarySortableSerDeWithEndPrefix.java b/serde/src/java/org/apache/hadoop/hive/serde2/binarysortable/BinarySortableSerDeWithEndPrefix.java
index ec43ae3..73e20a8 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/binarysortable/BinarySortableSerDeWithEndPrefix.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/binarysortable/BinarySortableSerDeWithEndPrefix.java
@@ -28,7 +28,7 @@ public class BinarySortableSerDeWithEndPrefix extends BinarySortableSerDe {
   public static void serializeStruct(Output byteStream, Object[] fieldData,
       List<ObjectInspector> fieldOis, boolean endPrefix) throws SerDeException {
     for (int i = 0; i < fieldData.length; i++) {
-      serialize(byteStream, fieldData[i], fieldOis.get(i), false);
+      serialize(byteStream, fieldData[i], fieldOis.get(i), false, ZERO, ONE);
     }
     if (endPrefix) {
       if (fieldData[fieldData.length-1]!=null) {

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/serde/src/java/org/apache/hadoop/hive/serde2/binarysortable/fast/BinarySortableSerializeWrite.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/binarysortable/fast/BinarySortableSerializeWrite.java b/serde/src/java/org/apache/hadoop/hive/serde2/binarysortable/fast/BinarySortableSerializeWrite.java
index 7456725..62bcaa5 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/binarysortable/fast/BinarySortableSerializeWrite.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/binarysortable/fast/BinarySortableSerializeWrite.java
@@ -23,8 +23,6 @@ import java.sql.Date;
 import java.sql.Timestamp;
 import java.util.Arrays;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.common.type.HiveChar;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.common.type.HiveIntervalDayTime;
@@ -32,10 +30,12 @@ import org.apache.hadoop.hive.common.type.HiveIntervalYearMonth;
 import org.apache.hadoop.hive.common.type.HiveVarchar;
 import org.apache.hadoop.hive.serde2.ByteStream.Output;
 import org.apache.hadoop.hive.serde2.binarysortable.BinarySortableSerDe;
+import org.apache.hadoop.hive.serde2.fast.SerializeWrite;
 import org.apache.hadoop.hive.serde2.io.DateWritable;
 import org.apache.hadoop.hive.serde2.io.TimestampWritable;
-import org.apache.hadoop.hive.serde2.fast.SerializeWrite;
 import org.apache.hive.common.util.DateUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /*
  * Directly serialize, field-by-field, the BinarySortable format.
@@ -49,6 +49,9 @@ public final class BinarySortableSerializeWrite implements SerializeWrite {
 
   // The sort order (ascending/descending) for each field. Set to true when descending (invert).
   private boolean[] columnSortOrderIsDesc;
+  // Null first/last
+  private byte[] columnNullMarker;
+  private byte[] columnNotNullMarker;
 
   // Which field we are on.  We start with -1 to be consistent in style with
   // BinarySortableDeserializeRead.
@@ -58,20 +61,28 @@ public final class BinarySortableSerializeWrite implements SerializeWrite {
 
   private TimestampWritable tempTimestampWritable;
 
-  public BinarySortableSerializeWrite(boolean[] columnSortOrderIsDesc) {
+  public BinarySortableSerializeWrite(boolean[] columnSortOrderIsDesc,
+          byte[] columnNullMarker, byte[] columnNotNullMarker) {
     this();
     fieldCount = columnSortOrderIsDesc.length;
     this.columnSortOrderIsDesc = columnSortOrderIsDesc;
+    this.columnNullMarker = columnNullMarker;
+    this.columnNotNullMarker = columnNotNullMarker;
   }
 
   /*
    * Use this constructor when only ascending sort order is used.
+   * By default for ascending order, NULL first.
    */
   public BinarySortableSerializeWrite(int fieldCount) {
     this();
     this.fieldCount = fieldCount;
     columnSortOrderIsDesc = new boolean[fieldCount];
     Arrays.fill(columnSortOrderIsDesc, false);
+    columnNullMarker = new byte[fieldCount];
+    Arrays.fill(columnNullMarker, BinarySortableSerDe.ZERO);
+    columnNotNullMarker = new byte[fieldCount];
+    Arrays.fill(columnNotNullMarker, BinarySortableSerDe.ONE);
   }
 
   // Not public since we must have the field count or column sort order information.
@@ -112,7 +123,8 @@ public final class BinarySortableSerializeWrite implements SerializeWrite {
    */
   @Override
   public void writeNull() throws IOException {
-    BinarySortableSerDe.writeByte(output, (byte) 0, columnSortOrderIsDesc[++index]);
+    ++index;
+    BinarySortableSerDe.writeByte(output, columnNullMarker[index], columnSortOrderIsDesc[index]);
   }
 
   /*
@@ -120,10 +132,12 @@ public final class BinarySortableSerializeWrite implements SerializeWrite {
    */
   @Override
   public void writeBoolean(boolean v) throws IOException {
-    final boolean invert = columnSortOrderIsDesc[++index];
+    ++index;
+
+    final boolean invert = columnSortOrderIsDesc[index];
 
     // This field is not a null.
-    BinarySortableSerDe.writeByte(output, (byte) 1, invert);
+    BinarySortableSerDe.writeByte(output, columnNotNullMarker[index], invert);
 
     BinarySortableSerDe.writeByte(output, (byte) (v ? 2 : 1), invert);
   }
@@ -133,10 +147,12 @@ public final class BinarySortableSerializeWrite implements SerializeWrite {
    */
   @Override
   public void writeByte(byte v) throws IOException {
-    final boolean invert = columnSortOrderIsDesc[++index];
+    ++index;
+
+    final boolean invert = columnSortOrderIsDesc[index];
 
     // This field is not a null.
-    BinarySortableSerDe.writeByte(output, (byte) 1, invert);
+    BinarySortableSerDe.writeByte(output, columnNotNullMarker[index], invert);
 
     BinarySortableSerDe.writeByte(output, (byte) (v ^ 0x80), invert);
   }
@@ -146,10 +162,12 @@ public final class BinarySortableSerializeWrite implements SerializeWrite {
    */
   @Override
   public void writeShort(short v) throws IOException {
-    final boolean invert = columnSortOrderIsDesc[++index];
+    ++index;
+
+    final boolean invert = columnSortOrderIsDesc[index];
 
     // This field is not a null.
-    BinarySortableSerDe.writeByte(output, (byte) 1, invert);
+    BinarySortableSerDe.writeByte(output, columnNotNullMarker[index], invert);
 
     BinarySortableSerDe.serializeShort(output, v, invert);
   }
@@ -159,10 +177,12 @@ public final class BinarySortableSerializeWrite implements SerializeWrite {
    */
   @Override
   public void writeInt(int v) throws IOException {
-    final boolean invert = columnSortOrderIsDesc[++index];
+    ++index;
+
+    final boolean invert = columnSortOrderIsDesc[index];
 
     // This field is not a null.
-    BinarySortableSerDe.writeByte(output, (byte) 1, invert);
+    BinarySortableSerDe.writeByte(output, columnNotNullMarker[index], invert);
 
     BinarySortableSerDe.serializeInt(output, v, invert);
   }
@@ -172,10 +192,12 @@ public final class BinarySortableSerializeWrite implements SerializeWrite {
    */
   @Override
   public void writeLong(long v) throws IOException {
-    final boolean invert = columnSortOrderIsDesc[++index];
+    ++index;
+
+    final boolean invert = columnSortOrderIsDesc[index];
 
     // This field is not a null.
-    BinarySortableSerDe.writeByte(output, (byte) 1, invert);
+    BinarySortableSerDe.writeByte(output, columnNotNullMarker[index], invert);
 
     BinarySortableSerDe.serializeLong(output, v, invert);
   }
@@ -185,10 +207,12 @@ public final class BinarySortableSerializeWrite implements SerializeWrite {
    */
   @Override
   public void writeFloat(float vf) throws IOException {
-    final boolean invert = columnSortOrderIsDesc[++index];
+    ++index;
+
+    final boolean invert = columnSortOrderIsDesc[index];
 
     // This field is not a null.
-    BinarySortableSerDe.writeByte(output, (byte) 1, invert);
+    BinarySortableSerDe.writeByte(output, columnNotNullMarker[index], invert);
 
     BinarySortableSerDe.serializeFloat(output, vf, invert);
   }
@@ -198,10 +222,12 @@ public final class BinarySortableSerializeWrite implements SerializeWrite {
    */
   @Override
   public void writeDouble(double vd) throws IOException {
-    final boolean invert = columnSortOrderIsDesc[++index];
+    ++index;
+
+    final boolean invert = columnSortOrderIsDesc[index];
 
     // This field is not a null.
-    BinarySortableSerDe.writeByte(output, (byte) 1, invert);
+    BinarySortableSerDe.writeByte(output, columnNotNullMarker[index], invert);
 
     BinarySortableSerDe.serializeDouble(output, vd, invert);
   }
@@ -214,20 +240,24 @@ public final class BinarySortableSerializeWrite implements SerializeWrite {
    */
   @Override
   public void writeString(byte[] v) throws IOException {
-    final boolean invert = columnSortOrderIsDesc[++index];
+    ++index;
+
+    final boolean invert = columnSortOrderIsDesc[index];
 
     // This field is not a null.
-    BinarySortableSerDe.writeByte(output, (byte) 1, invert);
+    BinarySortableSerDe.writeByte(output, columnNotNullMarker[index], invert);
 
     BinarySortableSerDe.serializeBytes(output, v, 0, v.length, invert);
   }
 
   @Override
   public void writeString(byte[] v, int start, int length) throws IOException {
-    final boolean invert = columnSortOrderIsDesc[++index];
+    ++index;
+
+    final boolean invert = columnSortOrderIsDesc[index];
 
     // This field is not a null.
-    BinarySortableSerDe.writeByte(output, (byte) 1, invert);
+    BinarySortableSerDe.writeByte(output, columnNotNullMarker[index], invert);
 
     BinarySortableSerDe.serializeBytes(output, v, start, length, invert);
   }
@@ -257,20 +287,24 @@ public final class BinarySortableSerializeWrite implements SerializeWrite {
    */
   @Override
   public void writeBinary(byte[] v) throws IOException {
-    final boolean invert = columnSortOrderIsDesc[++index];
+    ++index;
+
+    final boolean invert = columnSortOrderIsDesc[index];
 
     // This field is not a null.
-    BinarySortableSerDe.writeByte(output, (byte) 1, invert);
+    BinarySortableSerDe.writeByte(output, columnNotNullMarker[index], invert);
 
     BinarySortableSerDe.serializeBytes(output, v, 0, v.length, invert);
   }
 
   @Override
   public void writeBinary(byte[] v, int start, int length) {
-    final boolean invert = columnSortOrderIsDesc[++index];
+    ++index;
+
+    final boolean invert = columnSortOrderIsDesc[index];
 
     // This field is not a null.
-    BinarySortableSerDe.writeByte(output, (byte) 1, invert);
+    BinarySortableSerDe.writeByte(output, columnNotNullMarker[index], invert);
 
     BinarySortableSerDe.serializeBytes(output, v, start, length, invert);
   }
@@ -280,10 +314,12 @@ public final class BinarySortableSerializeWrite implements SerializeWrite {
    */
   @Override
   public void writeDate(Date date) throws IOException {
-    final boolean invert = columnSortOrderIsDesc[++index];
+    ++index;
+
+    final boolean invert = columnSortOrderIsDesc[index];
 
     // This field is not a null.
-    BinarySortableSerDe.writeByte(output, (byte) 1, invert);
+    BinarySortableSerDe.writeByte(output, columnNotNullMarker[index], invert);
 
     BinarySortableSerDe.serializeInt(output, DateWritable.dateToDays(date), invert);
   }
@@ -291,10 +327,12 @@ public final class BinarySortableSerializeWrite implements SerializeWrite {
   // We provide a faster way to write a date without a Date object.
   @Override
   public void writeDate(int dateAsDays) throws IOException {
-    final boolean invert = columnSortOrderIsDesc[++index];
+    ++index;
+
+    final boolean invert = columnSortOrderIsDesc[index];
 
     // This field is not a null.
-    BinarySortableSerDe.writeByte(output, (byte) 1, invert);
+    BinarySortableSerDe.writeByte(output, columnNotNullMarker[index], invert);
 
     BinarySortableSerDe.serializeInt(output, dateAsDays, invert);
   }
@@ -304,10 +342,12 @@ public final class BinarySortableSerializeWrite implements SerializeWrite {
    */
   @Override
   public void writeTimestamp(Timestamp vt) throws IOException {
-    final boolean invert = columnSortOrderIsDesc[++index];
+    ++index;
+
+    final boolean invert = columnSortOrderIsDesc[index];
 
     // This field is not a null.
-    BinarySortableSerDe.writeByte(output, (byte) 1, invert);
+    BinarySortableSerDe.writeByte(output, columnNotNullMarker[index], invert);
 
     tempTimestampWritable.set(vt);
     BinarySortableSerDe.serializeTimestampWritable(output, tempTimestampWritable, invert);
@@ -318,20 +358,24 @@ public final class BinarySortableSerializeWrite implements SerializeWrite {
    */
   @Override
   public void writeHiveIntervalYearMonth(HiveIntervalYearMonth viyt) throws IOException {
-    final boolean invert = columnSortOrderIsDesc[++index];
+    ++index;
+
+    final boolean invert = columnSortOrderIsDesc[index];
 
     // This field is not a null.
-    BinarySortableSerDe.writeByte(output, (byte) 1, invert);
+    BinarySortableSerDe.writeByte(output, columnNotNullMarker[index], invert);
 
     BinarySortableSerDe.serializeHiveIntervalYearMonth(output, viyt, invert);
   }
 
   @Override
   public void writeHiveIntervalYearMonth(int totalMonths) throws IOException {
-    final boolean invert = columnSortOrderIsDesc[++index];
+    ++index;
+
+    final boolean invert = columnSortOrderIsDesc[index];
 
     // This field is not a null.
-    BinarySortableSerDe.writeByte(output, (byte) 1, invert);
+    BinarySortableSerDe.writeByte(output, columnNotNullMarker[index], invert);
 
     BinarySortableSerDe.serializeInt(output, totalMonths, invert);
   }
@@ -341,10 +385,12 @@ public final class BinarySortableSerializeWrite implements SerializeWrite {
    */
   @Override
   public void writeHiveIntervalDayTime(HiveIntervalDayTime vidt) throws IOException {
-    final boolean invert = columnSortOrderIsDesc[++index];
+    ++index;
+
+    final boolean invert = columnSortOrderIsDesc[index];
 
     // This field is not a null.
-    BinarySortableSerDe.writeByte(output, (byte) 1, invert);
+    BinarySortableSerDe.writeByte(output, columnNotNullMarker[index], invert);
 
     BinarySortableSerDe.serializeHiveIntervalDayTime(output, vidt, invert);
   }
@@ -354,10 +400,12 @@ public final class BinarySortableSerializeWrite implements SerializeWrite {
    */
   @Override
   public void writeHiveDecimal(HiveDecimal dec, int scale) throws IOException {
-    final boolean invert = columnSortOrderIsDesc[++index];
+    ++index;
+
+    final boolean invert = columnSortOrderIsDesc[index];
 
     // This field is not a null.
-    BinarySortableSerDe.writeByte(output, (byte) 1, invert);
+    BinarySortableSerDe.writeByte(output, columnNotNullMarker[index], invert);
 
     BinarySortableSerDe.serializeHiveDecimal(output, dec, invert);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/serde/src/test/org/apache/hadoop/hive/serde2/binarysortable/TestBinarySortableFast.java
----------------------------------------------------------------------
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/binarysortable/TestBinarySortableFast.java b/serde/src/test/org/apache/hadoop/hive/serde2/binarysortable/TestBinarySortableFast.java
index ae476ae..0be3213 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/binarysortable/TestBinarySortableFast.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/binarysortable/TestBinarySortableFast.java
@@ -23,8 +23,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Random;
 
-import junit.framework.TestCase;
-
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.hive.serde2.ByteStream.Output;
 import org.apache.hadoop.hive.serde2.SerDe;
@@ -33,20 +31,23 @@ import org.apache.hadoop.hive.serde2.binarysortable.MyTestPrimitiveClass.ExtraTy
 import org.apache.hadoop.hive.serde2.binarysortable.fast.BinarySortableDeserializeRead;
 import org.apache.hadoop.hive.serde2.binarysortable.fast.BinarySortableSerializeWrite;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory.ObjectInspectorOptions;
-import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
 import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
 import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
 import org.apache.hadoop.io.BytesWritable;
 
+import junit.framework.TestCase;
+
 public class TestBinarySortableFast extends TestCase {
 
   private void testBinarySortableFast(MyTestPrimitiveClass[] myTestPrimitiveClasses,
-          boolean[] columnSortOrderIsDesc, SerDe serde, StructObjectInspector rowOI, boolean ascending,
+          boolean[] columnSortOrderIsDesc, byte[] columnNullMarker, byte[] columnNotNullMarker,
+          SerDe serde, StructObjectInspector rowOI, boolean ascending,
           Map<Object, PrimitiveTypeInfo[]> primitiveTypeInfoMap) throws Throwable {
 
-    BinarySortableSerializeWrite binarySortableSerializeWrite = new BinarySortableSerializeWrite(columnSortOrderIsDesc);
+    BinarySortableSerializeWrite binarySortableSerializeWrite =
+            new BinarySortableSerializeWrite(columnSortOrderIsDesc, columnNullMarker, columnNotNullMarker);
 
     // Try to serialize
 
@@ -227,15 +228,24 @@ public class TestBinarySortableFast extends TestCase {
       String fieldTypes = ObjectInspectorUtils.getFieldTypes(rowOI);
       String order;
       order = StringUtils.leftPad("", MyTestPrimitiveClass.primitiveCount, '+');
-      SerDe serde_ascending = TestBinarySortableSerDe.getSerDe(fieldNames, fieldTypes, order);
+      String nullOrder;
+      nullOrder = StringUtils.leftPad("", MyTestPrimitiveClass.primitiveCount, 'a');
+      SerDe serde_ascending = TestBinarySortableSerDe.getSerDe(fieldNames, fieldTypes, order, nullOrder);
       order = StringUtils.leftPad("", MyTestPrimitiveClass.primitiveCount, '-');
-      SerDe serde_descending = TestBinarySortableSerDe.getSerDe(fieldNames, fieldTypes, order);
+      nullOrder = StringUtils.leftPad("", MyTestPrimitiveClass.primitiveCount, 'z');
+      SerDe serde_descending = TestBinarySortableSerDe.getSerDe(fieldNames, fieldTypes, order, nullOrder);
 
       boolean[] columnSortOrderIsDesc = new boolean[MyTestPrimitiveClass.primitiveCount];
       Arrays.fill(columnSortOrderIsDesc, false);
-      testBinarySortableFast(myTestPrimitiveClasses, columnSortOrderIsDesc, serde_ascending, rowOI, true, primitiveTypeInfoMap);
+      byte[] columnNullMarker = new byte[MyTestPrimitiveClass.primitiveCount];
+      Arrays.fill(columnNullMarker, BinarySortableSerDe.ZERO);
+      byte[] columnNotNullMarker = new byte[MyTestPrimitiveClass.primitiveCount];
+      Arrays.fill(columnNotNullMarker, BinarySortableSerDe.ONE);
+      testBinarySortableFast(myTestPrimitiveClasses, columnSortOrderIsDesc, columnNullMarker,
+              columnNotNullMarker, serde_ascending, rowOI, true, primitiveTypeInfoMap);
       Arrays.fill(columnSortOrderIsDesc, true);
-      testBinarySortableFast(myTestPrimitiveClasses, columnSortOrderIsDesc, serde_descending, rowOI, false, primitiveTypeInfoMap);
+      testBinarySortableFast(myTestPrimitiveClasses, columnSortOrderIsDesc, columnNullMarker,
+              columnNotNullMarker, serde_descending, rowOI, false, primitiveTypeInfoMap);
     } catch (Throwable e) {
       e.printStackTrace();
       throw e;

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/serde/src/test/org/apache/hadoop/hive/serde2/binarysortable/TestBinarySortableSerDe.java
----------------------------------------------------------------------
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/binarysortable/TestBinarySortableSerDe.java b/serde/src/test/org/apache/hadoop/hive/serde2/binarysortable/TestBinarySortableSerDe.java
index af47e6f..935313b 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/binarysortable/TestBinarySortableSerDe.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/binarysortable/TestBinarySortableSerDe.java
@@ -17,18 +17,12 @@
  */
 package org.apache.hadoop.hive.serde2.binarysortable;
 
-import java.sql.Date;
-import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Properties;
 import java.util.Random;
 
-import junit.framework.TestCase;
-
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.SerDe;
 import org.apache.hadoop.hive.serde2.SerDeUtils;
@@ -40,6 +34,8 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
 import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
 import org.apache.hadoop.io.BytesWritable;
 
+import junit.framework.TestCase;
+
 /**
  * TestBinarySortableSerDe.
  *
@@ -66,12 +62,13 @@ public class TestBinarySortableSerDe extends TestCase {
     return sb.toString();
   }
 
-  public static SerDe getSerDe(String fieldNames, String fieldTypes, String order)
+  public static SerDe getSerDe(String fieldNames, String fieldTypes, String order, String nullOrder)
       throws Throwable {
     Properties schema = new Properties();
     schema.setProperty(serdeConstants.LIST_COLUMNS, fieldNames);
     schema.setProperty(serdeConstants.LIST_COLUMN_TYPES, fieldTypes);
     schema.setProperty(serdeConstants.SERIALIZATION_SORT_ORDER, order);
+    schema.setProperty(serdeConstants.SERIALIZATION_NULL_SORT_ORDER, nullOrder);
 
     BinarySortableSerDe serde = new BinarySortableSerDe();
     SerDeUtils.initializeSerDe(serde, new Configuration(), schema, null);
@@ -172,11 +169,14 @@ public class TestBinarySortableSerDe extends TestCase {
 
       String order;
       order = StringUtils.leftPad("", MyTestClass.fieldCount, '+');
+      String nullOrder;
+      nullOrder = StringUtils.leftPad("", MyTestClass.fieldCount, 'a');
       testBinarySortableSerDe(rows, rowOI, getSerDe(fieldNames, fieldTypes,
-          order), true);
+          order, nullOrder), true);
       order = StringUtils.leftPad("", MyTestClass.fieldCount, '-');
+      nullOrder = StringUtils.leftPad("", MyTestClass.fieldCount, 'z');
       testBinarySortableSerDe(rows, rowOI, getSerDe(fieldNames, fieldTypes,
-          order), false);
+          order, nullOrder), false);
 
       System.out.println("Test testTBinarySortableProtocol passed!");
     } catch (Throwable e) {


[14/18] hive git commit: HIVE-12994: Implement support for NULLS FIRST/NULLS LAST (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/infer_bucket_sort_map_operators.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/infer_bucket_sort_map_operators.q.out b/ql/src/test/results/clientpositive/infer_bucket_sort_map_operators.q.out
index 4ca6517..5ebcf41 100644
--- a/ql/src/test/results/clientpositive/infer_bucket_sort_map_operators.q.out
+++ b/ql/src/test/results/clientpositive/infer_bucket_sort_map_operators.q.out
@@ -361,7 +361,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[value]             	 
-Sort Columns:       	[Order(col:value, order:1)]	 
+Sort Columns:       	[Order(col:value, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: -- Test SMB join doesn't affect inference, should not be bucketed or sorted
@@ -640,6 +640,6 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[key]               	 
-Sort Columns:       	[Order(col:key, order:1)]	 
+Sort Columns:       	[Order(col:key, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/infer_bucket_sort_merge.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/infer_bucket_sort_merge.q.out b/ql/src/test/results/clientpositive/infer_bucket_sort_merge.q.out
index 53407c5..97cb92d 100644
--- a/ql/src/test/results/clientpositive/infer_bucket_sort_merge.q.out
+++ b/ql/src/test/results/clientpositive/infer_bucket_sort_merge.q.out
@@ -117,6 +117,6 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	2                   	 
 Bucket Columns:     	[key]               	 
-Sort Columns:       	[Order(col:key, order:1)]	 
+Sort Columns:       	[Order(col:key, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/infer_bucket_sort_multi_insert.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/infer_bucket_sort_multi_insert.q.out b/ql/src/test/results/clientpositive/infer_bucket_sort_multi_insert.q.out
index d4c22f4..e59026e 100644
--- a/ql/src/test/results/clientpositive/infer_bucket_sort_multi_insert.q.out
+++ b/ql/src/test/results/clientpositive/infer_bucket_sort_multi_insert.q.out
@@ -172,7 +172,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[key]               	 
-Sort Columns:       	[Order(col:key, order:1)]	 
+Sort Columns:       	[Order(col:key, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: DESCRIBE FORMATTED test_table PARTITION (part = '2')
@@ -211,7 +211,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[value]             	 
-Sort Columns:       	[Order(col:value, order:1)]	 
+Sort Columns:       	[Order(col:value, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: -- The first partition should be bucketed and sorted, the second should not
@@ -272,7 +272,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[key]               	 
-Sort Columns:       	[Order(col:key, order:1)]	 
+Sort Columns:       	[Order(col:key, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: DESCRIBE FORMATTED test_table PARTITION (part = '2')
@@ -372,7 +372,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[key]               	 
-Sort Columns:       	[Order(col:key, order:1)]	 
+Sort Columns:       	[Order(col:key, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: DESCRIBE FORMATTED test_table PARTITION (part = '2')
@@ -411,6 +411,6 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[key]               	 
-Sort Columns:       	[Order(col:key, order:1)]	 
+Sort Columns:       	[Order(col:key, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/infer_bucket_sort_reducers_power_two.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/infer_bucket_sort_reducers_power_two.q.out b/ql/src/test/results/clientpositive/infer_bucket_sort_reducers_power_two.q.out
index 1e4db29..91466f1 100644
--- a/ql/src/test/results/clientpositive/infer_bucket_sort_reducers_power_two.q.out
+++ b/ql/src/test/results/clientpositive/infer_bucket_sort_reducers_power_two.q.out
@@ -64,7 +64,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	4                   	 
 Bucket Columns:     	[key]               	 
-Sort Columns:       	[Order(col:key, order:1)]	 
+Sort Columns:       	[Order(col:key, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: -- Test join, should be bucketed and sorted by join key
@@ -117,7 +117,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	4                   	 
 Bucket Columns:     	[key]               	 
-Sort Columns:       	[Order(col:key, order:1)]	 
+Sort Columns:       	[Order(col:key, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: -- Test join with two keys, should be bucketed and sorted by join keys
@@ -170,7 +170,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	4                   	 
 Bucket Columns:     	[key, value]        	 
-Sort Columns:       	[Order(col:key, order:1), Order(col:value, order:1)]	 
+Sort Columns:       	[Order(col:key, order:1, nullOrder:0), Order(col:value, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: -- Test join on three tables on same key, should be bucketed and sorted by join key
@@ -223,7 +223,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	4                   	 
 Bucket Columns:     	[key]               	 
-Sort Columns:       	[Order(col:key, order:1)]	 
+Sort Columns:       	[Order(col:key, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: -- Test join on three tables on different keys, should be bucketed and sorted by latter key
@@ -276,7 +276,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	16                  	 
 Bucket Columns:     	[key]               	 
-Sort Columns:       	[Order(col:key, order:1)]	 
+Sort Columns:       	[Order(col:key, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: -- Test group by in subquery with another group by outside, should be bucketed and sorted by the
@@ -331,6 +331,6 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[value]             	 
-Sort Columns:       	[Order(col:value, order:1)]	 
+Sort Columns:       	[Order(col:value, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/input23.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/input23.q.out b/ql/src/test/results/clientpositive/input23.q.out
index 773eac3..33f2eb6 100644
--- a/ql/src/test/results/clientpositive/input23.q.out
+++ b/ql/src/test/results/clientpositive/input23.q.out
@@ -74,6 +74,7 @@ STAGE PLANS:
               outputColumnNames: _col0, _col1
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
+                null sort order: 
                 sort order: 
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                 tag: 0
@@ -92,6 +93,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                 Reduce Output Operator
+                  null sort order: 
                   sort order: 
                   Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                   tag: 1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/input_part7.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/input_part7.q.out b/ql/src/test/results/clientpositive/input_part7.q.out
index a0cf9af..53125d1 100644
--- a/ql/src/test/results/clientpositive/input_part7.q.out
+++ b/ql/src/test/results/clientpositive/input_part7.q.out
@@ -88,25 +88,29 @@ TOK_QUERY
             TOK_ALLCOLREF
       TOK_SORTBY
          TOK_TABSORTCOLNAMEASC
-            .
-               TOK_TABLE_OR_COL
-                  A
-               key
+            TOK_NULLS_FIRST
+               .
+                  TOK_TABLE_OR_COL
+                     A
+                  key
          TOK_TABSORTCOLNAMEASC
-            .
-               TOK_TABLE_OR_COL
-                  A
-               value
+            TOK_NULLS_FIRST
+               .
+                  TOK_TABLE_OR_COL
+                     A
+                  value
          TOK_TABSORTCOLNAMEASC
-            .
-               TOK_TABLE_OR_COL
-                  A
-               ds
+            TOK_NULLS_FIRST
+               .
+                  TOK_TABLE_OR_COL
+                     A
+                  ds
          TOK_TABSORTCOLNAMEASC
-            .
-               TOK_TABLE_OR_COL
-                  A
-               hr
+            TOK_NULLS_FIRST
+               .
+                  TOK_TABLE_OR_COL
+                     A
+                  hr
 
 
 STAGE DEPENDENCIES:
@@ -137,6 +141,7 @@ STAGE PLANS:
                     Statistics: Num rows: 666 Data size: 7074 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: string), _col1 (type: string), '2008-04-08' (type: string), _col3 (type: string)
+                      null sort order: aaaa
                       sort order: ++++
                       Statistics: Num rows: 666 Data size: 7074 Basic stats: COMPLETE Column stats: NONE
                       tag: -1
@@ -161,6 +166,7 @@ STAGE PLANS:
                     Statistics: Num rows: 666 Data size: 7074 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: string), _col1 (type: string), '2008-04-08' (type: string), _col3 (type: string)
+                      null sort order: aaaa
                       sort order: ++++
                       Statistics: Num rows: 666 Data size: 7074 Basic stats: COMPLETE Column stats: NONE
                       tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/join17.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/join17.q.out b/ql/src/test/results/clientpositive/join17.q.out
index 0f1286e..13260ef 100644
--- a/ql/src/test/results/clientpositive/join17.q.out
+++ b/ql/src/test/results/clientpositive/join17.q.out
@@ -79,6 +79,7 @@ STAGE PLANS:
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -99,6 +100,7 @@ STAGE PLANS:
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/join35.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/join35.q.out b/ql/src/test/results/clientpositive/join35.q.out
index 663642c..96f7cc8 100644
--- a/ql/src/test/results/clientpositive/join35.q.out
+++ b/ql/src/test/results/clientpositive/join35.q.out
@@ -178,6 +178,7 @@ STAGE PLANS:
                   Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col0 (type: string)
+                    null sort order: a
                     sort order: +
                     Map-reduce partition columns: _col0 (type: string)
                     Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
@@ -515,6 +516,7 @@ STAGE PLANS:
                   Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col0 (type: string)
+                    null sort order: a
                     sort order: +
                     Map-reduce partition columns: _col0 (type: string)
                     Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/join9.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/join9.q.out b/ql/src/test/results/clientpositive/join9.q.out
index 14ae66a..e904b31 100644
--- a/ql/src/test/results/clientpositive/join9.q.out
+++ b/ql/src/test/results/clientpositive/join9.q.out
@@ -95,6 +95,7 @@ STAGE PLANS:
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -114,6 +115,7 @@ STAGE PLANS:
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/join_filters_overlap.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/join_filters_overlap.q.out b/ql/src/test/results/clientpositive/join_filters_overlap.q.out
index d681a20..ca1e085 100644
--- a/ql/src/test/results/clientpositive/join_filters_overlap.q.out
+++ b/ql/src/test/results/clientpositive/join_filters_overlap.q.out
@@ -110,6 +110,7 @@ STAGE PLANS:
               Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
                 key expressions: _col0 (type: int)
+                null sort order: a
                 sort order: +
                 Map-reduce partition columns: _col0 (type: int)
                 Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE
@@ -130,6 +131,7 @@ STAGE PLANS:
                 Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE
@@ -150,6 +152,7 @@ STAGE PLANS:
                 Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE
@@ -374,6 +377,7 @@ STAGE PLANS:
                 Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE
@@ -390,6 +394,7 @@ STAGE PLANS:
               Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
                 key expressions: _col0 (type: int)
+                null sort order: a
                 sort order: +
                 Map-reduce partition columns: _col0 (type: int)
                 Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE
@@ -410,6 +415,7 @@ STAGE PLANS:
                 Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE
@@ -648,6 +654,7 @@ STAGE PLANS:
                 Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE
@@ -664,6 +671,7 @@ STAGE PLANS:
               Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
                 key expressions: _col0 (type: int)
+                null sort order: a
                 sort order: +
                 Map-reduce partition columns: _col0 (type: int)
                 Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE
@@ -684,6 +692,7 @@ STAGE PLANS:
                 Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE
@@ -932,6 +941,7 @@ STAGE PLANS:
               Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
                 key expressions: _col0 (type: int)
+                null sort order: a
                 sort order: +
                 Map-reduce partition columns: _col0 (type: int)
                 Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE
@@ -948,6 +958,7 @@ STAGE PLANS:
               Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
                 key expressions: _col0 (type: int)
+                null sort order: a
                 sort order: +
                 Map-reduce partition columns: _col0 (type: int)
                 Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE
@@ -968,6 +979,7 @@ STAGE PLANS:
                 Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE
@@ -988,6 +1000,7 @@ STAGE PLANS:
                 Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE
@@ -1231,6 +1244,7 @@ STAGE PLANS:
               Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
                 key expressions: _col0 (type: int)
+                null sort order: a
                 sort order: +
                 Map-reduce partition columns: _col0 (type: int)
                 Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE
@@ -1251,6 +1265,7 @@ STAGE PLANS:
                 Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE
@@ -1271,6 +1286,7 @@ STAGE PLANS:
                 Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE
@@ -1291,6 +1307,7 @@ STAGE PLANS:
                 Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/keyword_3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/keyword_3.q.out b/ql/src/test/results/clientpositive/keyword_3.q.out
new file mode 100644
index 0000000..067fdd2
--- /dev/null
+++ b/ql/src/test/results/clientpositive/keyword_3.q.out
@@ -0,0 +1,32 @@
+PREHOOK: query: drop table NULLS
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table NULLS
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: create table NULLS (LAST string)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@NULLS
+POSTHOOK: query: create table NULLS (LAST string)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@NULLS
+PREHOOK: query: insert overwrite table NULLS
+  select key from src where key = '238' limit 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@nulls
+POSTHOOK: query: insert overwrite table NULLS
+  select key from src where key = '238' limit 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@nulls
+POSTHOOK: Lineage: nulls.last SIMPLE []
+PREHOOK: query: select LAST from NULLS
+PREHOOK: type: QUERY
+PREHOOK: Input: default@nulls
+#### A masked pattern was here ####
+POSTHOOK: query: select LAST from NULLS
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@nulls
+#### A masked pattern was here ####
+238

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/lineage2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/lineage2.q.out b/ql/src/test/results/clientpositive/lineage2.q.out
index 66929dd..a08094a 100644
--- a/ql/src/test/results/clientpositive/lineage2.q.out
+++ b/ql/src/test/results/clientpositive/lineage2.q.out
@@ -634,7 +634,7 @@ having count(a.c2) > 0
 PREHOOK: type: QUERY
 PREHOOK: Input: default@dest_l2
 #### A masked pattern was here ####
-{"version":"1.0","engine":"mr","database":"default","hash":"0ae7aa4a0cbd1283210fa79e8a19104a","queryText":"select sum(a.c1) over (partition by a.c1 order by a.id)\nfrom dest_l2 a\nwhere a.c2 != 10\ngroup by a.c1, a.c2, a.id\nhaving count(a.c2) > 0","edges":[{"sources":[1,2,3],"targets":[0],"expression":"(tok_function sum (. (tok_table_or_col $hdt$_0) c1) (tok_windowspec (tok_partitioningspec (tok_distributeby (. (tok_table_or_col $hdt$_0) c1)) (tok_orderby (tok_tabsortcolnameasc (. (tok_table_or_col $hdt$_0) id)))) (tok_windowvalues (preceding 2147483647) current)))","edgeType":"PROJECTION"},{"sources":[2],"targets":[0],"expression":"(a.c2 <> 10)","edgeType":"PREDICATE"},{"sources":[2],"targets":[0],"expression":"(count(default.dest_l2.c2) > 0)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"c0"},{"id":1,"vertexType":"COLUMN","vertexId":"default.dest_l2.c1"},{"id":2,"vertexType":"COLUMN","vertexId":"default.dest_l2.c2"},{"id":3,"vertexType":"COLUMN","
 vertexId":"default.dest_l2.id"}]}
+{"version":"1.0","engine":"mr","database":"default","hash":"0ae7aa4a0cbd1283210fa79e8a19104a","queryText":"select sum(a.c1) over (partition by a.c1 order by a.id)\nfrom dest_l2 a\nwhere a.c2 != 10\ngroup by a.c1, a.c2, a.id\nhaving count(a.c2) > 0","edges":[{"sources":[1,2,3],"targets":[0],"expression":"(tok_function sum (. (tok_table_or_col $hdt$_0) c1) (tok_windowspec (tok_partitioningspec (tok_distributeby (. (tok_table_or_col $hdt$_0) c1)) (tok_orderby (tok_tabsortcolnameasc (tok_nulls_first (. (tok_table_or_col $hdt$_0) id))))) (tok_windowvalues (preceding 2147483647) current)))","edgeType":"PROJECTION"},{"sources":[2],"targets":[0],"expression":"(a.c2 <> 10)","edgeType":"PREDICATE"},{"sources":[2],"targets":[0],"expression":"(count(default.dest_l2.c2) > 0)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"c0"},{"id":1,"vertexType":"COLUMN","vertexId":"default.dest_l2.c1"},{"id":2,"vertexType":"COLUMN","vertexId":"default.dest_l2.c2"},{"id":3,"vert
 exType":"COLUMN","vertexId":"default.dest_l2.id"}]}
 1
 PREHOOK: query: select sum(a.c1), count(b.c1), b.c2, b.c3
 from dest_l2 a join dest_l3 b on (a.id = b.id)

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/lineage3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/lineage3.q.out b/ql/src/test/results/clientpositive/lineage3.q.out
index 0e9c2a9..61acf52 100644
--- a/ql/src/test/results/clientpositive/lineage3.q.out
+++ b/ql/src/test/results/clientpositive/lineage3.q.out
@@ -67,7 +67,7 @@ where cint > 10 and cint < 10000 limit 10
 PREHOOK: type: QUERY
 PREHOOK: Input: default@alltypesorc
 #### A masked pattern was here ####
-{"version":"1.0","engine":"mr","database":"default","hash":"351b08ec58591554ec10a6ded68ef25f","queryText":"select cint, rank() over(order by cint) from alltypesorc\nwhere cint > 10 and cint < 10000 limit 10","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3,4,2,5,6,7,8,9,10,11,12,13],"targets":[1],"expression":"(tok_function rank (tok_windowspec (tok_partitioningspec (tok_distributeby 0) (tok_orderby (tok_tabsortcolnameasc (. (tok_table_or_col alltypesorc) cint)))) (tok_windowrange (preceding 2147483647) (following 2147483647))))","edgeType":"PROJECTION"},{"sources":[2],"targets":[0,1],"expression":"((alltypesorc.cint > 10) and (alltypesorc.cint < 10000))","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"cint"},{"id":1,"vertexType":"COLUMN","vertexId":"c1"},{"id":2,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cint"},{"id":3,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"},{"id":4,"vertexType":"COLUMN",
 "vertexId":"default.alltypesorc.csmallint"},{"id":5,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cbigint"},{"id":6,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cfloat"},{"id":7,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cdouble"},{"id":8,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cstring1"},{"id":9,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cstring2"},{"id":10,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctimestamp1"},{"id":11,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctimestamp2"},{"id":12,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cboolean1"},{"id":13,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cboolean2"}]}
+{"version":"1.0","engine":"mr","database":"default","hash":"351b08ec58591554ec10a6ded68ef25f","queryText":"select cint, rank() over(order by cint) from alltypesorc\nwhere cint > 10 and cint < 10000 limit 10","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3,4,2,5,6,7,8,9,10,11,12,13],"targets":[1],"expression":"(tok_function rank (tok_windowspec (tok_partitioningspec (tok_distributeby 0) (tok_orderby (tok_tabsortcolnameasc (tok_nulls_first (. (tok_table_or_col alltypesorc) cint))))) (tok_windowrange (preceding 2147483647) (following 2147483647))))","edgeType":"PROJECTION"},{"sources":[2],"targets":[0,1],"expression":"((alltypesorc.cint > 10) and (alltypesorc.cint < 10000))","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"cint"},{"id":1,"vertexType":"COLUMN","vertexId":"c1"},{"id":2,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cint"},{"id":3,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"},{"id":4,"ver
 texType":"COLUMN","vertexId":"default.alltypesorc.csmallint"},{"id":5,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cbigint"},{"id":6,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cfloat"},{"id":7,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cdouble"},{"id":8,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cstring1"},{"id":9,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cstring2"},{"id":10,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctimestamp1"},{"id":11,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctimestamp2"},{"id":12,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cboolean1"},{"id":13,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cboolean2"}]}
 762	1
 762	1
 762	1
@@ -86,7 +86,7 @@ order by a.ctinyint, a.cint
 PREHOOK: type: QUERY
 PREHOOK: Input: default@alltypesorc
 #### A masked pattern was here ####
-{"version":"1.0","engine":"mr","database":"default","hash":"40c3faa7abd1cdb7f12c1047a8a1d2ce","queryText":"select a.ctinyint, a.cint, count(a.cdouble)\n  over(partition by a.ctinyint order by a.cint desc\n    rows between 1 preceding and 1 following)\nfrom alltypesorc a inner join alltypesorc b on a.cint = b.cbigint\norder by a.ctinyint, a.cint","edges":[{"sources":[3],"targets":[0],"edgeType":"PROJECTION"},{"sources":[4],"targets":[1],"edgeType":"PROJECTION"},{"sources":[3,4,5,6],"targets":[2],"expression":"(tok_function count (. (tok_table_or_col $hdt$_0) cdouble) (tok_windowspec (tok_partitioningspec (tok_distributeby (. (tok_table_or_col $hdt$_0) ctinyint)) (tok_orderby (tok_tabsortcolnamedesc (. (tok_table_or_col $hdt$_0) cint)))) (tok_windowrange (preceding 1) (following 1))))","edgeType":"PROJECTION"},{"sources":[4],"targets":[0,1,2],"expression":"a.cint is not null","edgeType":"PREDICATE"},{"sources":[4,6],"targets":[0,1,2],"expression":"(UDFToLong(a.cint) = a.cbigint)","edg
 eType":"PREDICATE"},{"sources":[6],"targets":[0,1,2],"expression":"a.cbigint is not null","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"a.ctinyint"},{"id":1,"vertexType":"COLUMN","vertexId":"a.cint"},{"id":2,"vertexType":"COLUMN","vertexId":"c2"},{"id":3,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"},{"id":4,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cint"},{"id":5,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cdouble"},{"id":6,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cbigint"}]}
+{"version":"1.0","engine":"mr","database":"default","hash":"40c3faa7abd1cdb7f12c1047a8a1d2ce","queryText":"select a.ctinyint, a.cint, count(a.cdouble)\n  over(partition by a.ctinyint order by a.cint desc\n    rows between 1 preceding and 1 following)\nfrom alltypesorc a inner join alltypesorc b on a.cint = b.cbigint\norder by a.ctinyint, a.cint","edges":[{"sources":[3],"targets":[0],"edgeType":"PROJECTION"},{"sources":[4],"targets":[1],"edgeType":"PROJECTION"},{"sources":[3,4,5,6],"targets":[2],"expression":"(tok_function count (. (tok_table_or_col $hdt$_0) cdouble) (tok_windowspec (tok_partitioningspec (tok_distributeby (. (tok_table_or_col $hdt$_0) ctinyint)) (tok_orderby (tok_tabsortcolnamedesc (tok_nulls_last (. (tok_table_or_col $hdt$_0) cint))))) (tok_windowrange (preceding 1) (following 1))))","edgeType":"PROJECTION"},{"sources":[4],"targets":[0,1,2],"expression":"a.cint is not null","edgeType":"PREDICATE"},{"sources":[4,6],"targets":[0,1,2],"expression":"(UDFToLong(a.cint) =
  a.cbigint)","edgeType":"PREDICATE"},{"sources":[6],"targets":[0,1,2],"expression":"a.cbigint is not null","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"a.ctinyint"},{"id":1,"vertexType":"COLUMN","vertexId":"a.cint"},{"id":2,"vertexType":"COLUMN","vertexId":"c2"},{"id":3,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"},{"id":4,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cint"},{"id":5,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cdouble"},{"id":6,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cbigint"}]}
 PREHOOK: query: with v2 as
   (select cdouble, count(cint) over() a,
     sum(cint + cbigint) over(partition by cboolean1) b
@@ -97,7 +97,7 @@ order by cdouble, a, b limit 5
 PREHOOK: type: QUERY
 PREHOOK: Input: default@alltypesorc
 #### A masked pattern was here ####
-{"version":"1.0","engine":"mr","database":"default","hash":"a083a5322b6a83af6f614f299d0361e4","queryText":"with v2 as\n  (select cdouble, count(cint) over() a,\n    sum(cint + cbigint) over(partition by cboolean1) b\n    from (select * from alltypesorc) v1)\nselect cdouble, a, b, a + b, cdouble + a from v2\nwhere cdouble is not null\norder by cdouble, a, b limit 5","edges":[{"sources":[5],"targets":[0],"edgeType":"PROJECTION"},{"sources":[6,7,8,9,10,5,11,12,13,14,15,16],"targets":[1],"expression":"(tok_function count (. (tok_table_or_col alltypesorc) cint) (tok_windowspec (tok_partitioningspec (tok_distributeby 0) (tok_orderby (tok_tabsortcolnameasc 0))) (tok_windowrange (preceding 2147483647) (following 2147483647))))","edgeType":"PROJECTION"},{"sources":[6,7,8,9,10,5,11,12,13,14,15,16],"targets":[2],"expression":"(tok_function sum (+ (tok_function tok_bigint (. (tok_table_or_col alltypesorc) cint)) (. (tok_table_or_col alltypesorc) cbigint)) (tok_windowspec (tok_partitioningspec (
 tok_distributeby (. (tok_table_or_col alltypesorc) cboolean1)) (tok_orderby (tok_tabsortcolnameasc (. (tok_table_or_col alltypesorc) cboolean1)))) (tok_windowrange (preceding 2147483647) (following 2147483647))))","edgeType":"PROJECTION"},{"sources":[6,7,8,9,10,5,11,12,13,14,15,16],"targets":[3],"expression":"((tok_function count (. (tok_table_or_col alltypesorc) cint) (tok_windowspec (tok_partitioningspec (tok_distributeby 0) (tok_orderby (tok_tabsortcolnameasc 0))) (tok_windowrange (preceding 2147483647) (following 2147483647)))) + (tok_function sum (+ (tok_function tok_bigint (. (tok_table_or_col alltypesorc) cint)) (. (tok_table_or_col alltypesorc) cbigint)) (tok_windowspec (tok_partitioningspec (tok_distributeby (. (tok_table_or_col alltypesorc) cboolean1)) (tok_orderby (tok_tabsortcolnameasc (. (tok_table_or_col alltypesorc) cboolean1)))) (tok_windowrange (preceding 2147483647) (following 2147483647)))))","edgeType":"PROJECTION"},{"sources":[5,6,7,8,9,10,11,12,13,14,15,16],"ta
 rgets":[4],"expression":"(alltypesorc.cdouble + UDFToDouble((tok_function count (. (tok_table_or_col alltypesorc) cint) (tok_windowspec (tok_partitioningspec (tok_distributeby 0) (tok_orderby (tok_tabsortcolnameasc 0))) (tok_windowrange (preceding 2147483647) (following 2147483647))))))","edgeType":"PROJECTION"},{"sources":[5],"targets":[0,1,2,3,4],"expression":"alltypesorc.cdouble is not null","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"cdouble"},{"id":1,"vertexType":"COLUMN","vertexId":"a"},{"id":2,"vertexType":"COLUMN","vertexId":"b"},{"id":3,"vertexType":"COLUMN","vertexId":"c3"},{"id":4,"vertexType":"COLUMN","vertexId":"c4"},{"id":5,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cdouble"},{"id":6,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"},{"id":7,"vertexType":"COLUMN","vertexId":"default.alltypesorc.csmallint"},{"id":8,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cint"},{"id":9,"vertexType":"COLUMN","verte
 xId":"default.alltypesorc.cbigint"},{"id":10,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cfloat"},{"id":11,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cstring1"},{"id":12,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cstring2"},{"id":13,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctimestamp1"},{"id":14,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctimestamp2"},{"id":15,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cboolean1"},{"id":16,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cboolean2"}]}
+{"version":"1.0","engine":"mr","database":"default","hash":"a083a5322b6a83af6f614f299d0361e4","queryText":"with v2 as\n  (select cdouble, count(cint) over() a,\n    sum(cint + cbigint) over(partition by cboolean1) b\n    from (select * from alltypesorc) v1)\nselect cdouble, a, b, a + b, cdouble + a from v2\nwhere cdouble is not null\norder by cdouble, a, b limit 5","edges":[{"sources":[5],"targets":[0],"edgeType":"PROJECTION"},{"sources":[6,7,8,9,10,5,11,12,13,14,15,16],"targets":[1],"expression":"(tok_function count (. (tok_table_or_col alltypesorc) cint) (tok_windowspec (tok_partitioningspec (tok_distributeby 0) (tok_orderby (tok_tabsortcolnameasc (tok_nulls_first 0)))) (tok_windowrange (preceding 2147483647) (following 2147483647))))","edgeType":"PROJECTION"},{"sources":[6,7,8,9,10,5,11,12,13,14,15,16],"targets":[2],"expression":"(tok_function sum (+ (tok_function tok_bigint (. (tok_table_or_col alltypesorc) cint)) (. (tok_table_or_col alltypesorc) cbigint)) (tok_windowspec (tok_
 partitioningspec (tok_distributeby (. (tok_table_or_col alltypesorc) cboolean1)) (tok_orderby (tok_tabsortcolnameasc (tok_nulls_first (. (tok_table_or_col alltypesorc) cboolean1))))) (tok_windowrange (preceding 2147483647) (following 2147483647))))","edgeType":"PROJECTION"},{"sources":[6,7,8,9,10,5,11,12,13,14,15,16],"targets":[3],"expression":"((tok_function count (. (tok_table_or_col alltypesorc) cint) (tok_windowspec (tok_partitioningspec (tok_distributeby 0) (tok_orderby (tok_tabsortcolnameasc (tok_nulls_first 0)))) (tok_windowrange (preceding 2147483647) (following 2147483647)))) + (tok_function sum (+ (tok_function tok_bigint (. (tok_table_or_col alltypesorc) cint)) (. (tok_table_or_col alltypesorc) cbigint)) (tok_windowspec (tok_partitioningspec (tok_distributeby (. (tok_table_or_col alltypesorc) cboolean1)) (tok_orderby (tok_tabsortcolnameasc (tok_nulls_first (. (tok_table_or_col alltypesorc) cboolean1))))) (tok_windowrange (preceding 2147483647) (following 2147483647)))))",
 "edgeType":"PROJECTION"},{"sources":[5,6,7,8,9,10,11,12,13,14,15,16],"targets":[4],"expression":"(alltypesorc.cdouble + UDFToDouble((tok_function count (. (tok_table_or_col alltypesorc) cint) (tok_windowspec (tok_partitioningspec (tok_distributeby 0) (tok_orderby (tok_tabsortcolnameasc (tok_nulls_first 0)))) (tok_windowrange (preceding 2147483647) (following 2147483647))))))","edgeType":"PROJECTION"},{"sources":[5],"targets":[0,1,2,3,4],"expression":"alltypesorc.cdouble is not null","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"cdouble"},{"id":1,"vertexType":"COLUMN","vertexId":"a"},{"id":2,"vertexType":"COLUMN","vertexId":"b"},{"id":3,"vertexType":"COLUMN","vertexId":"c3"},{"id":4,"vertexType":"COLUMN","vertexId":"c4"},{"id":5,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cdouble"},{"id":6,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"},{"id":7,"vertexType":"COLUMN","vertexId":"default.alltypesorc.csmallint"},{"id":8,"vertex
 Type":"COLUMN","vertexId":"default.alltypesorc.cint"},{"id":9,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cbigint"},{"id":10,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cfloat"},{"id":11,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cstring1"},{"id":12,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cstring2"},{"id":13,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctimestamp1"},{"id":14,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctimestamp2"},{"id":15,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cboolean1"},{"id":16,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cboolean2"}]}
 -16379.0	9173	-919551973060	-919551963887	-7206.0
 -16373.0	9173	-919551973060	-919551963887	-7200.0
 -16372.0	9173	-919551973060	-919551963887	-7199.0
@@ -311,13 +311,13 @@ PREHOOK: type: CREATEVIEW
 PREHOOK: Input: default@alltypesorc
 PREHOOK: Output: database:default
 PREHOOK: Output: default@dest_v3
-{"version":"1.0","engine":"mr","database":"default","hash":"949093880975cc807ad1a8003e8a8c7c","queryText":"alter view dest_v3 as\n  select * from (\n    select sum(a.ctinyint) over (partition by a.csmallint order by a.csmallint) a,\n      count(b.cstring1) x, b.cboolean1\n    from alltypesorc a join alltypesorc b on (a.cint = b.cint)\n    where a.cboolean2 = true and b.cfloat > 0\n    group by a.ctinyint, a.csmallint, b.cboolean1\n    having count(a.cint) > 10\n    order by a, x, b.cboolean1 limit 10) t","edges":[{"sources":[3,4,5,6,7],"targets":[0],"expression":"(tok_function sum (. (tok_table_or_col a) ctinyint) (tok_windowspec (tok_partitioningspec (tok_distributeby (. (tok_table_or_col a) csmallint)) (tok_orderby (tok_tabsortcolnameasc (. (tok_table_or_col a) csmallint))))))","edgeType":"PROJECTION"},{"sources":[6],"targets":[1],"expression":"count(default.alltypesorc.cstring1)","edgeType":"PROJECTION"},{"sources":[5],"targets":[2],"edgeType":"PROJECTION"},{"sources":[7],"target
 s":[0,1,2],"expression":"(a.cint = b.cint)","edgeType":"PREDICATE"},{"sources":[8,9],"targets":[0,1,2],"expression":"((a.cboolean2 = true) and (b.cfloat > 0.0))","edgeType":"PREDICATE"},{"sources":[7],"targets":[0,1,2],"expression":"(count(default.alltypesorc.cint) > 10)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"default.dest_v3.a"},{"id":1,"vertexType":"COLUMN","vertexId":"default.dest_v3.x"},{"id":2,"vertexType":"COLUMN","vertexId":"default.dest_v3.cboolean1"},{"id":3,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"},{"id":4,"vertexType":"COLUMN","vertexId":"default.alltypesorc.csmallint"},{"id":5,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cboolean1"},{"id":6,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cstring1"},{"id":7,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cint"},{"id":8,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cboolean2"},{"id":9,"vertexType":"COLUMN","vertexId":"default.alltyp
 esorc.cfloat"}]}
+{"version":"1.0","engine":"mr","database":"default","hash":"949093880975cc807ad1a8003e8a8c7c","queryText":"alter view dest_v3 as\n  select * from (\n    select sum(a.ctinyint) over (partition by a.csmallint order by a.csmallint) a,\n      count(b.cstring1) x, b.cboolean1\n    from alltypesorc a join alltypesorc b on (a.cint = b.cint)\n    where a.cboolean2 = true and b.cfloat > 0\n    group by a.ctinyint, a.csmallint, b.cboolean1\n    having count(a.cint) > 10\n    order by a, x, b.cboolean1 limit 10) t","edges":[{"sources":[3,4,5,6,7],"targets":[0],"expression":"(tok_function sum (. (tok_table_or_col a) ctinyint) (tok_windowspec (tok_partitioningspec (tok_distributeby (. (tok_table_or_col a) csmallint)) (tok_orderby (tok_tabsortcolnameasc (tok_nulls_first (. (tok_table_or_col a) csmallint)))))))","edgeType":"PROJECTION"},{"sources":[6],"targets":[1],"expression":"count(default.alltypesorc.cstring1)","edgeType":"PROJECTION"},{"sources":[5],"targets":[2],"edgeType":"PROJECTION"},{"so
 urces":[7],"targets":[0,1,2],"expression":"(a.cint = b.cint)","edgeType":"PREDICATE"},{"sources":[8,9],"targets":[0,1,2],"expression":"((a.cboolean2 = true) and (b.cfloat > 0.0))","edgeType":"PREDICATE"},{"sources":[7],"targets":[0,1,2],"expression":"(count(default.alltypesorc.cint) > 10)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"default.dest_v3.a"},{"id":1,"vertexType":"COLUMN","vertexId":"default.dest_v3.x"},{"id":2,"vertexType":"COLUMN","vertexId":"default.dest_v3.cboolean1"},{"id":3,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"},{"id":4,"vertexType":"COLUMN","vertexId":"default.alltypesorc.csmallint"},{"id":5,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cboolean1"},{"id":6,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cstring1"},{"id":7,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cint"},{"id":8,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cboolean2"},{"id":9,"vertexType":"COLUMN","vertexI
 d":"default.alltypesorc.cfloat"}]}
 PREHOOK: query: select * from dest_v3 limit 2
 PREHOOK: type: QUERY
 PREHOOK: Input: default@alltypesorc
 PREHOOK: Input: default@dest_v3
 #### A masked pattern was here ####
-{"version":"1.0","engine":"mr","database":"default","hash":"40bccc0722002f798d0548b59e369e83","queryText":"select * from dest_v3 limit 2","edges":[{"sources":[3,4,5,6,7],"targets":[0],"expression":"(tok_function sum (. (tok_table_or_col $hdt$_0) ctinyint) (tok_windowspec (tok_partitioningspec (tok_distributeby (. (tok_table_or_col $hdt$_0) csmallint)) (tok_orderby (tok_tabsortcolnameasc (. (tok_table_or_col $hdt$_0) csmallint)))) (tok_windowvalues (preceding 2147483647) current)))","edgeType":"PROJECTION"},{"sources":[6],"targets":[1],"expression":"count(default.alltypesorc.cstring1)","edgeType":"PROJECTION"},{"sources":[5],"targets":[2],"edgeType":"PROJECTION"},{"sources":[8,7],"targets":[0,1,2],"expression":"((a.cboolean2 = true) and a.cint is not null)","edgeType":"PREDICATE"},{"sources":[7],"targets":[0,1,2],"expression":"(a.cint = a.cint)","edgeType":"PREDICATE"},{"sources":[9,7],"targets":[0,1,2],"expression":"((a.cfloat > 0.0) and a.cint is not null)","edgeType":"PREDICATE"},
 {"sources":[7],"targets":[0,1,2],"expression":"(count(default.alltypesorc.cint) > 10)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"dest_v3.a"},{"id":1,"vertexType":"COLUMN","vertexId":"dest_v3.x"},{"id":2,"vertexType":"COLUMN","vertexId":"dest_v3.cboolean1"},{"id":3,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"},{"id":4,"vertexType":"COLUMN","vertexId":"default.alltypesorc.csmallint"},{"id":5,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cboolean1"},{"id":6,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cstring1"},{"id":7,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cint"},{"id":8,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cboolean2"},{"id":9,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cfloat"}]}
+{"version":"1.0","engine":"mr","database":"default","hash":"40bccc0722002f798d0548b59e369e83","queryText":"select * from dest_v3 limit 2","edges":[{"sources":[3,4,5,6,7],"targets":[0],"expression":"(tok_function sum (. (tok_table_or_col $hdt$_0) ctinyint) (tok_windowspec (tok_partitioningspec (tok_distributeby (. (tok_table_or_col $hdt$_0) csmallint)) (tok_orderby (tok_tabsortcolnameasc (tok_nulls_first (. (tok_table_or_col $hdt$_0) csmallint))))) (tok_windowvalues (preceding 2147483647) current)))","edgeType":"PROJECTION"},{"sources":[6],"targets":[1],"expression":"count(default.alltypesorc.cstring1)","edgeType":"PROJECTION"},{"sources":[5],"targets":[2],"edgeType":"PROJECTION"},{"sources":[8,7],"targets":[0,1,2],"expression":"((a.cboolean2 = true) and a.cint is not null)","edgeType":"PREDICATE"},{"sources":[7],"targets":[0,1,2],"expression":"(a.cint = a.cint)","edgeType":"PREDICATE"},{"sources":[9,7],"targets":[0,1,2],"expression":"((a.cfloat > 0.0) and a.cint is not null)","edgeT
 ype":"PREDICATE"},{"sources":[7],"targets":[0,1,2],"expression":"(count(default.alltypesorc.cint) > 10)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"dest_v3.a"},{"id":1,"vertexType":"COLUMN","vertexId":"dest_v3.x"},{"id":2,"vertexType":"COLUMN","vertexId":"dest_v3.cboolean1"},{"id":3,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"},{"id":4,"vertexType":"COLUMN","vertexId":"default.alltypesorc.csmallint"},{"id":5,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cboolean1"},{"id":6,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cstring1"},{"id":7,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cint"},{"id":8,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cboolean2"},{"id":9,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cfloat"}]}
 38	216	false
 38	229	true
 PREHOOK: query: drop table if exists src_dp

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/list_bucket_query_multiskew_3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/list_bucket_query_multiskew_3.q.out b/ql/src/test/results/clientpositive/list_bucket_query_multiskew_3.q.out
index cc1379a..2e77245 100644
--- a/ql/src/test/results/clientpositive/list_bucket_query_multiskew_3.q.out
+++ b/ql/src/test/results/clientpositive/list_bucket_query_multiskew_3.q.out
@@ -396,6 +396,7 @@ STAGE PLANS:
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
+                  null sort order: 
                   sort order: 
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/list_bucket_query_oneskew_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/list_bucket_query_oneskew_2.q.out b/ql/src/test/results/clientpositive/list_bucket_query_oneskew_2.q.out
index 804ef9d..135f5e1 100644
--- a/ql/src/test/results/clientpositive/list_bucket_query_oneskew_2.q.out
+++ b/ql/src/test/results/clientpositive/list_bucket_query_oneskew_2.q.out
@@ -499,6 +499,7 @@ STAGE PLANS:
                   Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col0 (type: string)
+                    null sort order: a
                     sort order: +
                     Map-reduce partition columns: _col0 (type: string)
                     Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
@@ -687,6 +688,7 @@ STAGE PLANS:
                   Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: 484 (type: int)
+                    null sort order: a
                     sort order: +
                     Map-reduce partition columns: 484 (type: int)
                     Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/llap/tez_join_result_complex.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/tez_join_result_complex.q.out b/ql/src/test/results/clientpositive/llap/tez_join_result_complex.q.out
index 1a7e831..d3c714c 100644
--- a/ql/src/test/results/clientpositive/llap/tez_join_result_complex.q.out
+++ b/ql/src/test/results/clientpositive/llap/tez_join_result_complex.q.out
@@ -444,6 +444,7 @@ STAGE PLANS:
                       Statistics: Num rows: 2 Data size: 7501 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 2 Data size: 7501 Basic stats: COMPLETE Column stats: NONE
@@ -1666,6 +1667,7 @@ STAGE PLANS:
                       Statistics: Num rows: 2 Data size: 7501 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 2 Data size: 7501 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/louter_join_ppr.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/louter_join_ppr.q.out b/ql/src/test/results/clientpositive/louter_join_ppr.q.out
index c170031..a1d7be2 100644
--- a/ql/src/test/results/clientpositive/louter_join_ppr.q.out
+++ b/ql/src/test/results/clientpositive/louter_join_ppr.q.out
@@ -126,6 +126,7 @@ STAGE PLANS:
                 Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
@@ -146,6 +147,7 @@ STAGE PLANS:
                 Statistics: Num rows: 111 Data size: 1179 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 111 Data size: 1179 Basic stats: COMPLETE Column stats: NONE
@@ -500,6 +502,7 @@ STAGE PLANS:
                 Statistics: Num rows: 222 Data size: 2358 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 222 Data size: 2358 Basic stats: COMPLETE Column stats: NONE
@@ -520,6 +523,7 @@ STAGE PLANS:
                 Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
@@ -981,6 +985,7 @@ STAGE PLANS:
                 Statistics: Num rows: 6 Data size: 63 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 6 Data size: 63 Basic stats: COMPLETE Column stats: NONE
@@ -1001,6 +1006,7 @@ STAGE PLANS:
                 Statistics: Num rows: 12 Data size: 127 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 12 Data size: 127 Basic stats: COMPLETE Column stats: NONE
@@ -1351,6 +1357,7 @@ STAGE PLANS:
                 Statistics: Num rows: 111 Data size: 1179 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 111 Data size: 1179 Basic stats: COMPLETE Column stats: NONE
@@ -1371,6 +1378,7 @@ STAGE PLANS:
                 Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/merge3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/merge3.q.out b/ql/src/test/results/clientpositive/merge3.q.out
index 296722c..266abdf 100644
--- a/ql/src/test/results/clientpositive/merge3.q.out
+++ b/ql/src/test/results/clientpositive/merge3.q.out
@@ -4910,6 +4910,7 @@ STAGE PLANS:
               outputColumnNames: _col0, _col1, _col2
               Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
+                null sort order: 
                 sort order: 
                 Map-reduce partition columns: _col2 (type: string)
                 Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/metadataonly1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/metadataonly1.q.out b/ql/src/test/results/clientpositive/metadataonly1.q.out
index 772d948..d6d86bb 100644
--- a/ql/src/test/results/clientpositive/metadataonly1.q.out
+++ b/ql/src/test/results/clientpositive/metadataonly1.q.out
@@ -51,6 +51,7 @@ STAGE PLANS:
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
+                  null sort order: 
                   sort order: 
                   Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE
                   tag: -1
@@ -153,6 +154,7 @@ STAGE PLANS:
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
+                  null sort order: 
                   sort order: 
                   Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE
                   tag: -1
@@ -296,6 +298,7 @@ STAGE PLANS:
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                   tag: -1
@@ -436,6 +439,7 @@ STAGE PLANS:
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
+                  null sort order: 
                   sort order: 
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   tag: -1
@@ -616,6 +620,7 @@ STAGE PLANS:
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
+                  null sort order: 
                   sort order: 
                   Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE
                   tag: -1
@@ -748,6 +753,7 @@ STAGE PLANS:
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Reduce Output Operator
                 key expressions: _col0 (type: string)
+                null sort order: a
                 sort order: +
                 Map-reduce partition columns: _col0 (type: string)
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -757,6 +763,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col0 (type: string)
+              null sort order: a
               sort order: +
               Map-reduce partition columns: _col0 (type: string)
               Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE
@@ -907,6 +914,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              null sort order: 
               sort order: 
               Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
               tag: -1
@@ -1068,6 +1076,7 @@ STAGE PLANS:
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string), _col1 (type: string)
+                  null sort order: aa
                   sort order: ++
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -1307,6 +1316,7 @@ STAGE PLANS:
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -1539,6 +1549,7 @@ STAGE PLANS:
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
+                  null sort order: 
                   sort order: 
                   Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE
                   tag: -1
@@ -1782,6 +1793,7 @@ STAGE PLANS:
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string), _col1 (type: string)
+                  null sort order: aa
                   sort order: ++
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/optimize_nullscan.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/optimize_nullscan.q.out b/ql/src/test/results/clientpositive/optimize_nullscan.q.out
index a814934..023b71b 100644
--- a/ql/src/test/results/clientpositive/optimize_nullscan.q.out
+++ b/ql/src/test/results/clientpositive/optimize_nullscan.q.out
@@ -194,6 +194,7 @@ STAGE PLANS:
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -333,6 +334,7 @@ STAGE PLANS:
                 Number of rows: 0
                 Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
                 Reduce Output Operator
+                  null sort order: 
                   sort order: 
                   Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
                   tag: -1
@@ -573,6 +575,7 @@ STAGE PLANS:
                 Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
@@ -582,6 +585,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col0 (type: string)
+              null sort order: a
               sort order: +
               Map-reduce partition columns: _col0 (type: string)
               Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
@@ -792,6 +796,7 @@ STAGE PLANS:
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
+                  null sort order: 
                   sort order: 
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   tag: -1
@@ -988,6 +993,7 @@ STAGE PLANS:
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
+                  null sort order: 
                   sort order: 
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   tag: -1
@@ -1312,6 +1318,7 @@ STAGE PLANS:
                 Number of rows: 0
                 Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
                 Reduce Output Operator
+                  null sort order: 
                   sort order: 
                   Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
                   tag: -1
@@ -1551,6 +1558,7 @@ STAGE PLANS:
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
+                  null sort order: 
                   sort order: 
                   Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
                   tag: 0
@@ -1559,6 +1567,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              null sort order: 
               sort order: 
               Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
               tag: 1
@@ -1776,6 +1785,7 @@ STAGE PLANS:
               Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
                 key expressions: key (type: string)
+                null sort order: a
                 sort order: +
                 Map-reduce partition columns: key (type: string)
                 Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
@@ -1791,6 +1801,7 @@ STAGE PLANS:
               Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
               Reduce Output Operator
                 key expressions: key (type: string)
+                null sort order: a
                 sort order: +
                 Map-reduce partition columns: key (type: string)
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -2082,6 +2093,7 @@ STAGE PLANS:
               Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
                 key expressions: value (type: string)
+                null sort order: a
                 sort order: +
                 Map-reduce partition columns: value (type: string)
                 Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
@@ -2098,6 +2110,7 @@ STAGE PLANS:
               Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
                 key expressions: value (type: string)
+                null sort order: a
                 sort order: +
                 Map-reduce partition columns: value (type: string)
                 Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
@@ -2253,6 +2266,7 @@ STAGE PLANS:
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
+                  null sort order: 
                   sort order: 
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                   tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/orc_analyze.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/orc_analyze.q.out b/ql/src/test/results/clientpositive/orc_analyze.q.out
index 87855fa..9b7e7b7 100644
--- a/ql/src/test/results/clientpositive/orc_analyze.q.out
+++ b/ql/src/test/results/clientpositive/orc_analyze.q.out
@@ -917,7 +917,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	4                   	 
 Bucket Columns:     	[first_name]        	 
-Sort Columns:       	[Order(col:last_name, order:1)]	 
+Sort Columns:       	[Order(col:last_name, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: desc formatted orc_create_people partition(state="Or")
@@ -960,7 +960,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	4                   	 
 Bucket Columns:     	[first_name]        	 
-Sort Columns:       	[Order(col:last_name, order:1)]	 
+Sort Columns:       	[Order(col:last_name, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: analyze table orc_create_people partition(state) compute statistics partialscan
@@ -1015,7 +1015,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	4                   	 
 Bucket Columns:     	[first_name]        	 
-Sort Columns:       	[Order(col:last_name, order:1)]	 
+Sort Columns:       	[Order(col:last_name, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: desc formatted orc_create_people partition(state="Or")
@@ -1058,7 +1058,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	4                   	 
 Bucket Columns:     	[first_name]        	 
-Sort Columns:       	[Order(col:last_name, order:1)]	 
+Sort Columns:       	[Order(col:last_name, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: analyze table orc_create_people partition(state) compute statistics noscan
@@ -1113,7 +1113,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	4                   	 
 Bucket Columns:     	[first_name]        	 
-Sort Columns:       	[Order(col:last_name, order:1)]	 
+Sort Columns:       	[Order(col:last_name, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: desc formatted orc_create_people partition(state="Or")
@@ -1156,7 +1156,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	4                   	 
 Bucket Columns:     	[first_name]        	 
-Sort Columns:       	[Order(col:last_name, order:1)]	 
+Sort Columns:       	[Order(col:last_name, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: drop table orc_create_people
@@ -1262,7 +1262,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	4                   	 
 Bucket Columns:     	[first_name]        	 
-Sort Columns:       	[Order(col:last_name, order:1)]	 
+Sort Columns:       	[Order(col:last_name, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: desc formatted orc_create_people partition(state="Or")
@@ -1305,7 +1305,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	4                   	 
 Bucket Columns:     	[first_name]        	 
-Sort Columns:       	[Order(col:last_name, order:1)]	 
+Sort Columns:       	[Order(col:last_name, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: drop table orc_create_people

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/order_null.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/order_null.q.out b/ql/src/test/results/clientpositive/order_null.q.out
new file mode 100644
index 0000000..23482b9
--- /dev/null
+++ b/ql/src/test/results/clientpositive/order_null.q.out
@@ -0,0 +1,222 @@
+PREHOOK: query: create table src_null (a int, b string)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@src_null
+POSTHOOK: query: create table src_null (a int, b string)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@src_null
+PREHOOK: query: insert into src_null values (1, 'A')
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__1
+PREHOOK: Output: default@src_null
+POSTHOOK: query: insert into src_null values (1, 'A')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__1
+POSTHOOK: Output: default@src_null
+POSTHOOK: Lineage: src_null.a EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+POSTHOOK: Lineage: src_null.b SIMPLE [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+PREHOOK: query: insert into src_null values (null, null)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__2
+PREHOOK: Output: default@src_null
+POSTHOOK: query: insert into src_null values (null, null)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__2
+POSTHOOK: Output: default@src_null
+POSTHOOK: Lineage: src_null.a EXPRESSION [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+POSTHOOK: Lineage: src_null.b SIMPLE [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+PREHOOK: query: insert into src_null values (3, null)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__3
+PREHOOK: Output: default@src_null
+POSTHOOK: query: insert into src_null values (3, null)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__3
+POSTHOOK: Output: default@src_null
+POSTHOOK: Lineage: src_null.a EXPRESSION [(values__tmp__table__3)values__tmp__table__3.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+POSTHOOK: Lineage: src_null.b SIMPLE [(values__tmp__table__3)values__tmp__table__3.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+PREHOOK: query: insert into src_null values (2, null)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__4
+PREHOOK: Output: default@src_null
+POSTHOOK: query: insert into src_null values (2, null)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__4
+POSTHOOK: Output: default@src_null
+POSTHOOK: Lineage: src_null.a EXPRESSION [(values__tmp__table__4)values__tmp__table__4.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+POSTHOOK: Lineage: src_null.b SIMPLE [(values__tmp__table__4)values__tmp__table__4.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+PREHOOK: query: insert into src_null values (2, 'A')
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__5
+PREHOOK: Output: default@src_null
+POSTHOOK: query: insert into src_null values (2, 'A')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__5
+POSTHOOK: Output: default@src_null
+POSTHOOK: Lineage: src_null.a EXPRESSION [(values__tmp__table__5)values__tmp__table__5.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+POSTHOOK: Lineage: src_null.b SIMPLE [(values__tmp__table__5)values__tmp__table__5.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+PREHOOK: query: insert into src_null values (2, 'B')
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__6
+PREHOOK: Output: default@src_null
+POSTHOOK: query: insert into src_null values (2, 'B')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__6
+POSTHOOK: Output: default@src_null
+POSTHOOK: Lineage: src_null.a EXPRESSION [(values__tmp__table__6)values__tmp__table__6.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+POSTHOOK: Lineage: src_null.b SIMPLE [(values__tmp__table__6)values__tmp__table__6.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+PREHOOK: query: SELECT x.* FROM src_null x ORDER BY a asc
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src_null
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT x.* FROM src_null x ORDER BY a asc
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src_null
+#### A masked pattern was here ####
+NULL	NULL
+1	A
+2	B
+2	A
+2	NULL
+3	NULL
+PREHOOK: query: SELECT x.* FROM src_null x ORDER BY a desc
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src_null
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT x.* FROM src_null x ORDER BY a desc
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src_null
+#### A masked pattern was here ####
+3	NULL
+2	B
+2	A
+2	NULL
+1	A
+NULL	NULL
+PREHOOK: query: SELECT x.* FROM src_null x ORDER BY b asc, a asc nulls last
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src_null
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT x.* FROM src_null x ORDER BY b asc, a asc nulls last
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src_null
+#### A masked pattern was here ####
+2	NULL
+3	NULL
+NULL	NULL
+1	A
+2	A
+2	B
+PREHOOK: query: SELECT x.* FROM src_null x ORDER BY b desc, a asc
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src_null
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT x.* FROM src_null x ORDER BY b desc, a asc
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src_null
+#### A masked pattern was here ####
+2	B
+1	A
+2	A
+NULL	NULL
+2	NULL
+3	NULL
+PREHOOK: query: SELECT x.* FROM src_null x ORDER BY a asc nulls first
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src_null
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT x.* FROM src_null x ORDER BY a asc nulls first
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src_null
+#### A masked pattern was here ####
+NULL	NULL
+1	A
+2	B
+2	A
+2	NULL
+3	NULL
+PREHOOK: query: SELECT x.* FROM src_null x ORDER BY a desc nulls first
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src_null
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT x.* FROM src_null x ORDER BY a desc nulls first
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src_null
+#### A masked pattern was here ####
+NULL	NULL
+3	NULL
+2	B
+2	A
+2	NULL
+1	A
+PREHOOK: query: SELECT x.* FROM src_null x ORDER BY b asc nulls last, a
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src_null
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT x.* FROM src_null x ORDER BY b asc nulls last, a
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src_null
+#### A masked pattern was here ####
+1	A
+2	A
+2	B
+NULL	NULL
+2	NULL
+3	NULL
+PREHOOK: query: SELECT x.* FROM src_null x ORDER BY b desc nulls last, a
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src_null
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT x.* FROM src_null x ORDER BY b desc nulls last, a
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src_null
+#### A masked pattern was here ####
+2	B
+1	A
+2	A
+NULL	NULL
+2	NULL
+3	NULL
+PREHOOK: query: SELECT x.* FROM src_null x ORDER BY a asc nulls last, b desc
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src_null
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT x.* FROM src_null x ORDER BY a asc nulls last, b desc
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src_null
+#### A masked pattern was here ####
+1	A
+2	B
+2	A
+2	NULL
+3	NULL
+NULL	NULL
+PREHOOK: query: SELECT x.* FROM src_null x ORDER BY b desc nulls last, a desc nulls last
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src_null
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT x.* FROM src_null x ORDER BY b desc nulls last, a desc nulls last
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src_null
+#### A masked pattern was here ####
+2	B
+2	A
+1	A
+3	NULL
+2	NULL
+NULL	NULL
+PREHOOK: query: SELECT x.* FROM src_null x ORDER BY b asc nulls first, a asc nulls last
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src_null
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT x.* FROM src_null x ORDER BY b asc nulls first, a asc nulls last
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src_null
+#### A masked pattern was here ####
+2	NULL
+3	NULL
+NULL	NULL
+1	A
+2	A
+2	B

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/outer_join_ppr.q.java1.7.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/outer_join_ppr.q.java1.7.out b/ql/src/test/results/clientpositive/outer_join_ppr.q.java1.7.out
index e54ef21..c471301 100644
--- a/ql/src/test/results/clientpositive/outer_join_ppr.q.java1.7.out
+++ b/ql/src/test/results/clientpositive/outer_join_ppr.q.java1.7.out
@@ -124,6 +124,7 @@ STAGE PLANS:
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
                 key expressions: _col0 (type: string)
+                null sort order: a
                 sort order: +
                 Map-reduce partition columns: _col0 (type: string)
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -140,6 +141,7 @@ STAGE PLANS:
               Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
                 key expressions: _col0 (type: string)
+                null sort order: a
                 sort order: +
                 Map-reduce partition columns: _col0 (type: string)
                 Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
@@ -597,6 +599,7 @@ STAGE PLANS:
                 Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
@@ -617,6 +620,7 @@ STAGE PLANS:
                 Statistics: Num rows: 111 Data size: 1179 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 111 Data size: 1179 Basic stats: COMPLETE Column stats: NONE


[05/18] hive git commit: HIVE-12994: Implement support for NULLS FIRST/NULLS LAST (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/tez/unionDistinct_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/unionDistinct_1.q.out b/ql/src/test/results/clientpositive/tez/unionDistinct_1.q.out
index de3cf5b..ea92dac 100644
--- a/ql/src/test/results/clientpositive/tez/unionDistinct_1.q.out
+++ b/ql/src/test/results/clientpositive/tez/unionDistinct_1.q.out
@@ -6644,6 +6644,7 @@ STAGE PLANS:
                         Statistics: Num rows: 348 Data size: 9684 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
                           key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string)
+                          null sort order: aaaa
                           sort order: ++++
                           Map-reduce partition columns: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string)
                           Statistics: Num rows: 348 Data size: 9684 Basic stats: COMPLETE Column stats: NONE
@@ -6742,6 +6743,7 @@ STAGE PLANS:
                             Statistics: Num rows: 348 Data size: 9684 Basic stats: COMPLETE Column stats: NONE
                             Reduce Output Operator
                               key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string)
+                              null sort order: aaaa
                               sort order: ++++
                               Map-reduce partition columns: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string)
                               Statistics: Num rows: 348 Data size: 9684 Basic stats: COMPLETE Column stats: NONE
@@ -6813,6 +6815,7 @@ STAGE PLANS:
                       Statistics: Num rows: 55 Data size: 1862 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 55 Data size: 1862 Basic stats: COMPLETE Column stats: NONE
@@ -8014,6 +8017,7 @@ STAGE PLANS:
                         Statistics: Num rows: 206 Data size: 988 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
                           key expressions: _col0 (type: string), _col1 (type: bigint)
+                          null sort order: aa
                           sort order: ++
                           Map-reduce partition columns: _col0 (type: string), _col1 (type: bigint)
                           Statistics: Num rows: 206 Data size: 988 Basic stats: COMPLETE Column stats: NONE
@@ -8090,6 +8094,7 @@ STAGE PLANS:
                         Statistics: Num rows: 103 Data size: 494 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
                           key expressions: _col0 (type: string)
+                          null sort order: a
                           sort order: +
                           Map-reduce partition columns: _col0 (type: string)
                           Statistics: Num rows: 103 Data size: 494 Basic stats: COMPLETE Column stats: NONE
@@ -8166,6 +8171,7 @@ STAGE PLANS:
                         Statistics: Num rows: 206 Data size: 988 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
                           key expressions: _col0 (type: string), _col1 (type: bigint)
+                          null sort order: aa
                           sort order: ++
                           Map-reduce partition columns: _col0 (type: string), _col1 (type: bigint)
                           Statistics: Num rows: 206 Data size: 988 Basic stats: COMPLETE Column stats: NONE
@@ -8241,6 +8247,7 @@ STAGE PLANS:
                         Statistics: Num rows: 206 Data size: 988 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
                           key expressions: _col0 (type: string), _col1 (type: bigint)
+                          null sort order: aa
                           sort order: ++
                           Map-reduce partition columns: _col0 (type: string), _col1 (type: bigint)
                           Statistics: Num rows: 206 Data size: 988 Basic stats: COMPLETE Column stats: NONE
@@ -8311,6 +8318,7 @@ STAGE PLANS:
                   Statistics: Num rows: 154 Data size: 738 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col0 (type: string), _col1 (type: bigint)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: _col0 (type: string), _col1 (type: bigint)
                     Statistics: Num rows: 154 Data size: 738 Basic stats: COMPLETE Column stats: NONE
@@ -8331,6 +8339,7 @@ STAGE PLANS:
                   Statistics: Num rows: 206 Data size: 988 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col0 (type: string), _col1 (type: bigint)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: _col0 (type: string), _col1 (type: bigint)
                     Statistics: Num rows: 206 Data size: 988 Basic stats: COMPLETE Column stats: NONE
@@ -8351,6 +8360,7 @@ STAGE PLANS:
                   Statistics: Num rows: 154 Data size: 738 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col0 (type: string), _col1 (type: bigint)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: _col0 (type: string), _col1 (type: bigint)
                     Statistics: Num rows: 154 Data size: 738 Basic stats: COMPLETE Column stats: NONE
@@ -8625,6 +8635,7 @@ STAGE PLANS:
                         Statistics: Num rows: 206 Data size: 988 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
                           key expressions: _col0 (type: string), _col1 (type: bigint)
+                          null sort order: aa
                           sort order: ++
                           Map-reduce partition columns: _col0 (type: string), _col1 (type: bigint)
                           Statistics: Num rows: 206 Data size: 988 Basic stats: COMPLETE Column stats: NONE
@@ -8700,6 +8711,7 @@ STAGE PLANS:
                         Statistics: Num rows: 206 Data size: 988 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
                           key expressions: _col0 (type: string), _col1 (type: bigint)
+                          null sort order: aa
                           sort order: ++
                           Map-reduce partition columns: _col0 (type: string), _col1 (type: bigint)
                           Statistics: Num rows: 206 Data size: 988 Basic stats: COMPLETE Column stats: NONE
@@ -8792,6 +8804,7 @@ STAGE PLANS:
                             Statistics: Num rows: 216 Data size: 1037 Basic stats: COMPLETE Column stats: NONE
                             Reduce Output Operator
                               key expressions: _col0 (type: string), _col1 (type: bigint)
+                              null sort order: aa
                               sort order: ++
                               Map-reduce partition columns: _col0 (type: string), _col1 (type: bigint)
                               Statistics: Num rows: 216 Data size: 1037 Basic stats: COMPLETE Column stats: NONE
@@ -8862,6 +8875,7 @@ STAGE PLANS:
                       Statistics: Num rows: 103 Data size: 494 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 103 Data size: 494 Basic stats: COMPLETE Column stats: NONE
@@ -8932,6 +8946,7 @@ STAGE PLANS:
                   Statistics: Num rows: 216 Data size: 1037 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col0 (type: string), _col1 (type: bigint)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: _col0 (type: string), _col1 (type: bigint)
                     Statistics: Num rows: 216 Data size: 1037 Basic stats: COMPLETE Column stats: NONE
@@ -9203,6 +9218,7 @@ STAGE PLANS:
                         Statistics: Num rows: 206 Data size: 988 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
                           key expressions: _col0 (type: string), _col1 (type: bigint)
+                          null sort order: aa
                           sort order: ++
                           Map-reduce partition columns: _col0 (type: string), _col1 (type: bigint)
                           Statistics: Num rows: 206 Data size: 988 Basic stats: COMPLETE Column stats: NONE
@@ -9278,6 +9294,7 @@ STAGE PLANS:
                         Statistics: Num rows: 206 Data size: 988 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
                           key expressions: _col0 (type: string), _col1 (type: bigint)
+                          null sort order: aa
                           sort order: ++
                           Map-reduce partition columns: _col0 (type: string), _col1 (type: bigint)
                           Statistics: Num rows: 206 Data size: 988 Basic stats: COMPLETE Column stats: NONE
@@ -9367,6 +9384,7 @@ STAGE PLANS:
                           Statistics: Num rows: 113 Data size: 543 Basic stats: COMPLETE Column stats: NONE
                           Reduce Output Operator
                             key expressions: _col0 (type: string)
+                            null sort order: a
                             sort order: +
                             Map-reduce partition columns: _col0 (type: string)
                             Statistics: Num rows: 113 Data size: 543 Basic stats: COMPLETE Column stats: NONE
@@ -9438,6 +9456,7 @@ STAGE PLANS:
                       Statistics: Num rows: 103 Data size: 494 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 103 Data size: 494 Basic stats: COMPLETE Column stats: NONE
@@ -9507,6 +9526,7 @@ STAGE PLANS:
                   Statistics: Num rows: 159 Data size: 763 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col0 (type: string), _col1 (type: bigint)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: _col0 (type: string), _col1 (type: bigint)
                     Statistics: Num rows: 159 Data size: 763 Basic stats: COMPLETE Column stats: NONE
@@ -9558,6 +9578,7 @@ STAGE PLANS:
                   Statistics: Num rows: 159 Data size: 763 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col0 (type: string), _col1 (type: bigint)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: _col0 (type: string), _col1 (type: bigint)
                     Statistics: Num rows: 159 Data size: 763 Basic stats: COMPLETE Column stats: NONE


[11/18] hive git commit: HIVE-12994: Implement support for NULLS FIRST/NULLS LAST (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_1.q.out b/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_1.q.out
index f580ac4..c32a0dd 100644
--- a/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_1.q.out
+++ b/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_1.q.out
@@ -181,6 +181,7 @@ STAGE PLANS:
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           Reduce Output Operator
+                            null sort order: 
                             sort order: 
                             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                             tag: -1
@@ -412,6 +413,7 @@ STAGE PLANS:
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           Reduce Output Operator
+                            null sort order: 
                             sort order: 
                             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                             tag: -1
@@ -727,6 +729,7 @@ STAGE PLANS:
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           Reduce Output Operator
+                            null sort order: 
                             sort order: 
                             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                             tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_12.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_12.q.out b/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_12.q.out
index 166b837..d004ff3 100644
--- a/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_12.q.out
+++ b/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_12.q.out
@@ -472,6 +472,7 @@ STAGE PLANS:
                             outputColumnNames: _col0
                             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                             Reduce Output Operator
+                              null sort order: 
                               sort order: 
                               Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                               tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_2.q.out b/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_2.q.out
index 945133d..3eef456 100644
--- a/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_2.q.out
+++ b/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_2.q.out
@@ -161,6 +161,7 @@ STAGE PLANS:
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           Reduce Output Operator
+                            null sort order: 
                             sort order: 
                             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                             tag: -1
@@ -478,6 +479,7 @@ STAGE PLANS:
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           Reduce Output Operator
+                            null sort order: 
                             sort order: 
                             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                             tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_3.q.out b/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_3.q.out
index de0b4cd..00d5138 100644
--- a/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_3.q.out
+++ b/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_3.q.out
@@ -161,6 +161,7 @@ STAGE PLANS:
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           Reduce Output Operator
+                            null sort order: 
                             sort order: 
                             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                             tag: -1
@@ -346,6 +347,7 @@ STAGE PLANS:
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           Reduce Output Operator
+                            null sort order: 
                             sort order: 
                             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                             tag: -1
@@ -661,6 +663,7 @@ STAGE PLANS:
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           Reduce Output Operator
+                            null sort order: 
                             sort order: 
                             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                             tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_4.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_4.q.out b/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_4.q.out
index 8707e82..b06df04 100644
--- a/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_4.q.out
+++ b/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_4.q.out
@@ -177,6 +177,7 @@ STAGE PLANS:
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           Reduce Output Operator
+                            null sort order: 
                             sort order: 
                             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                             tag: -1
@@ -362,6 +363,7 @@ STAGE PLANS:
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           Reduce Output Operator
+                            null sort order: 
                             sort order: 
                             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                             tag: -1
@@ -677,6 +679,7 @@ STAGE PLANS:
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           Reduce Output Operator
+                            null sort order: 
                             sort order: 
                             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                             tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_5.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_5.q.out b/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_5.q.out
index 9178441..18ab5c8 100644
--- a/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_5.q.out
+++ b/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_5.q.out
@@ -148,6 +148,7 @@ STAGE PLANS:
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           Reduce Output Operator
+                            null sort order: 
                             sort order: 
                             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                             tag: -1
@@ -324,6 +325,7 @@ STAGE PLANS:
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           Reduce Output Operator
+                            null sort order: 
                             sort order: 
                             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                             tag: -1
@@ -581,6 +583,7 @@ STAGE PLANS:
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           Reduce Output Operator
+                            null sort order: 
                             sort order: 
                             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                             tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_7.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_7.q.out b/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_7.q.out
index 91dc16e..dd01e69 100644
--- a/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_7.q.out
+++ b/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_7.q.out
@@ -194,6 +194,7 @@ STAGE PLANS:
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           Reduce Output Operator
+                            null sort order: 
                             sort order: 
                             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                             tag: -1
@@ -427,6 +428,7 @@ STAGE PLANS:
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           Reduce Output Operator
+                            null sort order: 
                             sort order: 
                             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                             tag: -1
@@ -790,6 +792,7 @@ STAGE PLANS:
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           Reduce Output Operator
+                            null sort order: 
                             sort order: 
                             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                             tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_8.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_8.q.out b/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_8.q.out
index e3ae9a4..f0f27f0 100644
--- a/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_8.q.out
+++ b/ql/src/test/results/clientpositive/spark/auto_sortmerge_join_8.q.out
@@ -194,6 +194,7 @@ STAGE PLANS:
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           Reduce Output Operator
+                            null sort order: 
                             sort order: 
                             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                             tag: -1
@@ -427,6 +428,7 @@ STAGE PLANS:
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           Reduce Output Operator
+                            null sort order: 
                             sort order: 
                             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                             tag: -1
@@ -792,6 +794,7 @@ STAGE PLANS:
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           Reduce Output Operator
+                            null sort order: 
                             sort order: 
                             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                             tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/bucket2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/bucket2.q.out b/ql/src/test/results/clientpositive/spark/bucket2.q.out
index 8d0795f..f4f87c2 100644
--- a/ql/src/test/results/clientpositive/spark/bucket2.q.out
+++ b/ql/src/test/results/clientpositive/spark/bucket2.q.out
@@ -58,6 +58,7 @@ STAGE PLANS:
                     outputColumnNames: _col0, _col1
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
+                      null sort order: 
                       sort order: 
                       Map-reduce partition columns: UDFToInteger(_col0) (type: int)
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/bucket3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/bucket3.q.out b/ql/src/test/results/clientpositive/spark/bucket3.q.out
index 6cb6ef6..96b2818 100644
--- a/ql/src/test/results/clientpositive/spark/bucket3.q.out
+++ b/ql/src/test/results/clientpositive/spark/bucket3.q.out
@@ -62,6 +62,7 @@ STAGE PLANS:
                     outputColumnNames: _col0, _col1
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
+                      null sort order: 
                       sort order: 
                       Map-reduce partition columns: UDFToInteger(_col0) (type: int)
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/bucket4.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/bucket4.q.out b/ql/src/test/results/clientpositive/spark/bucket4.q.out
index ef101cd..4049f22 100644
--- a/ql/src/test/results/clientpositive/spark/bucket4.q.out
+++ b/ql/src/test/results/clientpositive/spark/bucket4.q.out
@@ -55,6 +55,7 @@ STAGE PLANS:
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: UDFToInteger(_col0) (type: int)
+                      null sort order: a
                       sort order: +
                       Map-reduce partition columns: UDFToInteger(_col0) (type: int)
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/bucket5.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/bucket5.q.out b/ql/src/test/results/clientpositive/spark/bucket5.q.out
index 547b0cd..af6d399 100644
--- a/ql/src/test/results/clientpositive/spark/bucket5.q.out
+++ b/ql/src/test/results/clientpositive/spark/bucket5.q.out
@@ -93,6 +93,7 @@ STAGE PLANS:
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: UDFToInteger(_col0) (type: int)
+                      null sort order: a
                       sort order: +
                       Map-reduce partition columns: UDFToInteger(_col0) (type: int)
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -160,6 +161,7 @@ STAGE PLANS:
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: string)
+                      null sort order: a
                       sort order: +
                       Map-reduce partition columns: _col0 (type: string)
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -389,7 +391,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	2                   	 
 Bucket Columns:     	[key]               	 
-Sort Columns:       	[Order(col:key, order:1)]	 
+Sort Columns:       	[Order(col:key, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: SELECT * FROM bucketed_table TABLESAMPLE (BUCKET 1 OUT OF 2) s LIMIT 10

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/bucket_map_join_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/bucket_map_join_1.q.out b/ql/src/test/results/clientpositive/spark/bucket_map_join_1.q.out
index d6b6eca..fd17425 100644
--- a/ql/src/test/results/clientpositive/spark/bucket_map_join_1.q.out
+++ b/ql/src/test/results/clientpositive/spark/bucket_map_join_1.q.out
@@ -210,6 +210,7 @@ STAGE PLANS:
                         outputColumnNames: _col0
                         Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
+                          null sort order: 
                           sort order: 
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/bucket_map_join_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/bucket_map_join_2.q.out b/ql/src/test/results/clientpositive/spark/bucket_map_join_2.q.out
index d4aee47..4884907 100644
--- a/ql/src/test/results/clientpositive/spark/bucket_map_join_2.q.out
+++ b/ql/src/test/results/clientpositive/spark/bucket_map_join_2.q.out
@@ -210,6 +210,7 @@ STAGE PLANS:
                         outputColumnNames: _col0
                         Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
+                          null sort order: 
                           sort order: 
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/bucketmapjoin10.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/bucketmapjoin10.q.out b/ql/src/test/results/clientpositive/spark/bucketmapjoin10.q.out
index 73fe1cd..8764bc3 100644
--- a/ql/src/test/results/clientpositive/spark/bucketmapjoin10.q.out
+++ b/ql/src/test/results/clientpositive/spark/bucketmapjoin10.q.out
@@ -345,6 +345,7 @@ STAGE PLANS:
                         outputColumnNames: _col0
                         Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
+                          null sort order: 
                           sort order: 
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/bucketmapjoin11.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/bucketmapjoin11.q.out b/ql/src/test/results/clientpositive/spark/bucketmapjoin11.q.out
index 97c7f8f..58ac20c 100644
--- a/ql/src/test/results/clientpositive/spark/bucketmapjoin11.q.out
+++ b/ql/src/test/results/clientpositive/spark/bucketmapjoin11.q.out
@@ -361,6 +361,7 @@ STAGE PLANS:
                         outputColumnNames: _col0
                         Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
+                          null sort order: 
                           sort order: 
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           tag: -1
@@ -758,6 +759,7 @@ STAGE PLANS:
                         outputColumnNames: _col0
                         Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
+                          null sort order: 
                           sort order: 
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/bucketmapjoin12.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/bucketmapjoin12.q.out b/ql/src/test/results/clientpositive/spark/bucketmapjoin12.q.out
index d15f251..bde9085 100644
--- a/ql/src/test/results/clientpositive/spark/bucketmapjoin12.q.out
+++ b/ql/src/test/results/clientpositive/spark/bucketmapjoin12.q.out
@@ -274,6 +274,7 @@ STAGE PLANS:
                         outputColumnNames: _col0
                         Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
+                          null sort order: 
                           sort order: 
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           tag: -1
@@ -564,6 +565,7 @@ STAGE PLANS:
                         outputColumnNames: _col0
                         Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
+                          null sort order: 
                           sort order: 
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/bucketmapjoin13.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/bucketmapjoin13.q.out b/ql/src/test/results/clientpositive/spark/bucketmapjoin13.q.out
index d9ac9c2..7a4d32a 100644
--- a/ql/src/test/results/clientpositive/spark/bucketmapjoin13.q.out
+++ b/ql/src/test/results/clientpositive/spark/bucketmapjoin13.q.out
@@ -231,6 +231,7 @@ STAGE PLANS:
                         outputColumnNames: _col0
                         Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
+                          null sort order: 
                           sort order: 
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           tag: -1
@@ -572,6 +573,7 @@ STAGE PLANS:
                         outputColumnNames: _col0
                         Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
+                          null sort order: 
                           sort order: 
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           tag: -1
@@ -869,6 +871,7 @@ STAGE PLANS:
                         outputColumnNames: _col0
                         Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
+                          null sort order: 
                           sort order: 
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           tag: -1
@@ -1168,6 +1171,7 @@ STAGE PLANS:
                         outputColumnNames: _col0
                         Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
+                          null sort order: 
                           sort order: 
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/bucketmapjoin7.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/bucketmapjoin7.q.out b/ql/src/test/results/clientpositive/spark/bucketmapjoin7.q.out
index d754df2..d89bd58 100644
--- a/ql/src/test/results/clientpositive/spark/bucketmapjoin7.q.out
+++ b/ql/src/test/results/clientpositive/spark/bucketmapjoin7.q.out
@@ -126,15 +126,17 @@ TOK_QUERY
                value
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            .
-               TOK_TABLE_OR_COL
-                  a
-               key
+            TOK_NULLS_FIRST
+               .
+                  TOK_TABLE_OR_COL
+                     a
+                  key
          TOK_TABSORTCOLNAMEASC
-            .
-               TOK_TABLE_OR_COL
-                  b
-               value
+            TOK_NULLS_FIRST
+               .
+                  TOK_TABLE_OR_COL
+                     b
+                  value
       TOK_LIMIT
          1
 
@@ -256,6 +258,7 @@ STAGE PLANS:
                         Statistics: Num rows: 755 Data size: 3025 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
                           key expressions: _col0 (type: int), _col1 (type: string)
+                          null sort order: aa
                           sort order: ++
                           Statistics: Num rows: 755 Data size: 3025 Basic stats: COMPLETE Column stats: NONE
                           tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/bucketmapjoin8.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/bucketmapjoin8.q.out b/ql/src/test/results/clientpositive/spark/bucketmapjoin8.q.out
index f29d54c..80dd175 100644
--- a/ql/src/test/results/clientpositive/spark/bucketmapjoin8.q.out
+++ b/ql/src/test/results/clientpositive/spark/bucketmapjoin8.q.out
@@ -240,6 +240,7 @@ STAGE PLANS:
                         outputColumnNames: _col0
                         Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
+                          null sort order: 
                           sort order: 
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           tag: -1
@@ -545,6 +546,7 @@ STAGE PLANS:
                         outputColumnNames: _col0
                         Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
+                          null sort order: 
                           sort order: 
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/bucketmapjoin9.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/bucketmapjoin9.q.out b/ql/src/test/results/clientpositive/spark/bucketmapjoin9.q.out
index 9d0c95f..2c3dd41 100644
--- a/ql/src/test/results/clientpositive/spark/bucketmapjoin9.q.out
+++ b/ql/src/test/results/clientpositive/spark/bucketmapjoin9.q.out
@@ -242,6 +242,7 @@ STAGE PLANS:
                         outputColumnNames: _col0
                         Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
+                          null sort order: 
                           sort order: 
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           tag: -1
@@ -569,6 +570,7 @@ STAGE PLANS:
                         outputColumnNames: _col0
                         Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
+                          null sort order: 
                           sort order: 
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/ctas.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/ctas.q.out b/ql/src/test/results/clientpositive/spark/ctas.q.out
index 175344c..490e957 100644
--- a/ql/src/test/results/clientpositive/spark/ctas.q.out
+++ b/ql/src/test/results/clientpositive/spark/ctas.q.out
@@ -702,11 +702,13 @@ TOK_CREATETABLE
                   value
          TOK_SORTBY
             TOK_TABSORTCOLNAMEASC
-               TOK_TABLE_OR_COL
-                  key
+               TOK_NULLS_FIRST
+                  TOK_TABLE_OR_COL
+                     key
             TOK_TABSORTCOLNAMEASC
-               TOK_TABLE_OR_COL
-                  value
+               TOK_NULLS_FIRST
+                  TOK_TABLE_OR_COL
+                     value
          TOK_LIMIT
             10
 
@@ -737,6 +739,7 @@ STAGE PLANS:
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: string), _col1 (type: string)
+                      null sort order: aa
                       sort order: ++
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                       tag: -1
@@ -804,6 +807,7 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 100 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col0 (type: string), _col1 (type: string)
+                    null sort order: aa
                     sort order: ++
                     Statistics: Num rows: 10 Data size: 100 Basic stats: COMPLETE Column stats: NONE
                     tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/disable_merge_for_bucketing.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/disable_merge_for_bucketing.q.out b/ql/src/test/results/clientpositive/spark/disable_merge_for_bucketing.q.out
index 0b92309..fc2ff7b 100644
--- a/ql/src/test/results/clientpositive/spark/disable_merge_for_bucketing.q.out
+++ b/ql/src/test/results/clientpositive/spark/disable_merge_for_bucketing.q.out
@@ -54,6 +54,7 @@ STAGE PLANS:
                     outputColumnNames: _col0, _col1
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
+                      null sort order: 
                       sort order: 
                       Map-reduce partition columns: UDFToInteger(_col0) (type: int)
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/filter_join_breaktask.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/filter_join_breaktask.q.out b/ql/src/test/results/clientpositive/spark/filter_join_breaktask.q.out
index 425be20..c5f16ed 100644
--- a/ql/src/test/results/clientpositive/spark/filter_join_breaktask.q.out
+++ b/ql/src/test/results/clientpositive/spark/filter_join_breaktask.q.out
@@ -162,6 +162,7 @@ STAGE PLANS:
                       Statistics: Num rows: 25 Data size: 211 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: int)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 25 Data size: 211 Basic stats: COMPLETE Column stats: NONE
@@ -233,6 +234,7 @@ STAGE PLANS:
                       Statistics: Num rows: 25 Data size: 211 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: int)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 25 Data size: 211 Basic stats: COMPLETE Column stats: NONE
@@ -305,6 +307,7 @@ STAGE PLANS:
                       Statistics: Num rows: 25 Data size: 211 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 25 Data size: 211 Basic stats: COMPLETE Column stats: NONE
@@ -373,6 +376,7 @@ STAGE PLANS:
                 Statistics: Num rows: 27 Data size: 232 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col3 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col3 (type: string)
                   Statistics: Num rows: 27 Data size: 232 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/groupby_map_ppr.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/groupby_map_ppr.q.out b/ql/src/test/results/clientpositive/spark/groupby_map_ppr.q.out
index d051057..b5eae57 100644
--- a/ql/src/test/results/clientpositive/spark/groupby_map_ppr.q.out
+++ b/ql/src/test/results/clientpositive/spark/groupby_map_ppr.q.out
@@ -125,6 +125,7 @@ STAGE PLANS:
                       Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string), _col1 (type: string)
+                        null sort order: aa
                         sort order: ++
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/groupby_map_ppr_multi_distinct.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/groupby_map_ppr_multi_distinct.q.out b/ql/src/test/results/clientpositive/spark/groupby_map_ppr_multi_distinct.q.out
index df6a889..ada2f91 100644
--- a/ql/src/test/results/clientpositive/spark/groupby_map_ppr_multi_distinct.q.out
+++ b/ql/src/test/results/clientpositive/spark/groupby_map_ppr_multi_distinct.q.out
@@ -142,6 +142,7 @@ STAGE PLANS:
                       Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
+                        null sort order: aaa
                         sort order: +++
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/groupby_ppr.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/groupby_ppr.q.out b/ql/src/test/results/clientpositive/spark/groupby_ppr.q.out
index fbeb071..97a9a6d 100644
--- a/ql/src/test/results/clientpositive/spark/groupby_ppr.q.out
+++ b/ql/src/test/results/clientpositive/spark/groupby_ppr.q.out
@@ -119,6 +119,7 @@ STAGE PLANS:
                     Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: string), _col1 (type: string)
+                      null sort order: aa
                       sort order: ++
                       Map-reduce partition columns: _col0 (type: string)
                       Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/groupby_ppr_multi_distinct.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/groupby_ppr_multi_distinct.q.out b/ql/src/test/results/clientpositive/spark/groupby_ppr_multi_distinct.q.out
index c774f87..a8724f5 100644
--- a/ql/src/test/results/clientpositive/spark/groupby_ppr_multi_distinct.q.out
+++ b/ql/src/test/results/clientpositive/spark/groupby_ppr_multi_distinct.q.out
@@ -136,6 +136,7 @@ STAGE PLANS:
                     Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
+                      null sort order: aaa
                       sort order: +++
                       Map-reduce partition columns: _col0 (type: string)
                       Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/groupby_resolution.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/groupby_resolution.q.out b/ql/src/test/results/clientpositive/spark/groupby_resolution.q.out
index bea6479..fd6e423 100644
--- a/ql/src/test/results/clientpositive/spark/groupby_resolution.q.out
+++ b/ql/src/test/results/clientpositive/spark/groupby_resolution.q.out
@@ -675,7 +675,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: 0
                         raw input shape:
                         window functions:

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/groupby_sort_1_23.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/groupby_sort_1_23.q.out b/ql/src/test/results/clientpositive/spark/groupby_sort_1_23.q.out
index 36a7d75..7937d4e 100644
--- a/ql/src/test/results/clientpositive/spark/groupby_sort_1_23.q.out
+++ b/ql/src/test/results/clientpositive/spark/groupby_sort_1_23.q.out
@@ -318,6 +318,7 @@ STAGE PLANS:
                       Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string), _col1 (type: string)
+                        null sort order: aa
                         sort order: ++
                         Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
                         Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
@@ -1205,6 +1206,7 @@ STAGE PLANS:
                       Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string), 1 (type: int), _col2 (type: string)
+                        null sort order: aaa
                         sort order: +++
                         Map-reduce partition columns: _col0 (type: string), 1 (type: int), _col2 (type: string)
                         Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
@@ -1432,6 +1434,7 @@ STAGE PLANS:
                       Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string), _col1 (type: double)
+                        null sort order: aa
                         sort order: ++
                         Map-reduce partition columns: _col0 (type: string), _col1 (type: double)
                         Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
@@ -1705,6 +1708,7 @@ STAGE PLANS:
                           Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
                           Reduce Output Operator
                             key expressions: _col0 (type: double)
+                            null sort order: a
                             sort order: +
                             Map-reduce partition columns: _col0 (type: double)
                             Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
@@ -2452,6 +2456,7 @@ STAGE PLANS:
                       Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: double)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: double)
                         Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
@@ -2760,6 +2765,7 @@ STAGE PLANS:
                         Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
                           key expressions: _col0 (type: string)
+                          null sort order: a
                           sort order: +
                           Map-reduce partition columns: _col0 (type: string)
                           Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
@@ -2841,6 +2847,7 @@ STAGE PLANS:
                         Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
                           key expressions: _col0 (type: string)
+                          null sort order: a
                           sort order: +
                           Map-reduce partition columns: _col0 (type: string)
                           Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
@@ -3133,6 +3140,7 @@ STAGE PLANS:
                         Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
                           key expressions: _col0 (type: string)
+                          null sort order: a
                           sort order: +
                           Map-reduce partition columns: _col0 (type: string)
                           Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
@@ -3214,6 +3222,7 @@ STAGE PLANS:
                         Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
                           key expressions: _col0 (type: string), _col1 (type: string)
+                          null sort order: aa
                           sort order: ++
                           Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
                           Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
@@ -3317,6 +3326,7 @@ STAGE PLANS:
                 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
@@ -3418,6 +3428,7 @@ STAGE PLANS:
                       Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/groupby_sort_skew_1_23.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/groupby_sort_skew_1_23.q.out b/ql/src/test/results/clientpositive/spark/groupby_sort_skew_1_23.q.out
index 06c0323..199405f 100644
--- a/ql/src/test/results/clientpositive/spark/groupby_sort_skew_1_23.q.out
+++ b/ql/src/test/results/clientpositive/spark/groupby_sort_skew_1_23.q.out
@@ -319,6 +319,7 @@ STAGE PLANS:
                       Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string), _col1 (type: string)
+                        null sort order: aa
                         sort order: ++
                         Map-reduce partition columns: rand() (type: double)
                         Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
@@ -389,6 +390,7 @@ STAGE PLANS:
                 Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string), _col1 (type: string)
+                  null sort order: aa
                   sort order: ++
                   Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
                   Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
@@ -1224,6 +1226,7 @@ STAGE PLANS:
                       Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string), 1 (type: int), _col2 (type: string)
+                        null sort order: aaa
                         sort order: +++
                         Map-reduce partition columns: rand() (type: double)
                         Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
@@ -1294,6 +1297,7 @@ STAGE PLANS:
                 Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string), 1 (type: int), _col2 (type: string)
+                  null sort order: aaa
                   sort order: +++
                   Map-reduce partition columns: _col0 (type: string), 1 (type: int), _col2 (type: string)
                   Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
@@ -1469,6 +1473,7 @@ STAGE PLANS:
                       Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string), _col1 (type: double)
+                        null sort order: aa
                         sort order: ++
                         Map-reduce partition columns: rand() (type: double)
                         Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
@@ -1539,6 +1544,7 @@ STAGE PLANS:
                 Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string), _col1 (type: double)
+                  null sort order: aa
                   sort order: ++
                   Map-reduce partition columns: _col0 (type: string), _col1 (type: double)
                   Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
@@ -1760,6 +1766,7 @@ STAGE PLANS:
                           Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
                           Reduce Output Operator
                             key expressions: _col0 (type: double)
+                            null sort order: a
                             sort order: +
                             Map-reduce partition columns: rand() (type: double)
                             Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
@@ -1830,6 +1837,7 @@ STAGE PLANS:
                 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: double)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: double)
                   Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
@@ -2525,6 +2533,7 @@ STAGE PLANS:
                       Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: double)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: rand() (type: double)
                         Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
@@ -2595,6 +2604,7 @@ STAGE PLANS:
                 Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: double)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: double)
                   Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
@@ -2850,6 +2860,7 @@ STAGE PLANS:
                         Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
                           key expressions: _col0 (type: string)
+                          null sort order: a
                           sort order: +
                           Map-reduce partition columns: _col0 (type: string)
                           Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
@@ -2931,6 +2942,7 @@ STAGE PLANS:
                         Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
                           key expressions: _col0 (type: string)
+                          null sort order: a
                           sort order: +
                           Map-reduce partition columns: _col0 (type: string)
                           Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
@@ -3224,6 +3236,7 @@ STAGE PLANS:
                         Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
                           key expressions: _col0 (type: string)
+                          null sort order: a
                           sort order: +
                           Map-reduce partition columns: _col0 (type: string)
                           Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
@@ -3305,6 +3318,7 @@ STAGE PLANS:
                         Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
                           key expressions: _col0 (type: string), _col1 (type: string)
+                          null sort order: aa
                           sort order: ++
                           Map-reduce partition columns: rand() (type: double)
                           Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
@@ -3408,6 +3422,7 @@ STAGE PLANS:
                 Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string), _col1 (type: string)
+                  null sort order: aa
                   sort order: ++
                   Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
                   Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
@@ -3425,6 +3440,7 @@ STAGE PLANS:
                 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
@@ -3527,6 +3543,7 @@ STAGE PLANS:
                       Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: rand() (type: double)
                         Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
@@ -3597,6 +3614,7 @@ STAGE PLANS:
                 Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/infer_bucket_sort_bucketed_table.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/infer_bucket_sort_bucketed_table.q.out b/ql/src/test/results/clientpositive/spark/infer_bucket_sort_bucketed_table.q.out
index 33d795b..6db9428 100644
--- a/ql/src/test/results/clientpositive/spark/infer_bucket_sort_bucketed_table.q.out
+++ b/ql/src/test/results/clientpositive/spark/infer_bucket_sort_bucketed_table.q.out
@@ -64,7 +64,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	3                   	 
 Bucket Columns:     	[value]             	 
-Sort Columns:       	[Order(col:value, order:1)]	 
+Sort Columns:       	[Order(col:value, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: -- If the count(*) from sampling the buckets matches the count(*) from each file, the table is

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/join17.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/join17.q.out b/ql/src/test/results/clientpositive/spark/join17.q.out
index 0b17aa3..026bf1a 100644
--- a/ql/src/test/results/clientpositive/spark/join17.q.out
+++ b/ql/src/test/results/clientpositive/spark/join17.q.out
@@ -84,6 +84,7 @@ STAGE PLANS:
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -155,6 +156,7 @@ STAGE PLANS:
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/join34.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/join34.q.out b/ql/src/test/results/clientpositive/spark/join34.q.out
index a4cbc79..bb55191 100644
--- a/ql/src/test/results/clientpositive/spark/join34.q.out
+++ b/ql/src/test/results/clientpositive/spark/join34.q.out
@@ -166,6 +166,7 @@ STAGE PLANS:
                       Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 332 Data size: 3526 Basic stats: COMPLETE Column stats: NONE
@@ -237,6 +238,7 @@ STAGE PLANS:
                       Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 332 Data size: 3526 Basic stats: COMPLETE Column stats: NONE
@@ -308,6 +310,7 @@ STAGE PLANS:
                       Statistics: Num rows: 16 Data size: 122 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 16 Data size: 122 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/join35.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/join35.q.out b/ql/src/test/results/clientpositive/spark/join35.q.out
index 598143f..60ceca0 100644
--- a/ql/src/test/results/clientpositive/spark/join35.q.out
+++ b/ql/src/test/results/clientpositive/spark/join35.q.out
@@ -182,6 +182,7 @@ STAGE PLANS:
                         Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
                           key expressions: _col0 (type: string)
+                          null sort order: a
                           sort order: +
                           Map-reduce partition columns: _col0 (type: string)
                           Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
@@ -259,6 +260,7 @@ STAGE PLANS:
                         Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
                           key expressions: _col0 (type: string)
+                          null sort order: a
                           sort order: +
                           Map-reduce partition columns: _col0 (type: string)
                           Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
@@ -330,6 +332,7 @@ STAGE PLANS:
                       Statistics: Num rows: 16 Data size: 122 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 16 Data size: 122 Basic stats: COMPLETE Column stats: NONE
@@ -396,6 +399,7 @@ STAGE PLANS:
                 Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 166 Data size: 1762 Basic stats: COMPLETE Column stats: NONE
@@ -454,6 +458,7 @@ STAGE PLANS:
                 Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 166 Data size: 1762 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/join9.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/join9.q.out b/ql/src/test/results/clientpositive/spark/join9.q.out
index 81f359e..5f26aaf 100644
--- a/ql/src/test/results/clientpositive/spark/join9.q.out
+++ b/ql/src/test/results/clientpositive/spark/join9.q.out
@@ -100,6 +100,7 @@ STAGE PLANS:
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -172,6 +173,7 @@ STAGE PLANS:
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/join_filters_overlap.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/join_filters_overlap.q.out b/ql/src/test/results/clientpositive/spark/join_filters_overlap.q.out
index 374f2ce..49d1baa 100644
--- a/ql/src/test/results/clientpositive/spark/join_filters_overlap.q.out
+++ b/ql/src/test/results/clientpositive/spark/join_filters_overlap.q.out
@@ -115,6 +115,7 @@ STAGE PLANS:
                     Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: int)
+                      null sort order: a
                       sort order: +
                       Map-reduce partition columns: _col0 (type: int)
                       Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE
@@ -186,6 +187,7 @@ STAGE PLANS:
                       Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: int)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE
@@ -257,6 +259,7 @@ STAGE PLANS:
                       Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: int)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE
@@ -487,6 +490,7 @@ STAGE PLANS:
                       Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: int)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE
@@ -554,6 +558,7 @@ STAGE PLANS:
                     Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: int)
+                      null sort order: a
                       sort order: +
                       Map-reduce partition columns: _col0 (type: int)
                       Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE
@@ -625,6 +630,7 @@ STAGE PLANS:
                       Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: int)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE
@@ -869,6 +875,7 @@ STAGE PLANS:
                       Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: int)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE
@@ -936,6 +943,7 @@ STAGE PLANS:
                     Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: int)
+                      null sort order: a
                       sort order: +
                       Map-reduce partition columns: _col0 (type: int)
                       Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE
@@ -1007,6 +1015,7 @@ STAGE PLANS:
                       Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: int)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE
@@ -1261,6 +1270,7 @@ STAGE PLANS:
                     Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: int)
+                      null sort order: a
                       sort order: +
                       Map-reduce partition columns: _col0 (type: int)
                       Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE
@@ -1328,6 +1338,7 @@ STAGE PLANS:
                     Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: int)
+                      null sort order: a
                       sort order: +
                       Map-reduce partition columns: _col0 (type: int)
                       Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE
@@ -1399,6 +1410,7 @@ STAGE PLANS:
                       Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: int)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE
@@ -1470,6 +1482,7 @@ STAGE PLANS:
                       Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: int)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE
@@ -1719,6 +1732,7 @@ STAGE PLANS:
                     Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: int)
+                      null sort order: a
                       sort order: +
                       Map-reduce partition columns: _col0 (type: int)
                       Statistics: Num rows: 3 Data size: 18 Basic stats: COMPLETE Column stats: NONE
@@ -1790,6 +1804,7 @@ STAGE PLANS:
                       Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: int)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE
@@ -1861,6 +1876,7 @@ STAGE PLANS:
                       Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: int)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE
@@ -1932,6 +1948,7 @@ STAGE PLANS:
                       Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: int)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/louter_join_ppr.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/louter_join_ppr.q.out b/ql/src/test/results/clientpositive/spark/louter_join_ppr.q.out
index d789eb9..1817ff1 100644
--- a/ql/src/test/results/clientpositive/spark/louter_join_ppr.q.out
+++ b/ql/src/test/results/clientpositive/spark/louter_join_ppr.q.out
@@ -131,6 +131,7 @@ STAGE PLANS:
                       Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
@@ -202,6 +203,7 @@ STAGE PLANS:
                       Statistics: Num rows: 111 Data size: 1179 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 111 Data size: 1179 Basic stats: COMPLETE Column stats: NONE
@@ -517,6 +519,7 @@ STAGE PLANS:
                       Statistics: Num rows: 222 Data size: 2358 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 222 Data size: 2358 Basic stats: COMPLETE Column stats: NONE
@@ -731,6 +734,7 @@ STAGE PLANS:
                       Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
@@ -1010,6 +1014,7 @@ STAGE PLANS:
                       Statistics: Num rows: 6 Data size: 63 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 6 Data size: 63 Basic stats: COMPLETE Column stats: NONE
@@ -1081,6 +1086,7 @@ STAGE PLANS:
                       Statistics: Num rows: 12 Data size: 127 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 12 Data size: 127 Basic stats: COMPLETE Column stats: NONE
@@ -1392,6 +1398,7 @@ STAGE PLANS:
                       Statistics: Num rows: 111 Data size: 1179 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 111 Data size: 1179 Basic stats: COMPLETE Column stats: NONE
@@ -1512,6 +1519,7 @@ STAGE PLANS:
                       Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/optimize_nullscan.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/optimize_nullscan.q.out b/ql/src/test/results/clientpositive/spark/optimize_nullscan.q.out
index c71a22f..2d15f65 100644
--- a/ql/src/test/results/clientpositive/spark/optimize_nullscan.q.out
+++ b/ql/src/test/results/clientpositive/spark/optimize_nullscan.q.out
@@ -121,6 +121,7 @@ STAGE PLANS:
                       Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -268,6 +269,7 @@ STAGE PLANS:
                       Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
@@ -335,6 +337,7 @@ STAGE PLANS:
                       Number of rows: 0
                       Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
                       Reduce Output Operator
+                        null sort order: 
                         sort order: 
                         Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
                         tag: -1
@@ -576,6 +579,7 @@ STAGE PLANS:
                   Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col0 (type: string)
+                    null sort order: a
                     sort order: +
                     Map-reduce partition columns: _col0 (type: string)
                     Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
@@ -688,6 +692,7 @@ STAGE PLANS:
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
+                        null sort order: 
                         sort order: 
                         Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                         tag: -1
@@ -757,6 +762,7 @@ STAGE PLANS:
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
+                        null sort order: 
                         sort order: 
                         Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                         tag: -1
@@ -1123,6 +1129,7 @@ STAGE PLANS:
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
+                        null sort order: 
                         sort order: 
                         Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
                         tag: 0
@@ -1190,6 +1197,7 @@ STAGE PLANS:
                       Number of rows: 0
                       Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
                       Reduce Output Operator
+                        null sort order: 
                         sort order: 
                         Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
                         tag: -1
@@ -1430,6 +1438,7 @@ STAGE PLANS:
                   Number of rows: 0
                   Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
                   Reduce Output Operator
+                    null sort order: 
                     sort order: 
                     Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
                     tag: 1
@@ -1634,6 +1643,7 @@ STAGE PLANS:
                     Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: key (type: string)
+                      null sort order: a
                       sort order: +
                       Map-reduce partition columns: key (type: string)
                       Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
@@ -1699,6 +1709,7 @@ STAGE PLANS:
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Reduce Output Operator
                       key expressions: key (type: string)
+                      null sort order: a
                       sort order: +
                       Map-reduce partition columns: key (type: string)
                       Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -1814,6 +1825,7 @@ STAGE PLANS:
                     Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: value (type: string)
+                      null sort order: a
                       sort order: +
                       Map-reduce partition columns: value (type: string)
                       Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
@@ -1880,6 +1892,7 @@ STAGE PLANS:
                     Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: value (type: string)
+                      null sort order: a
                       sort order: +
                       Map-reduce partition columns: value (type: string)
                       Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
@@ -2041,6 +2054,7 @@ STAGE PLANS:
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
+                        null sort order: 
                         sort order: 
                         Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                         tag: -1


[09/18] hive git commit: HIVE-12994: Implement support for NULLS FIRST/NULLS LAST (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/router_join_ppr.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/router_join_ppr.q.out b/ql/src/test/results/clientpositive/spark/router_join_ppr.q.out
index 7d32a79..853c454 100644
--- a/ql/src/test/results/clientpositive/spark/router_join_ppr.q.out
+++ b/ql/src/test/results/clientpositive/spark/router_join_ppr.q.out
@@ -131,6 +131,7 @@ STAGE PLANS:
                       Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
@@ -202,6 +203,7 @@ STAGE PLANS:
                       Statistics: Num rows: 222 Data size: 2358 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 222 Data size: 2358 Basic stats: COMPLETE Column stats: NONE
@@ -620,6 +622,7 @@ STAGE PLANS:
                       Statistics: Num rows: 111 Data size: 1179 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 111 Data size: 1179 Basic stats: COMPLETE Column stats: NONE
@@ -740,6 +743,7 @@ STAGE PLANS:
                       Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
@@ -1010,6 +1014,7 @@ STAGE PLANS:
                       Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
@@ -1081,6 +1086,7 @@ STAGE PLANS:
                       Statistics: Num rows: 111 Data size: 1179 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 111 Data size: 1179 Basic stats: COMPLETE Column stats: NONE
@@ -1396,6 +1402,7 @@ STAGE PLANS:
                       Statistics: Num rows: 12 Data size: 127 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 12 Data size: 127 Basic stats: COMPLETE Column stats: NONE
@@ -1516,6 +1523,7 @@ STAGE PLANS:
                       Statistics: Num rows: 6 Data size: 63 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 6 Data size: 63 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/sample10.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/sample10.q.out b/ql/src/test/results/clientpositive/spark/sample10.q.out
index 372545a..3130d69 100644
--- a/ql/src/test/results/clientpositive/spark/sample10.q.out
+++ b/ql/src/test/results/clientpositive/spark/sample10.q.out
@@ -77,8 +77,9 @@ TOK_QUERY
             ds
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               ds
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  ds
 
 
 STAGE DEPENDENCIES:
@@ -116,6 +117,7 @@ STAGE PLANS:
                         Statistics: Num rows: 20 Data size: 120 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
                           key expressions: _col0 (type: string)
+                          null sort order: a
                           sort order: +
                           Map-reduce partition columns: _col0 (type: string)
                           Statistics: Num rows: 20 Data size: 120 Basic stats: COMPLETE Column stats: NONE
@@ -333,6 +335,7 @@ STAGE PLANS:
                 Statistics: Num rows: 10 Data size: 60 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Statistics: Num rows: 10 Data size: 60 Basic stats: COMPLETE Column stats: NONE
                   tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/sample6.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/sample6.q.out b/ql/src/test/results/clientpositive/spark/sample6.q.out
index 70537de..42dc25b 100644
--- a/ql/src/test/results/clientpositive/spark/sample6.q.out
+++ b/ql/src/test/results/clientpositive/spark/sample6.q.out
@@ -469,11 +469,13 @@ TOK_QUERY
                   s
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               value
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  value
 
 
 STAGE DEPENDENCIES:
@@ -504,6 +506,7 @@ STAGE PLANS:
                       Statistics: Num rows: 500 Data size: 5301 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: int), _col1 (type: string)
+                        null sort order: aa
                         sort order: ++
                         Statistics: Num rows: 500 Data size: 5301 Basic stats: COMPLETE Column stats: NONE
                         tag: -1
@@ -874,11 +877,13 @@ TOK_QUERY
                   s
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               value
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  value
 
 
 STAGE DEPENDENCIES:
@@ -909,6 +914,7 @@ STAGE PLANS:
                       Statistics: Num rows: 500 Data size: 5301 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: int), _col1 (type: string)
+                        null sort order: aa
                         sort order: ++
                         Statistics: Num rows: 500 Data size: 5301 Basic stats: COMPLETE Column stats: NONE
                         tag: -1
@@ -1533,11 +1539,13 @@ TOK_QUERY
                   s
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               value
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  value
 
 
 STAGE DEPENDENCIES:
@@ -1567,6 +1575,7 @@ STAGE PLANS:
                       Statistics: Num rows: 500 Data size: 5301 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: int), _col1 (type: string)
+                        null sort order: aa
                         sort order: ++
                         Statistics: Num rows: 500 Data size: 5301 Basic stats: COMPLETE Column stats: NONE
                         tag: -1
@@ -2034,11 +2043,13 @@ TOK_QUERY
                   s
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               value
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  value
 
 
 STAGE DEPENDENCIES:
@@ -2068,6 +2079,7 @@ STAGE PLANS:
                       Statistics: Num rows: 500 Data size: 5301 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: int), _col1 (type: string)
+                        null sort order: aa
                         sort order: ++
                         Statistics: Num rows: 500 Data size: 5301 Basic stats: COMPLETE Column stats: NONE
                         tag: -1
@@ -2521,11 +2533,13 @@ TOK_QUERY
                   s
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               value
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  value
 
 
 STAGE DEPENDENCIES:
@@ -2556,6 +2570,7 @@ STAGE PLANS:
                       Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: int), _col1 (type: string)
+                        null sort order: aa
                         sort order: ++
                         Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
                         tag: -1
@@ -2856,11 +2871,13 @@ TOK_QUERY
                   s
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               value
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  value
 
 
 STAGE DEPENDENCIES:
@@ -2891,6 +2908,7 @@ STAGE PLANS:
                       Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: int), _col1 (type: string)
+                        null sort order: aa
                         sort order: ++
                         Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
                         tag: -1
@@ -3068,11 +3086,13 @@ TOK_QUERY
                   s
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               value
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  value
 
 
 STAGE DEPENDENCIES:
@@ -3103,6 +3123,7 @@ STAGE PLANS:
                       Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: int), _col1 (type: string)
+                        null sort order: aa
                         sort order: ++
                         Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                         tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/sample8.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/sample8.q.out b/ql/src/test/results/clientpositive/spark/sample8.q.out
index 48293e1..1734361 100644
--- a/ql/src/test/results/clientpositive/spark/sample8.q.out
+++ b/ql/src/test/results/clientpositive/spark/sample8.q.out
@@ -107,6 +107,7 @@ STAGE PLANS:
                     Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: key (type: string), value (type: string)
+                      null sort order: aa
                       sort order: ++
                       Map-reduce partition columns: key (type: string), value (type: string)
                       Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
@@ -175,6 +176,7 @@ STAGE PLANS:
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: key (type: string), value (type: string)
+                      null sort order: aa
                       sort order: ++
                       Map-reduce partition columns: key (type: string), value (type: string)
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/smb_mapjoin_11.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/smb_mapjoin_11.q.out b/ql/src/test/results/clientpositive/spark/smb_mapjoin_11.q.out
index 2ff157e..a14ca28 100644
--- a/ql/src/test/results/clientpositive/spark/smb_mapjoin_11.q.out
+++ b/ql/src/test/results/clientpositive/spark/smb_mapjoin_11.q.out
@@ -156,6 +156,7 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1
                         Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
+                          null sort order: 
                           sort order: 
                           Map-reduce partition columns: _col0 (type: int)
                           Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/smb_mapjoin_12.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/smb_mapjoin_12.q.out b/ql/src/test/results/clientpositive/spark/smb_mapjoin_12.q.out
index f6c92ef..59ad406 100644
--- a/ql/src/test/results/clientpositive/spark/smb_mapjoin_12.q.out
+++ b/ql/src/test/results/clientpositive/spark/smb_mapjoin_12.q.out
@@ -169,6 +169,7 @@ STAGE PLANS:
                         Statistics: Num rows: 1650 Data size: 17529 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
                           key expressions: _col0 (type: int)
+                          null sort order: a
                           sort order: +
                           Map-reduce partition columns: _col0 (type: int)
                           Statistics: Num rows: 1650 Data size: 17529 Basic stats: COMPLETE Column stats: NONE
@@ -453,6 +454,7 @@ STAGE PLANS:
                         Statistics: Num rows: 3392 Data size: 36194 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
                           key expressions: _col0 (type: int)
+                          null sort order: a
                           sort order: +
                           Map-reduce partition columns: _col0 (type: int)
                           Statistics: Num rows: 3392 Data size: 36194 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/smb_mapjoin_13.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/smb_mapjoin_13.q.out b/ql/src/test/results/clientpositive/spark/smb_mapjoin_13.q.out
index 41379b6..eee4ddb 100644
--- a/ql/src/test/results/clientpositive/spark/smb_mapjoin_13.q.out
+++ b/ql/src/test/results/clientpositive/spark/smb_mapjoin_13.q.out
@@ -112,10 +112,11 @@ TOK_QUERY
             TOK_ALLCOLREF
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            .
-               TOK_TABLE_OR_COL
-                  a
-               key
+            TOK_NULLS_FIRST
+               .
+                  TOK_TABLE_OR_COL
+                     a
+                  key
       TOK_LIMIT
          10
 
@@ -157,6 +158,7 @@ STAGE PLANS:
                         Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
                           key expressions: _col0 (type: int)
+                          null sort order: a
                           sort order: +
                           Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
                           tag: -1
@@ -322,10 +324,11 @@ TOK_QUERY
             TOK_ALLCOLREF
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            .
-               TOK_TABLE_OR_COL
-                  a
-               key
+            TOK_NULLS_FIRST
+               .
+                  TOK_TABLE_OR_COL
+                     a
+                  key
       TOK_LIMIT
          10
 
@@ -444,6 +447,7 @@ STAGE PLANS:
                         Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
                           key expressions: _col0 (type: int)
+                          null sort order: a
                           sort order: +
                           Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
                           tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/smb_mapjoin_15.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/smb_mapjoin_15.q.out b/ql/src/test/results/clientpositive/spark/smb_mapjoin_15.q.out
index 3779fb6..bafa636 100644
--- a/ql/src/test/results/clientpositive/spark/smb_mapjoin_15.q.out
+++ b/ql/src/test/results/clientpositive/spark/smb_mapjoin_15.q.out
@@ -84,10 +84,11 @@ TOK_QUERY
             TOK_ALLCOLREF
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            .
-               TOK_TABLE_OR_COL
-                  a
-               key
+            TOK_NULLS_FIRST
+               .
+                  TOK_TABLE_OR_COL
+                     a
+                  key
       TOK_LIMIT
          10
 
@@ -129,6 +130,7 @@ STAGE PLANS:
                         Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
                           key expressions: _col0 (type: int)
+                          null sort order: a
                           sort order: +
                           Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
                           tag: -1
@@ -356,10 +358,11 @@ TOK_QUERY
             TOK_ALLCOLREF
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            .
-               TOK_TABLE_OR_COL
-                  a
-               key
+            TOK_NULLS_FIRST
+               .
+                  TOK_TABLE_OR_COL
+                     a
+                  key
       TOK_LIMIT
          10
 
@@ -401,6 +404,7 @@ STAGE PLANS:
                         Statistics: Num rows: 550 Data size: 7939 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
                           key expressions: _col0 (type: int)
+                          null sort order: a
                           sort order: +
                           Statistics: Num rows: 550 Data size: 7939 Basic stats: COMPLETE Column stats: NONE
                           tag: -1
@@ -576,10 +580,11 @@ TOK_QUERY
             TOK_ALLCOLREF
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            .
-               TOK_TABLE_OR_COL
-                  a
-               key
+            TOK_NULLS_FIRST
+               .
+                  TOK_TABLE_OR_COL
+                     a
+                  key
       TOK_LIMIT
          10
 
@@ -621,6 +626,7 @@ STAGE PLANS:
                         Statistics: Num rows: 550 Data size: 7939 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
                           key expressions: _col0 (type: int)
+                          null sort order: a
                           sort order: +
                           Statistics: Num rows: 550 Data size: 7939 Basic stats: COMPLETE Column stats: NONE
                           tag: -1
@@ -796,10 +802,11 @@ TOK_QUERY
             TOK_ALLCOLREF
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            .
-               TOK_TABLE_OR_COL
-                  a
-               key
+            TOK_NULLS_FIRST
+               .
+                  TOK_TABLE_OR_COL
+                     a
+                  key
       TOK_LIMIT
          10
 
@@ -924,6 +931,7 @@ STAGE PLANS:
                         Statistics: Num rows: 550 Data size: 7939 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
                           key expressions: _col0 (type: int)
+                          null sort order: a
                           sort order: +
                           Statistics: Num rows: 550 Data size: 7939 Basic stats: COMPLETE Column stats: NONE
                           tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/subquery_in.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/subquery_in.q.out b/ql/src/test/results/clientpositive/spark/subquery_in.q.out
index 62be682..eb6ff15 100644
--- a/ql/src/test/results/clientpositive/spark/subquery_in.q.out
+++ b/ql/src/test/results/clientpositive/spark/subquery_in.q.out
@@ -331,7 +331,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col5
+                        order by: _col5 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -507,7 +507,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col5
+                        order by: _col5 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/transform_ppr1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/transform_ppr1.q.out b/ql/src/test/results/clientpositive/spark/transform_ppr1.q.out
index 2f88651..8b8bff6 100644
--- a/ql/src/test/results/clientpositive/spark/transform_ppr1.q.out
+++ b/ql/src/test/results/clientpositive/spark/transform_ppr1.q.out
@@ -135,6 +135,7 @@ STAGE PLANS:
                         Statistics: Num rows: 333 Data size: 3537 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
                           key expressions: _col1 (type: string)
+                          null sort order: a
                           sort order: +
                           Map-reduce partition columns: _col1 (type: string)
                           Statistics: Num rows: 333 Data size: 3537 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/transform_ppr2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/transform_ppr2.q.out b/ql/src/test/results/clientpositive/spark/transform_ppr2.q.out
index dd1a38e..33f0d3e 100644
--- a/ql/src/test/results/clientpositive/spark/transform_ppr2.q.out
+++ b/ql/src/test/results/clientpositive/spark/transform_ppr2.q.out
@@ -137,6 +137,7 @@ STAGE PLANS:
                         Statistics: Num rows: 333 Data size: 3537 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
                           key expressions: _col1 (type: string)
+                          null sort order: a
                           sort order: +
                           Map-reduce partition columns: _col1 (type: string)
                           Statistics: Num rows: 333 Data size: 3537 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/union24.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/union24.q.out b/ql/src/test/results/clientpositive/spark/union24.q.out
index 31f25b5..36ec00f 100644
--- a/ql/src/test/results/clientpositive/spark/union24.q.out
+++ b/ql/src/test/results/clientpositive/spark/union24.q.out
@@ -467,6 +467,7 @@ STAGE PLANS:
                         Statistics: Num rows: 103 Data size: 494 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
                           key expressions: _col0 (type: string)
+                          null sort order: a
                           sort order: +
                           Map-reduce partition columns: _col0 (type: string)
                           Statistics: Num rows: 103 Data size: 494 Basic stats: COMPLETE Column stats: NONE
@@ -940,6 +941,7 @@ STAGE PLANS:
                       Statistics: Num rows: 103 Data size: 494 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 103 Data size: 494 Basic stats: COMPLETE Column stats: NONE
@@ -1010,6 +1012,7 @@ STAGE PLANS:
                       Statistics: Num rows: 103 Data size: 494 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 103 Data size: 494 Basic stats: COMPLETE Column stats: NONE
@@ -1481,6 +1484,7 @@ STAGE PLANS:
                       Statistics: Num rows: 103 Data size: 494 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 103 Data size: 494 Basic stats: COMPLETE Column stats: NONE
@@ -1551,6 +1555,7 @@ STAGE PLANS:
                       Statistics: Num rows: 103 Data size: 494 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 103 Data size: 494 Basic stats: COMPLETE Column stats: NONE
@@ -1624,6 +1629,7 @@ STAGE PLANS:
                   Statistics: Num rows: 113 Data size: 543 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col0 (type: string)
+                    null sort order: a
                     sort order: +
                     Map-reduce partition columns: _col0 (type: string)
                     Statistics: Num rows: 113 Data size: 543 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/union_ppr.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/union_ppr.q.out b/ql/src/test/results/clientpositive/spark/union_ppr.q.out
index e951283..1d2120a 100644
--- a/ql/src/test/results/clientpositive/spark/union_ppr.q.out
+++ b/ql/src/test/results/clientpositive/spark/union_ppr.q.out
@@ -87,25 +87,29 @@ TOK_QUERY
             '2008-04-08'
       TOK_SORTBY
          TOK_TABSORTCOLNAMEASC
-            .
-               TOK_TABLE_OR_COL
-                  A
-               key
+            TOK_NULLS_FIRST
+               .
+                  TOK_TABLE_OR_COL
+                     A
+                  key
          TOK_TABSORTCOLNAMEASC
-            .
-               TOK_TABLE_OR_COL
-                  A
-               value
+            TOK_NULLS_FIRST
+               .
+                  TOK_TABLE_OR_COL
+                     A
+                  value
          TOK_TABSORTCOLNAMEASC
-            .
-               TOK_TABLE_OR_COL
-                  A
-               ds
+            TOK_NULLS_FIRST
+               .
+                  TOK_TABLE_OR_COL
+                     A
+                  ds
          TOK_TABSORTCOLNAMEASC
-            .
-               TOK_TABLE_OR_COL
-                  A
-               hr
+            TOK_NULLS_FIRST
+               .
+                  TOK_TABLE_OR_COL
+                     A
+                  hr
 
 
 STAGE DEPENDENCIES:
@@ -139,6 +143,7 @@ STAGE PLANS:
                         Statistics: Num rows: 666 Data size: 7074 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
                           key expressions: _col0 (type: string), _col1 (type: string), '2008-04-08' (type: string), _col3 (type: string)
+                          null sort order: aaaa
                           sort order: ++++
                           Statistics: Num rows: 666 Data size: 7074 Basic stats: COMPLETE Column stats: NONE
                           tag: -1
@@ -261,6 +266,7 @@ STAGE PLANS:
                         Statistics: Num rows: 666 Data size: 7074 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
                           key expressions: _col0 (type: string), _col1 (type: string), '2008-04-08' (type: string), _col3 (type: string)
+                          null sort order: aaaa
                           sort order: ++++
                           Statistics: Num rows: 666 Data size: 7074 Basic stats: COMPLETE Column stats: NONE
                           tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/union_remove_6_subq.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/union_remove_6_subq.q.out b/ql/src/test/results/clientpositive/spark/union_remove_6_subq.q.out
index 22b0fea..3c88c95 100644
--- a/ql/src/test/results/clientpositive/spark/union_remove_6_subq.q.out
+++ b/ql/src/test/results/clientpositive/spark/union_remove_6_subq.q.out
@@ -446,7 +446,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col0
                         raw input shape:
                         window functions:


[17/18] hive git commit: HIVE-12994: Implement support for NULLS FIRST/NULLS LAST (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java
index 074da6b..74cb2e0 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/MoveTask.java
@@ -593,7 +593,9 @@ public class MoveTask extends Task<MoveWork> implements Serializable {
           newSortCols.add(new Order(
             partn.getCols().get(sortCol.getIndexes().get(0)).getName(),
             sortCol.getSortOrder() == '+' ? BaseSemanticAnalyzer.HIVE_COLUMN_ORDER_ASC :
-              BaseSemanticAnalyzer.HIVE_COLUMN_ORDER_DESC));
+              BaseSemanticAnalyzer.HIVE_COLUMN_ORDER_DESC,
+            sortCol.getNullSortOrder() == 'a' ? BaseSemanticAnalyzer.HIVE_COLUMN_NULLS_FIRST :
+              BaseSemanticAnalyzer.HIVE_COLUMN_NULLS_LAST));
         } else {
           // If the table is sorted on a partition column, not valid for sorting
           updateSortCols = false;

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/HybridHashTableContainer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/HybridHashTableContainer.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/HybridHashTableContainer.java
index fdc1dff..f6471db 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/HybridHashTableContainer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/HybridHashTableContainer.java
@@ -91,6 +91,8 @@ public class HybridHashTableContainer
   /** The OI used to deserialize values. We never deserialize keys. */
   private LazyBinaryStructObjectInspector internalValueOi;
   private boolean[] sortableSortOrders;
+  private byte[] nullMarkers;
+  private byte[] notNullMarkers;
   private MapJoinBytesTableContainer.KeyValueHelper writeHelper;
   private final MapJoinBytesTableContainer.DirectKeyValueWriter directWriteHelper;
   /*
@@ -417,6 +419,14 @@ public class HybridHashTableContainer
     return sortableSortOrders;
   }
 
+  public byte[] getNullMarkers() {
+    return nullMarkers;
+  }
+
+  public byte[] getNotNullMarkers() {
+    return notNullMarkers;
+  }
+
   /* For a given row, put it into proper partition based on its hash value.
    * When memory threshold is reached, the biggest hash table in memory will be spilled to disk.
    * If the hash table of a specific partition is already on disk, all later rows will be put into
@@ -708,7 +718,8 @@ public class HybridHashTableContainer
         nulls[i] = currentKey[i] == null;
       }
       return currentValue.setFromOutput(
-          MapJoinKey.serializeRow(output, currentKey, vectorKeyOIs, sortableSortOrders));
+          MapJoinKey.serializeRow(output, currentKey, vectorKeyOIs,
+                  sortableSortOrders, nullMarkers, notNullMarkers));
     }
 
     @Override
@@ -723,7 +734,8 @@ public class HybridHashTableContainer
         nulls[keyIndex] = currentKey[keyIndex] == null;
       }
       return currentValue.setFromOutput(
-          MapJoinKey.serializeRow(output, currentKey, ois, sortableSortOrders));
+          MapJoinKey.serializeRow(output, currentKey, ois,
+                  sortableSortOrders, nullMarkers, notNullMarkers));
     }
 
     @Override
@@ -1064,6 +1076,12 @@ public class HybridHashTableContainer
       if (sortableSortOrders == null) {
         sortableSortOrders = ((BinarySortableSerDe) keySerde).getSortOrders();
       }
+      if (nullMarkers == null) {
+        nullMarkers = ((BinarySortableSerDe) keySerde).getNullMarkers();
+      }
+      if (notNullMarkers == null) {
+        notNullMarkers = ((BinarySortableSerDe) keySerde).getNotNullMarkers();
+      }
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinBytesTableContainer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinBytesTableContainer.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinBytesTableContainer.java
index 5c2ff92..a8aa71a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinBytesTableContainer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinBytesTableContainer.java
@@ -25,11 +25,8 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.ql.debug.Utils;
 import org.apache.hadoop.hive.ql.exec.ExprNodeEvaluator;
 import org.apache.hadoop.hive.ql.exec.JoinUtil;
 import org.apache.hadoop.hive.ql.exec.vector.VectorHashKeyWrapper;
@@ -53,9 +50,9 @@ import org.apache.hadoop.hive.serde2.lazybinary.objectinspector.LazyBinaryStruct
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption;
 import org.apache.hadoop.hive.serde2.objectinspector.StructField;
 import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.ShortObjectInspector;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
@@ -63,6 +60,8 @@ import org.apache.hadoop.io.BinaryComparable;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.Writable;
 import org.apache.hive.common.util.HashCodeUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Table container that serializes keys and values using LazyBinarySerDe into
@@ -83,6 +82,8 @@ public class MapJoinBytesTableContainer
    * ordering. Hence, remember the ordering here; it is null if we do use LazyBinarySerDe.
    */
   private boolean[] sortableSortOrders;
+  private byte[] nullMarkers;
+  private byte[] notNullMarkers;
   private KeyValueHelper writeHelper;
   private DirectKeyValueWriter directWriteHelper;
 
@@ -138,6 +139,14 @@ public class MapJoinBytesTableContainer
     this.sortableSortOrders = sortableSortOrders;
   }
 
+  public void setNullMarkers(byte[] nullMarkers) {
+    this.nullMarkers = nullMarkers;
+  }
+
+  public void setNotNullMarkers(byte[] notNullMarkers) {
+    this.notNullMarkers = notNullMarkers;
+  }
+
   public static interface KeyValueHelper extends BytesBytesMultiHashMap.KvSource {
     void setKeyValue(Writable key, Writable val) throws SerDeException;
     /** Get hash value from the key. */
@@ -269,7 +278,14 @@ public class MapJoinBytesTableContainer
         fois.add(fields.get(i).getFieldObjectInspector());
       }
       Output output = new Output();
-      BinarySortableSerDe.serializeStruct(output, data, fois, new boolean[fields.size()]);
+      boolean[] sortableSortOrders = new boolean[fields.size()];
+      Arrays.fill(sortableSortOrders, false);
+      byte[] columnNullMarker = new byte[fields.size()];
+      Arrays.fill(columnNullMarker, BinarySortableSerDe.ZERO);
+      byte[] columnNotNullMarker = new byte[fields.size()];
+      Arrays.fill(columnNotNullMarker, BinarySortableSerDe.ONE);
+      BinarySortableSerDe.serializeStruct(output, data, fois, sortableSortOrders,
+              columnNullMarker, columnNotNullMarker);
       hasTag = (output.getLength() != b.getLength());
       if (hasTag) {
         LOG.error("Tag found in keys and will be removed. This should not happen.");
@@ -360,10 +376,14 @@ public class MapJoinBytesTableContainer
         writeHelper = new LazyBinaryKvWriter(keySerde, valSoi, valueContext.hasFilterTag());
         internalValueOi = valSoi;
         sortableSortOrders = ((BinarySortableSerDe) keySerde).getSortOrders();
+        nullMarkers = ((BinarySortableSerDe) keySerde).getNullMarkers();
+        notNullMarkers = ((BinarySortableSerDe) keySerde).getNotNullMarkers();
       } else {
         writeHelper = new KeyValueWriter(keySerde, valSerde, valueContext.hasFilterTag());
         internalValueOi = createInternalOi(valueContext);
         sortableSortOrders = null;
+        nullMarkers = null;
+        notNullMarkers = null;
       }
     }
   }
@@ -476,7 +496,8 @@ public class MapJoinBytesTableContainer
         nulls[i] = currentKey[i] == null;
       }
       return currentValue.setFromOutput(
-          MapJoinKey.serializeRow(output, currentKey, vectorKeyOIs, sortableSortOrders));
+          MapJoinKey.serializeRow(output, currentKey, vectorKeyOIs,
+                  sortableSortOrders, nullMarkers, notNullMarkers));
     }
 
     @Override
@@ -491,7 +512,8 @@ public class MapJoinBytesTableContainer
         nulls[keyIndex] = currentKey[keyIndex] == null;
       }
       return currentValue.setFromOutput(
-          MapJoinKey.serializeRow(output, currentKey, ois, sortableSortOrders));
+          MapJoinKey.serializeRow(output, currentKey, ois,
+                  sortableSortOrders, nullMarkers, notNullMarkers));
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinKey.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinKey.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinKey.java
index cfb9abc..9f27f56 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinKey.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/MapJoinKey.java
@@ -117,7 +117,8 @@ public abstract class MapJoinKey {
    */
   public static Output serializeVector(Output byteStream, VectorHashKeyWrapper kw,
       VectorExpressionWriter[] keyOutputWriters, VectorHashKeyWrapperBatch keyWrapperBatch,
-      boolean[] nulls, boolean[] sortableSortOrders) throws HiveException, SerDeException {
+      boolean[] nulls, boolean[] sortableSortOrders, byte[] nullMarkers, byte[] notNullMarkers)
+              throws HiveException, SerDeException {
     Object[] fieldData = new Object[keyOutputWriters.length];
     List<ObjectInspector> fieldOis = new ArrayList<ObjectInspector>();
     for (int i = 0; i < keyOutputWriters.length; ++i) {
@@ -130,7 +131,8 @@ public abstract class MapJoinKey {
         nulls[i] = (fieldData[i] == null);
       }
     }
-    return serializeRow(byteStream, fieldData, fieldOis, sortableSortOrders);
+    return serializeRow(byteStream, fieldData, fieldOis, sortableSortOrders,
+            nullMarkers, notNullMarkers);
   }
 
   public static MapJoinKey readFromRow(Output output, MapJoinKey key, Object[] keyObject,
@@ -145,7 +147,8 @@ public abstract class MapJoinKey {
    * @param byteStream Output to reuse. Can be null, in that case a new one would be created.
    */
   public static Output serializeRow(Output byteStream, Object[] fieldData,
-      List<ObjectInspector> fieldOis, boolean[] sortableSortOrders) throws HiveException {
+      List<ObjectInspector> fieldOis, boolean[] sortableSortOrders,
+      byte[] nullMarkers, byte[] notNullMarkers) throws HiveException {
     if (byteStream == null) {
       byteStream = new Output();
     } else {
@@ -157,7 +160,8 @@ public abstract class MapJoinKey {
       } else if (sortableSortOrders == null) {
         LazyBinarySerDe.serializeStruct(byteStream, fieldData, fieldOis);
       } else {
-        BinarySortableSerDe.serializeStruct(byteStream, fieldData, fieldOis, sortableSortOrders);
+        BinarySortableSerDe.serializeStruct(byteStream, fieldData, fieldOis, sortableSortOrders,
+                nullMarkers, notNullMarkers);
       }
     } catch (SerDeException e) {
       throw new HiveException("Serialization error", e);

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DynamicPartitionPruner.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DynamicPartitionPruner.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DynamicPartitionPruner.java
index 1510fdd..3f16359 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DynamicPartitionPruner.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DynamicPartitionPruner.java
@@ -180,7 +180,7 @@ public class DynamicPartitionPruner {
         numExpectedEventsPerSource.get(s).decrement();
         ++sourceInfoCount;
         String columnName = cit.next();
-	String columnType = typit.next();
+        String columnType = typit.next();
         ExprNodeDesc partKeyExpr = pit.next();
         SourceInfo si = createSourceInfo(t, partKeyExpr, columnName, columnType, jobConf);
         if (!sourceInfoMap.containsKey(s)) {

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkCommonOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkCommonOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkCommonOperator.java
index 7bdd11a..8133aef 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkCommonOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkCommonOperator.java
@@ -27,9 +27,9 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.exec.Operator;
+import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator.Counter;
 import org.apache.hadoop.hive.ql.exec.TerminalOperator;
 import org.apache.hadoop.hive.ql.exec.Utilities;
-import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator.Counter;
 import org.apache.hadoop.hive.ql.exec.vector.VectorSerializeRow;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizationContext;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizationContextRegion;
@@ -39,14 +39,15 @@ import org.apache.hadoop.hive.ql.exec.vector.keyseries.VectorKeySeriesSerialized
 import org.apache.hadoop.hive.ql.io.HiveKey;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.BaseWork;
-import org.apache.hadoop.hive.ql.plan.ReduceSinkDesc;
 import org.apache.hadoop.hive.ql.plan.OperatorDesc;
+import org.apache.hadoop.hive.ql.plan.ReduceSinkDesc;
 import org.apache.hadoop.hive.ql.plan.TableDesc;
 import org.apache.hadoop.hive.ql.plan.VectorReduceSinkDesc;
 import org.apache.hadoop.hive.ql.plan.VectorReduceSinkInfo;
 import org.apache.hadoop.hive.ql.plan.api.OperatorType;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hadoop.hive.serde2.ByteStream.Output;
+import org.apache.hadoop.hive.serde2.binarysortable.BinarySortableSerDe;
 import org.apache.hadoop.hive.serde2.binarysortable.fast.BinarySortableSerializeWrite;
 import org.apache.hadoop.hive.serde2.lazybinary.fast.LazyBinarySerializeWrite;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
@@ -188,6 +189,60 @@ public abstract class VectorReduceSinkCommonOperator extends TerminalOperator<Re
     return columnSortOrderIsDesc;
   }
 
+  private byte[] getColumnNullMarker(Properties properties, int columnCount, boolean[] columnSortOrder) {
+    String columnNullOrder = properties.getProperty(serdeConstants.SERIALIZATION_NULL_SORT_ORDER);
+    byte[] columnNullMarker = new byte[columnCount];
+      for (int i = 0; i < columnNullMarker.length; i++) {
+        if (columnSortOrder[i]) {
+          // Descending
+          if (columnNullOrder != null && columnNullOrder.charAt(i) == 'a') {
+            // Null first
+            columnNullMarker[i] = BinarySortableSerDe.ONE;
+          } else {
+            // Null last (default for descending order)
+            columnNullMarker[i] = BinarySortableSerDe.ZERO;
+          }
+        } else {
+          // Ascending
+          if (columnNullOrder != null && columnNullOrder.charAt(i) == 'z') {
+            // Null last
+            columnNullMarker[i] = BinarySortableSerDe.ONE;
+          } else {
+            // Null first (default for ascending order)
+            columnNullMarker[i] = BinarySortableSerDe.ZERO;
+          }
+        }
+    }
+    return columnNullMarker;
+  }
+
+  private byte[] getColumnNotNullMarker(Properties properties, int columnCount, boolean[] columnSortOrder) {
+    String columnNullOrder = properties.getProperty(serdeConstants.SERIALIZATION_NULL_SORT_ORDER);
+    byte[] columnNotNullMarker = new byte[columnCount];
+      for (int i = 0; i < columnNotNullMarker.length; i++) {
+        if (columnSortOrder[i]) {
+          // Descending
+          if (columnNullOrder != null && columnNullOrder.charAt(i) == 'a') {
+            // Null first
+            columnNotNullMarker[i] = BinarySortableSerDe.ZERO;
+          } else {
+            // Null last (default for descending order)
+            columnNotNullMarker[i] = BinarySortableSerDe.ONE;
+          }
+        } else {
+          // Ascending
+          if (columnNullOrder != null && columnNullOrder.charAt(i) == 'z') {
+            // Null last
+            columnNotNullMarker[i] = BinarySortableSerDe.ZERO;
+          } else {
+            // Null first (default for ascending order)
+            columnNotNullMarker[i] = BinarySortableSerDe.ONE;
+          }
+        }
+    }
+    return columnNotNullMarker;
+  }
+
   @Override
   protected void initializeOp(Configuration hconf) throws HiveException {
     super.initializeOp(hconf);
@@ -217,8 +272,13 @@ public abstract class VectorReduceSinkCommonOperator extends TerminalOperator<Re
     TableDesc keyTableDesc = conf.getKeySerializeInfo();
     boolean[] columnSortOrder =
         getColumnSortOrder(keyTableDesc.getProperties(), reduceSinkKeyColumnMap.length);
+    byte[] columnNullMarker =
+        getColumnNullMarker(keyTableDesc.getProperties(), reduceSinkKeyColumnMap.length, columnSortOrder);
+    byte[] columnNotNullMarker =
+        getColumnNotNullMarker(keyTableDesc.getProperties(), reduceSinkKeyColumnMap.length, columnSortOrder);
 
-    keyBinarySortableSerializeWrite = new BinarySortableSerializeWrite(columnSortOrder);
+    keyBinarySortableSerializeWrite = new BinarySortableSerializeWrite(columnSortOrder,
+            columnNullMarker, columnNotNullMarker);
 
     // Create all nulls key.
     try {

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
index fdc7956..ad17096 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
@@ -946,7 +946,7 @@ public class Hive {
         FieldSchema col = cols.get(i);
         if (indexedCols.contains(col.getName())) {
           indexTblCols.add(col);
-          sortCols.add(new Order(col.getName(), 1));
+          sortCols.add(new Order(col.getName(), 1, 0));
           k++;
         }
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/java/org/apache/hadoop/hive/ql/optimizer/AbstractSMBJoinProc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/AbstractSMBJoinProc.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/AbstractSMBJoinProc.java
index b57dc77..677649d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/AbstractSMBJoinProc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/AbstractSMBJoinProc.java
@@ -357,7 +357,8 @@ abstract public class AbstractSMBJoinProc extends AbstractBucketJoinProc impleme
       Order o = sortCols.get(pos);
 
       if (pos < sortColumnsFirstPartition.size()) {
-        if (o.getOrder() != sortColumnsFirstPartition.get(pos).getOrder()) {
+        if (o.getOrder() != sortColumnsFirstPartition.get(pos).getOrder() ||
+                o.getNullOrder() != sortColumnsFirstPartition.get(pos).getNullOrder()) {
           return false;
         }
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketingSortingReduceSinkOptimizer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketingSortingReduceSinkOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketingSortingReduceSinkOptimizer.java
index d5f3057..3d580d8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketingSortingReduceSinkOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketingSortingReduceSinkOptimizer.java
@@ -148,23 +148,55 @@ public class BucketingSortingReduceSinkOptimizer extends Transform {
 
     // Get the sort positions and sort order for the table
     // The sort order contains whether the sorting is happening ascending or descending
-    private ObjectPair<List<Integer>, List<Integer>> getSortPositionsOrder(
+    private List<Integer> getSortPositions(
         List<Order> tabSortCols,
         List<FieldSchema> tabCols) {
       List<Integer> sortPositions = new ArrayList<Integer>();
-      List<Integer> sortOrders = new ArrayList<Integer>();
       for (Order sortCol : tabSortCols) {
         int pos = 0;
         for (FieldSchema tabCol : tabCols) {
           if (sortCol.getCol().equals(tabCol.getName())) {
             sortPositions.add(pos);
+            break;
+          }
+          pos++;
+        }
+      }
+      return sortPositions;
+    }
+
+    private List<Integer> getSortOrder(
+        List<Order> tabSortCols,
+        List<FieldSchema> tabCols) {
+      List<Integer> sortOrders = new ArrayList<Integer>();
+      for (Order sortCol : tabSortCols) {
+        int pos = 0;
+        for (FieldSchema tabCol : tabCols) {
+          if (sortCol.getCol().equals(tabCol.getName())) {
             sortOrders.add(sortCol.getOrder());
             break;
           }
           pos++;
         }
       }
-      return new ObjectPair<List<Integer>, List<Integer>>(sortPositions, sortOrders);
+      return sortOrders;
+    }
+
+    private List<Integer> getNullSortOrder(
+        List<Order> tabSortCols,
+        List<FieldSchema> tabCols) {
+      List<Integer> nullSortOrders = new ArrayList<Integer>();
+      for (Order sortCol : tabSortCols) {
+        int pos = 0;
+        for (FieldSchema tabCol : tabCols) {
+          if (sortCol.getCol().equals(tabCol.getName())) {
+            nullSortOrders.add(sortCol.getNullOrder());
+            break;
+          }
+          pos++;
+        }
+      }
+      return nullSortOrders;
     }
 
     // Return true if the partition is bucketed/sorted by the specified positions
@@ -174,6 +206,7 @@ public class BucketingSortingReduceSinkOptimizer extends Transform {
         List<Integer> bucketPositionsDest,
         List<Integer> sortPositionsDest,
         List<Integer> sortOrderDest,
+        List<Integer> sortNullOrderDest,
         int numBucketsDest) {
       // The bucketing and sorting positions should exactly match
       int numBuckets = partition.getBucketCount();
@@ -183,11 +216,16 @@ public class BucketingSortingReduceSinkOptimizer extends Transform {
 
       List<Integer> partnBucketPositions =
           getBucketPositions(partition.getBucketCols(), partition.getTable().getCols());
-      ObjectPair<List<Integer>, List<Integer>> partnSortPositionsOrder =
-          getSortPositionsOrder(partition.getSortCols(), partition.getTable().getCols());
+      List<Integer> sortPositions =
+          getSortPositions(partition.getSortCols(), partition.getTable().getCols());
+      List<Integer> sortOrder =
+          getSortOrder(partition.getSortCols(), partition.getTable().getCols());
+      List<Integer> sortNullOrder =
+          getNullSortOrder(partition.getSortCols(), partition.getTable().getCols());
       return bucketPositionsDest.equals(partnBucketPositions) &&
-          sortPositionsDest.equals(partnSortPositionsOrder.getFirst()) &&
-          sortOrderDest.equals(partnSortPositionsOrder.getSecond());
+          sortPositionsDest.equals(sortPositions) &&
+          sortOrderDest.equals(sortOrder) &&
+          sortNullOrderDest.equals(sortNullOrder);
     }
 
     // Return true if the table is bucketed/sorted by the specified positions
@@ -197,6 +235,7 @@ public class BucketingSortingReduceSinkOptimizer extends Transform {
         List<Integer> bucketPositionsDest,
         List<Integer> sortPositionsDest,
         List<Integer> sortOrderDest,
+        List<Integer> sortNullOrderDest,
         int numBucketsDest) {
       // The bucketing and sorting positions should exactly match
       int numBuckets = table.getNumBuckets();
@@ -206,11 +245,16 @@ public class BucketingSortingReduceSinkOptimizer extends Transform {
 
       List<Integer> tableBucketPositions =
           getBucketPositions(table.getBucketCols(), table.getCols());
-      ObjectPair<List<Integer>, List<Integer>> tableSortPositionsOrder =
-          getSortPositionsOrder(table.getSortCols(), table.getCols());
+      List<Integer> sortPositions =
+          getSortPositions(table.getSortCols(), table.getCols());
+      List<Integer> sortOrder =
+          getSortOrder(table.getSortCols(), table.getCols());
+      List<Integer> sortNullOrder =
+          getNullSortOrder(table.getSortCols(), table.getCols());
       return bucketPositionsDest.equals(tableBucketPositions) &&
-          sortPositionsDest.equals(tableSortPositionsOrder.getFirst()) &&
-          sortOrderDest.equals(tableSortPositionsOrder.getSecond());
+          sortPositionsDest.equals(sortPositions) &&
+          sortOrderDest.equals(sortOrder) &&
+          sortNullOrderDest.equals(sortNullOrder);
     }
 
     // Store the bucket path to bucket number mapping in the table scan operator.
@@ -288,7 +332,8 @@ public class BucketingSortingReduceSinkOptimizer extends Transform {
     private boolean validateSMBJoinKeys(SMBJoinDesc smbJoinDesc,
         List<ExprNodeColumnDesc> sourceTableBucketCols,
         List<ExprNodeColumnDesc> sourceTableSortCols,
-        List<Integer> sortOrder) {
+        List<Integer> sortOrder,
+        List<Integer> sortNullOrder) {
       // The sort-merge join creates the output sorted and bucketized by the same columns.
       // This can be relaxed in the future if there is a requirement.
       if (!sourceTableBucketCols.equals(sourceTableSortCols)) {
@@ -426,10 +471,12 @@ public class BucketingSortingReduceSinkOptimizer extends Transform {
       // also match for this to be converted to a map-only job.
       List<Integer> bucketPositions =
           getBucketPositions(destTable.getBucketCols(), destTable.getCols());
-      ObjectPair<List<Integer>, List<Integer>> sortOrderPositions =
-          getSortPositionsOrder(destTable.getSortCols(), destTable.getCols());
-      List<Integer> sortPositions = sortOrderPositions.getFirst();
-      List<Integer> sortOrder = sortOrderPositions.getSecond();
+      List<Integer> sortPositions =
+          getSortPositions(destTable.getSortCols(), destTable.getCols());
+      List<Integer> sortOrder =
+          getSortOrder(destTable.getSortCols(), destTable.getCols());
+      List<Integer> sortNullOrder =
+          getNullSortOrder(destTable.getSortCols(), destTable.getCols());
       boolean useBucketSortPositions = true;
 
       // Only selects and filters are allowed
@@ -464,7 +511,7 @@ public class BucketingSortingReduceSinkOptimizer extends Transform {
           }
 
           if (!validateSMBJoinKeys(smbJoinDesc, sourceTableBucketCols,
-              sourceTableSortCols, sortOrder)) {
+              sourceTableSortCols, sortOrder, sortNullOrder)) {
             return null;
           }
 
@@ -539,7 +586,7 @@ public class BucketingSortingReduceSinkOptimizer extends Transform {
               }
               for (Partition partition : partitions) {
                 if (!checkPartition(partition, newBucketPositions, newSortPositions, sortOrder,
-                    numBucketsDestination)) {
+                    sortNullOrder, numBucketsDestination)) {
                   return null;
                 }
               }
@@ -550,7 +597,7 @@ public class BucketingSortingReduceSinkOptimizer extends Transform {
             }
             else {
               if (!checkTable(srcTable, newBucketPositions, newSortPositions, sortOrder,
-                  numBucketsDestination)) {
+                  sortNullOrder, numBucketsDestination)) {
                 return null;
               }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java
index c38c6d7..1e8f30e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java
@@ -18,18 +18,16 @@
 
 package org.apache.hadoop.hive.ql.optimizer;
 
+import static org.apache.hadoop.hive.ql.plan.ReduceSinkDesc.ReducerTraits.FIXED;
+
 import java.util.ArrayList;
 import java.util.EnumSet;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.Stack;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.exec.HashTableDummyOperator;
 import org.apache.hadoop.hive.ql.exec.MapJoinOperator;
 import org.apache.hadoop.hive.ql.exec.Operator;
@@ -60,11 +58,11 @@ import org.apache.hadoop.hive.ql.plan.TezEdgeProperty.EdgeType;
 import org.apache.hadoop.hive.ql.plan.TezWork;
 import org.apache.hadoop.hive.ql.plan.TezWork.VertexType;
 import org.apache.hadoop.hive.ql.stats.StatsUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.collect.Sets;
 
-import static org.apache.hadoop.hive.ql.plan.ReduceSinkDesc.ReducerTraits.FIXED;
-
 public class ReduceSinkMapJoinProc implements NodeProcessor {
 
   private final static Logger LOG = LoggerFactory.getLogger(ReduceSinkMapJoinProc.class.getName());
@@ -347,11 +345,14 @@ public class ReduceSinkMapJoinProc implements NodeProcessor {
     Map<Byte, List<ExprNodeDesc>> keyExprMap = mapJoinOp.getConf().getKeys();
     List<ExprNodeDesc> keyCols = keyExprMap.get(Byte.valueOf((byte) 0));
     StringBuilder keyOrder = new StringBuilder();
+    StringBuilder keyNullOrder = new StringBuilder();
     for (ExprNodeDesc k: keyCols) {
       keyOrder.append("+");
+      keyNullOrder.append("a");
     }
     TableDesc keyTableDesc = PlanUtils.getReduceKeyTableDesc(PlanUtils
-        .getFieldSchemasFromColumnList(keyCols, "mapjoinkey"), keyOrder.toString());
+        .getFieldSchemasFromColumnList(keyCols, "mapjoinkey"), keyOrder.toString(),
+        keyNullOrder.toString());
     mapJoinOp.getConf().setKeyTableDesc(keyTableDesc);
 
     // let the dummy op be the parent of mapjoin op

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionOptimizer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionOptimizer.java
index ad12091..3e6c7c7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionOptimizer.java
@@ -50,6 +50,7 @@ import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.lib.Rule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.DynamicPartitionCtx;
@@ -182,10 +183,9 @@ public class SortedDynPartitionOptimizer extends Transform {
       // Get the positions for partition, bucket and sort columns
       List<Integer> bucketPositions = getBucketPositions(destTable.getBucketCols(),
           destTable.getCols());
-      ObjectPair<List<Integer>, List<Integer>> sortOrderPositions = getSortPositionsOrder(
-          destTable.getSortCols(), destTable.getCols());
       List<Integer> sortPositions = null;
       List<Integer> sortOrder = null;
+      List<Integer> sortNullOrder = null;
       if (fsOp.getConf().getWriteType() == AcidUtils.Operation.UPDATE ||
           fsOp.getConf().getWriteType() == AcidUtils.Operation.DELETE) {
         // When doing updates and deletes we always want to sort on the rowid because the ACID
@@ -193,13 +193,16 @@ public class SortedDynPartitionOptimizer extends Transform {
         // ignore whatever comes from the table and enforce this sort order instead.
         sortPositions = Arrays.asList(0);
         sortOrder = Arrays.asList(1); // 1 means asc, could really use enum here in the thrift if
+        sortNullOrder = Arrays.asList(0);
       } else {
-        sortPositions = sortOrderPositions.getFirst();
-        sortOrder = sortOrderPositions.getSecond();
+        sortPositions = getSortPositions(destTable.getSortCols(), destTable.getCols());
+        sortOrder = getSortOrders(destTable.getSortCols(), destTable.getCols());
+        sortNullOrder = getSortNullOrders(destTable.getSortCols(), destTable.getCols());
       }
       LOG.debug("Got sort order");
       for (int i : sortPositions) LOG.debug("sort position " + i);
       for (int i : sortOrder) LOG.debug("sort order " + i);
+      for (int i : sortNullOrder) LOG.debug("sort null order " + i);
       List<Integer> partitionPositions = getPartitionPositions(dpCtx, fsParent.getSchema());
       List<ColumnInfo> colInfos = fsParent.getSchema().getSignature();
       ArrayList<ExprNodeDesc> bucketColumns = getPositionsToExprNodes(bucketPositions, colInfos);
@@ -214,8 +217,9 @@ public class SortedDynPartitionOptimizer extends Transform {
       for (ColumnInfo ci : parentCols) {
         allRSCols.add(new ExprNodeColumnDesc(ci));
       }
+
       // Create ReduceSink operator
-      ReduceSinkOperator rsOp = getReduceSinkOp(partitionPositions, sortPositions, sortOrder,
+      ReduceSinkOperator rsOp = getReduceSinkOp(partitionPositions, sortPositions, sortOrder, sortNullOrder,
           allRSCols, bucketColumns, numBuckets, fsParent, fsOp.getConf().getWriteType());
 
       List<ExprNodeDesc> descs = new ArrayList<ExprNodeDesc>(allRSCols.size());
@@ -408,17 +412,19 @@ public class SortedDynPartitionOptimizer extends Transform {
     }
 
     public ReduceSinkOperator getReduceSinkOp(List<Integer> partitionPositions,
-        List<Integer> sortPositions, List<Integer> sortOrder, ArrayList<ExprNodeDesc> allCols,
-        ArrayList<ExprNodeDesc> bucketColumns, int numBuckets,
+        List<Integer> sortPositions, List<Integer> sortOrder, List<Integer> sortNullOrder,
+        ArrayList<ExprNodeDesc> allCols, ArrayList<ExprNodeDesc> bucketColumns, int numBuckets,
         Operator<? extends OperatorDesc> parent, AcidUtils.Operation writeType) {
 
       // Order of KEY columns
       // 1) Partition columns
       // 2) Bucket number column
       // 3) Sort columns
+      // 4) Null sort columns
       Set<Integer> keyColsPosInVal = Sets.newLinkedHashSet();
       ArrayList<ExprNodeDesc> keyCols = Lists.newArrayList();
       List<Integer> newSortOrder = Lists.newArrayList();
+      List<Integer> newSortNullOrder = Lists.newArrayList();
       int numPartAndBuck = partitionPositions.size();
 
       keyColsPosInVal.addAll(partitionPositions);
@@ -449,8 +455,33 @@ public class SortedDynPartitionOptimizer extends Transform {
         }
       }
 
+      // if partition and bucket columns are sorted in ascending order, by default
+      // nulls come first; otherwise nulls come last
+      Integer nullOrder = order == 1 ? 0 : 1;
+      if (sortNullOrder != null && !sortNullOrder.isEmpty()) {
+        if (sortNullOrder.get(0).intValue() == 0) {
+          nullOrder = 0;
+        } else {
+          nullOrder = 1;
+        }
+      }
+      for (int i = 0; i < numPartAndBuck; i++) {
+        newSortNullOrder.add(nullOrder);
+      }
+      newSortNullOrder.addAll(sortNullOrder);
+
+      String nullOrderStr = "";
+      for (Integer i : newSortNullOrder) {
+        if(i.intValue() == 0) {
+          nullOrderStr += "a";
+        } else {
+          nullOrderStr += "z";
+        }
+      }
+
       Map<String, ExprNodeDesc> colExprMap = Maps.newHashMap();
       ArrayList<ExprNodeDesc> partCols = Lists.newArrayList();
+
       // we will clone here as RS will update bucket column key with its
       // corresponding with bucket number and hence their OIs
       for (Integer idx : keyColsPosInVal) {
@@ -480,9 +511,11 @@ public class SortedDynPartitionOptimizer extends Transform {
           ReduceSinkOperator.class);
       if (parentRSOp != null && parseCtx.getQueryProperties().hasOuterOrderBy()) {
         String parentRSOpOrder = parentRSOp.getConf().getOrder();
+        String parentRSOpNullOrder = parentRSOp.getConf().getNullOrder();
         if (parentRSOpOrder != null && !parentRSOpOrder.isEmpty() && sortPositions.isEmpty()) {
           keyCols.addAll(parentRSOp.getConf().getKeyCols());
           orderStr += parentRSOpOrder;
+          nullOrderStr += parentRSOpNullOrder;
         }
       }
 
@@ -504,7 +537,7 @@ public class SortedDynPartitionOptimizer extends Transform {
       // from Key and Value TableDesc
       List<FieldSchema> fields = PlanUtils.getFieldSchemasFromColumnList(keyCols,
           keyColNames, 0, "");
-      TableDesc keyTable = PlanUtils.getReduceKeyTableDesc(fields, orderStr);
+      TableDesc keyTable = PlanUtils.getReduceKeyTableDesc(fields, orderStr, nullOrderStr);
       List<FieldSchema> valFields = PlanUtils.getFieldSchemasFromColumnList(valCols,
           valColNames, 0, "");
       TableDesc valueTable = PlanUtils.getReduceValueTableDesc(valFields);
@@ -523,27 +556,65 @@ public class SortedDynPartitionOptimizer extends Transform {
     }
 
     /**
-     * Get the sort positions and sort order for the sort columns
+     * Get the sort positions for the sort columns
      * @param tabSortCols
      * @param tabCols
      * @return
      */
-    private ObjectPair<List<Integer>, List<Integer>> getSortPositionsOrder(List<Order> tabSortCols,
+    private List<Integer> getSortPositions(List<Order> tabSortCols,
         List<FieldSchema> tabCols) {
       List<Integer> sortPositions = Lists.newArrayList();
-      List<Integer> sortOrders = Lists.newArrayList();
       for (Order sortCol : tabSortCols) {
         int pos = 0;
         for (FieldSchema tabCol : tabCols) {
           if (sortCol.getCol().equals(tabCol.getName())) {
             sortPositions.add(pos);
-            sortOrders.add(sortCol.getOrder());
             break;
           }
           pos++;
         }
       }
-      return new ObjectPair<List<Integer>, List<Integer>>(sortPositions, sortOrders);
+      return sortPositions;
+    }
+
+    /**
+     * Get the sort order for the sort columns
+     * @param tabSortCols
+     * @param tabCols
+     * @return
+     */
+    private List<Integer> getSortOrders(List<Order> tabSortCols,
+        List<FieldSchema> tabCols) {
+      List<Integer> sortOrders = Lists.newArrayList();
+      for (Order sortCol : tabSortCols) {
+        for (FieldSchema tabCol : tabCols) {
+          if (sortCol.getCol().equals(tabCol.getName())) {
+            sortOrders.add(sortCol.getOrder());
+            break;
+          }
+        }
+      }
+      return sortOrders;
+    }
+
+    /**
+     * Get the null sort order for the sort columns
+     * @param tabSortCols
+     * @param tabCols
+     * @return
+     */
+    private List<Integer> getSortNullOrders(List<Order> tabSortCols,
+        List<FieldSchema> tabCols) {
+      List<Integer> sortNullOrders = Lists.newArrayList();
+      for (Order sortCol : tabSortCols) {
+        for (FieldSchema tabCol : tabCols) {
+          if (sortCol.getCol().equals(tabCol.getName())) {
+            sortNullOrders.add(sortCol.getNullOrder());
+            break;
+          }
+        }
+      }
+      return sortNullOrders;
     }
 
     private ArrayList<ExprNodeDesc> getPositionsToExprNodes(List<Integer> pos,

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/RelOptHiveTable.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/RelOptHiveTable.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/RelOptHiveTable.java
index 02db680..a95da0a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/RelOptHiveTable.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/RelOptHiveTable.java
@@ -33,6 +33,7 @@ import org.apache.calcite.rel.RelCollationTraitDef;
 import org.apache.calcite.rel.RelDistribution;
 import org.apache.calcite.rel.RelFieldCollation;
 import org.apache.calcite.rel.RelFieldCollation.Direction;
+import org.apache.calcite.rel.RelFieldCollation.NullDirection;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.logical.LogicalTableScan;
 import org.apache.calcite.rel.type.RelDataType;
@@ -165,7 +166,14 @@ public class RelOptHiveTable extends RelOptAbstractTable {
           else {
             direction = Direction.DESCENDING;
           }
-          collationList.add(new RelFieldCollation(i,direction));
+          NullDirection nullDirection;
+          if (sortColumn.getNullOrder() == BaseSemanticAnalyzer.HIVE_COLUMN_NULLS_FIRST) {
+            nullDirection = NullDirection.FIRST;
+          }
+          else {
+            nullDirection = NullDirection.LAST;
+          }
+          collationList.add(new RelFieldCollation(i,direction,nullDirection));
           break;
         }
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRelFieldTrimmer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRelFieldTrimmer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRelFieldTrimmer.java
index 997b82c..03002cc 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRelFieldTrimmer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRelFieldTrimmer.java
@@ -287,8 +287,8 @@ public class HiveRelFieldTrimmer extends RelFieldTrimmer {
    final List<RexNode> originalExtraNodes = ImmutableList.copyOf(extraNodes);
    for (RexNode node : nodes) {
      fieldCollations.add(
-         collation(node, RelFieldCollation.Direction.ASCENDING, null,
-             extraNodes));
+         collation(node, RelFieldCollation.Direction.ASCENDING,
+                 RelFieldCollation.NullDirection.FIRST, extraNodes));
    }
    final RexNode offsetNode = offset <= 0 ? null : relBuilder.literal(offset);
    final RexNode fetchNode = fetch < 0 ? null : relBuilder.literal(fetch);

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java
index 3f2267d..de7e2f8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java
@@ -52,8 +52,6 @@ import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.util.ImmutableBitSet;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.ql.metadata.VirtualColumn;
 import org.apache.hadoop.hive.ql.optimizer.calcite.CalciteSemanticException;
@@ -64,6 +62,8 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.translator.SqlFunctionConvert
 import org.apache.hadoop.hive.ql.parse.ASTNode;
 import org.apache.hadoop.hive.ql.parse.HiveParser;
 import org.apache.hadoop.hive.ql.parse.ParseDriver;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.collect.Iterables;
 
@@ -226,6 +226,24 @@ public class ASTConverter {
           ASTNode directionAST = c.getDirection() == RelFieldCollation.Direction.ASCENDING ? ASTBuilder
               .createAST(HiveParser.TOK_TABSORTCOLNAMEASC, "TOK_TABSORTCOLNAMEASC") : ASTBuilder
               .createAST(HiveParser.TOK_TABSORTCOLNAMEDESC, "TOK_TABSORTCOLNAMEDESC");
+          ASTNode nullDirectionAST;
+          // Null direction
+          if (c.nullDirection == RelFieldCollation.NullDirection.FIRST) {
+            nullDirectionAST = ASTBuilder.createAST(HiveParser.TOK_NULLS_FIRST, "TOK_NULLS_FIRST");
+            directionAST.addChild(nullDirectionAST);
+          } else if (c.nullDirection == RelFieldCollation.NullDirection.LAST) {
+            nullDirectionAST = ASTBuilder.createAST(HiveParser.TOK_NULLS_LAST, "TOK_NULLS_LAST");
+            directionAST.addChild(nullDirectionAST);
+          } else {
+            // Default
+            if (c.getDirection() == RelFieldCollation.Direction.ASCENDING) {
+              nullDirectionAST = ASTBuilder.createAST(HiveParser.TOK_NULLS_FIRST, "TOK_NULLS_FIRST");
+              directionAST.addChild(nullDirectionAST);
+            } else {
+              nullDirectionAST = ASTBuilder.createAST(HiveParser.TOK_NULLS_LAST, "TOK_NULLS_LAST");
+              directionAST.addChild(nullDirectionAST);
+            }
+          }
 
           // 3 Convert OB expr (OB Expr is usually an input ref except for top
           // level OB; top level OB will have RexCall kept in a map.)
@@ -245,7 +263,7 @@ public class ASTConverter {
           }
 
           // 4 buildup the ob expr AST
-          directionAST.addChild(astCol);
+          nullDirectionAST.addChild(astCol);
           orderAst.addChild(directionAST);
         }
         hiveAST.order = orderAst;
@@ -430,12 +448,31 @@ public class ASTConverter {
       if (window.orderKeys != null && !window.orderKeys.isEmpty()) {
         oByAst = ASTBuilder.createAST(HiveParser.TOK_ORDERBY, "TOK_ORDERBY");
         for (RexFieldCollation ok : window.orderKeys) {
-          ASTNode astNode = ok.getDirection() == RelFieldCollation.Direction.ASCENDING ? ASTBuilder
+          ASTNode directionAST = ok.getDirection() == RelFieldCollation.Direction.ASCENDING ? ASTBuilder
               .createAST(HiveParser.TOK_TABSORTCOLNAMEASC, "TOK_TABSORTCOLNAMEASC") : ASTBuilder
               .createAST(HiveParser.TOK_TABSORTCOLNAMEDESC, "TOK_TABSORTCOLNAMEDESC");
+          ASTNode nullDirectionAST;
+          // Null direction
+          if (ok.right.contains(SqlKind.NULLS_FIRST)) {
+            nullDirectionAST = ASTBuilder.createAST(HiveParser.TOK_NULLS_FIRST, "TOK_NULLS_FIRST");
+            directionAST.addChild(nullDirectionAST);
+          } else if (ok.right.contains(SqlKind.NULLS_LAST)) {
+            nullDirectionAST = ASTBuilder.createAST(HiveParser.TOK_NULLS_LAST, "TOK_NULLS_LAST");
+            directionAST.addChild(nullDirectionAST);
+          } else {
+            // Default
+            if (ok.getDirection() == RelFieldCollation.Direction.ASCENDING) {
+              nullDirectionAST = ASTBuilder.createAST(HiveParser.TOK_NULLS_FIRST, "TOK_NULLS_FIRST");
+              directionAST.addChild(nullDirectionAST);
+            } else {
+              nullDirectionAST = ASTBuilder.createAST(HiveParser.TOK_NULLS_LAST, "TOK_NULLS_LAST");
+              directionAST.addChild(nullDirectionAST);
+            }
+          }
           ASTNode astCol = ok.left.accept(this);
-          astNode.addChild(astCol);
-          oByAst.addChild(astNode);
+          
+          nullDirectionAST.addChild(astCol);
+          oByAst.addChild(directionAST);
         }
       }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ExprNodeConverter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ExprNodeConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ExprNodeConverter.java
index 739faa9..e51b6c4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ExprNodeConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ExprNodeConverter.java
@@ -48,6 +48,7 @@ import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
 import org.apache.hadoop.hive.ql.optimizer.calcite.translator.ASTConverter.RexVisitor;
 import org.apache.hadoop.hive.ql.optimizer.calcite.translator.ASTConverter.Schema;
 import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.NullOrder;
 import org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.Order;
 import org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.OrderExpression;
 import org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.OrderSpec;
@@ -315,7 +316,18 @@ public class ExprNodeConverter extends RexVisitorImpl<ExprNodeDesc> {
         OrderExpression exprSpec = new OrderExpression();
         Order order = ok.getDirection() == RelFieldCollation.Direction.ASCENDING ?
                 Order.ASC : Order.DESC;
+        NullOrder nullOrder;
+        if ( ok.right.contains(SqlKind.NULLS_FIRST) ) {
+          nullOrder = NullOrder.NULLS_FIRST;
+        } else if ( ok.right.contains(SqlKind.NULLS_LAST) ) {
+          nullOrder = NullOrder.NULLS_LAST;
+        } else {
+          // Default
+          nullOrder = ok.getDirection() == RelFieldCollation.Direction.ASCENDING ?
+                  NullOrder.NULLS_FIRST : NullOrder.NULLS_LAST;
+        }
         exprSpec.setOrder(order);
+        exprSpec.setNullOrder(nullOrder);
         ASTNode astNode = ok.left.accept(new RexVisitor(schema));
         exprSpec.setExpression(astNode);
         oSpec.addExpression(exprSpec);

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveOpConverter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveOpConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveOpConverter.java
index b841315..1307808 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveOpConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/HiveOpConverter.java
@@ -41,8 +41,6 @@ import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.Pair;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.StrictChecks;
 import org.apache.hadoop.hive.ql.exec.ColumnInfo;
@@ -65,8 +63,8 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveJoin;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveMultiJoin;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveProject;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSemiJoin;
-import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSortLimit;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSortExchange;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSortLimit;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveTableScan;
 import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveUnion;
 import org.apache.hadoop.hive.ql.parse.JoinCond;
@@ -99,6 +97,8 @@ import org.apache.hadoop.hive.ql.plan.SelectDesc;
 import org.apache.hadoop.hive.ql.plan.TableScanDesc;
 import org.apache.hadoop.hive.ql.plan.UnionDesc;
 import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
@@ -435,6 +435,7 @@ public class HiveOpConverter {
       Map<Integer, RexNode> obRefToCallMap = sortRel.getInputRefToCallMap();
       List<ExprNodeDesc> sortCols = new ArrayList<ExprNodeDesc>();
       StringBuilder order = new StringBuilder();
+      StringBuilder nullOrder = new StringBuilder();
       for (RelFieldCollation sortInfo : sortRel.getCollation().getFieldCollations()) {
         int sortColumnPos = sortInfo.getFieldIndex();
         ColumnInfo columnInfo = new ColumnInfo(inputOp.getSchema().getSignature()
@@ -447,6 +448,14 @@ public class HiveOpConverter {
         } else {
           order.append("+");
         }
+        if (sortInfo.nullDirection == RelFieldCollation.NullDirection.FIRST) {
+          nullOrder.append("a");
+        } else if (sortInfo.nullDirection == RelFieldCollation.NullDirection.LAST) {
+          nullOrder.append("z");
+        } else {
+          // Default
+          nullOrder.append(sortInfo.getDirection() == RelFieldCollation.Direction.DESCENDING ? "z" : "a");
+        }
 
         if (obRefToCallMap != null) {
           RexNode obExpr = obRefToCallMap.get(sortColumnPos);
@@ -474,7 +483,7 @@ public class HiveOpConverter {
       // 1.b. Generate reduce sink and project operator
       resultOp = genReduceSinkAndBacktrackSelect(resultOp,
           sortCols.toArray(new ExprNodeDesc[sortCols.size()]), 0, new ArrayList<ExprNodeDesc>(),
-          order.toString(), numReducers, Operation.NOT_ACID, hiveConf, keepColumns);
+          order.toString(), nullOrder.toString(), numReducers, Operation.NOT_ACID, hiveConf, keepColumns);
     }
 
     // 2. If we need to generate limit
@@ -626,6 +635,7 @@ public class HiveOpConverter {
       ArrayList<ExprNodeDesc> keyCols = new ArrayList<ExprNodeDesc>();
       ArrayList<ExprNodeDesc> partCols = new ArrayList<ExprNodeDesc>();
       StringBuilder order = new StringBuilder();
+      StringBuilder nullOrder = new StringBuilder();
 
       for (PartitionExpression partCol : wSpec.getQueryPartitionSpec().getExpressions()) {
         ExprNodeDesc partExpr = semanticAnalyzer.genExprNodeDesc(partCol.getExpression(), rr);
@@ -633,6 +643,7 @@ public class HiveOpConverter {
           keyCols.add(partExpr);
           partCols.add(partExpr);
           order.append('+');
+          nullOrder.append('a');
         }
       }
 
@@ -640,19 +651,22 @@ public class HiveOpConverter {
         for (OrderExpression orderCol : wSpec.getQueryOrderSpec().getExpressions()) {
           ExprNodeDesc orderExpr = semanticAnalyzer.genExprNodeDesc(orderCol.getExpression(), rr);
           char orderChar = orderCol.getOrder() == PTFInvocationSpec.Order.ASC ? '+' : '-';
+          char nullOrderChar = orderCol.getNullOrder() == PTFInvocationSpec.NullOrder.NULLS_FIRST ? 'a' : 'z';
           int index = ExprNodeDescUtils.indexOf(orderExpr, keyCols);
           if (index >= 0) {
             order.setCharAt(index, orderChar);
+            nullOrder.setCharAt(index, nullOrderChar);
             continue;
           }
           keyCols.add(orderExpr);
           order.append(orderChar);
+          nullOrder.append(nullOrderChar);
         }
       }
 
       SelectOperator selectOp = genReduceSinkAndBacktrackSelect(input,
           keyCols.toArray(new ExprNodeDesc[keyCols.size()]), 0, partCols,
-          order.toString(), -1, Operation.NOT_ACID, hiveConf);
+          order.toString(), nullOrder.toString(), -1, Operation.NOT_ACID, hiveConf);
 
       // 2. Finally create PTF
       PTFTranslator translator = new PTFTranslator();
@@ -677,14 +691,14 @@ public class HiveOpConverter {
 
   private static SelectOperator genReduceSinkAndBacktrackSelect(Operator<?> input,
           ExprNodeDesc[] keys, int tag, ArrayList<ExprNodeDesc> partitionCols, String order,
-          int numReducers, Operation acidOperation, HiveConf hiveConf)
+          String nullOrder, int numReducers, Operation acidOperation, HiveConf hiveConf)
               throws SemanticException {
-    return genReduceSinkAndBacktrackSelect(input, keys, tag, partitionCols, order,
+    return genReduceSinkAndBacktrackSelect(input, keys, tag, partitionCols, order, nullOrder,
         numReducers, acidOperation, hiveConf, input.getSchema().getColumnNames());
   }
 
   private static SelectOperator genReduceSinkAndBacktrackSelect(Operator<?> input,
-      ExprNodeDesc[] keys, int tag, ArrayList<ExprNodeDesc> partitionCols, String order,
+      ExprNodeDesc[] keys, int tag, ArrayList<ExprNodeDesc> partitionCols, String order, String nullOrder,
       int numReducers, Operation acidOperation, HiveConf hiveConf,
       List<String> keepColNames) throws SemanticException {
     // 1. Generate RS operator
@@ -715,7 +729,8 @@ public class HiveOpConverter {
           "In CBO return path, genReduceSinkAndBacktrackSelect is expecting only one tableAlias but there is none");
     }
     // 1.2 Now generate RS operator
-    ReduceSinkOperator rsOp = genReduceSink(input, tableAlias, keys, tag, partitionCols, order, numReducers, acidOperation, hiveConf);
+    ReduceSinkOperator rsOp = genReduceSink(input, tableAlias, keys, tag, partitionCols, order,
+            nullOrder, numReducers, acidOperation, hiveConf);
 
     // 2. Generate backtrack Select operator
     Map<String, ExprNodeDesc> descriptors = buildBacktrackFromReduceSink(keepColNames,
@@ -737,13 +752,13 @@ public class HiveOpConverter {
 
   private static ReduceSinkOperator genReduceSink(Operator<?> input, String tableAlias, ExprNodeDesc[] keys, int tag,
       int numReducers, Operation acidOperation, HiveConf hiveConf) throws SemanticException {
-    return genReduceSink(input, tableAlias, keys, tag, new ArrayList<ExprNodeDesc>(), "", numReducers,
+    return genReduceSink(input, tableAlias, keys, tag, new ArrayList<ExprNodeDesc>(), "", "", numReducers,
         acidOperation, hiveConf);
   }
 
   @SuppressWarnings({ "rawtypes", "unchecked" })
   private static ReduceSinkOperator genReduceSink(Operator<?> input, String tableAlias, ExprNodeDesc[] keys, int tag,
-      ArrayList<ExprNodeDesc> partitionCols, String order, int numReducers,
+      ArrayList<ExprNodeDesc> partitionCols, String order, String nullOrder, int numReducers,
       Operation acidOperation, HiveConf hiveConf) throws SemanticException {
     Operator dummy = Operator.createDummy(); // dummy for backtracking
     dummy.setParentOperators(Arrays.asList(input));
@@ -818,7 +833,7 @@ public class HiveOpConverter {
           reduceKeys.size(), numReducers, acidOperation);
     } else {
       rsDesc = PlanUtils.getReduceSinkDesc(reduceKeys, reduceValues, outputColumnNames, false, tag,
-          partitionCols, order, numReducers, acidOperation);
+          partitionCols, order, nullOrder, numReducers, acidOperation);
     }
 
     ReduceSinkOperator rsOp = (ReduceSinkOperator) OperatorFactory.getAndMakeChild(

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/ReduceSinkDeDuplication.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/ReduceSinkDeDuplication.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/ReduceSinkDeDuplication.java
index 638b91e..59c87a3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/ReduceSinkDeDuplication.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/ReduceSinkDeDuplication.java
@@ -201,7 +201,8 @@ public class ReduceSinkDeDuplication extends Transform {
           return false;
         }
 
-        Integer moveRSOrderTo = checkOrder(cRSc.getOrder(), pRSNc.getOrder());
+        Integer moveRSOrderTo = checkOrder(cRSc.getOrder(), pRSNc.getOrder(),
+                cRSc.getNullOrder(), pRSNc.getNullOrder());
         if (moveRSOrderTo == null) {
           return false;
         }
@@ -298,6 +299,7 @@ public class ReduceSinkDeDuplication extends Transform {
               "Try set " + HiveConf.ConfVars.HIVEOPTREDUCEDEDUPLICATION + "=false;");
         }
         pRS.getConf().setOrder(cRS.getConf().getOrder());
+        pRS.getConf().setNullOrder(cRS.getConf().getNullOrder());
       }
 
       if (result[3] > 0) {
@@ -313,7 +315,8 @@ public class ReduceSinkDeDuplication extends Transform {
         pRS.getConf().setNumDistributionKeys(cRS.getConf().getNumDistributionKeys());
         List<FieldSchema> fields = PlanUtils.getFieldSchemasFromColumnList(pRS.getConf()
             .getKeyCols(), "reducesinkkey");
-        TableDesc keyTable = PlanUtils.getReduceKeyTableDesc(fields, pRS.getConf().getOrder());
+        TableDesc keyTable = PlanUtils.getReduceKeyTableDesc(fields, pRS.getConf().getOrder(),
+                pRS.getConf().getNullOrder());
         ArrayList<String> outputKeyCols = Lists.newArrayList();
         for (int i = 0; i < fields.size(); i++) {
           outputKeyCols.add(fields.get(i).getName());
@@ -337,7 +340,8 @@ public class ReduceSinkDeDuplication extends Transform {
         throws SemanticException {
       ReduceSinkDesc cConf = cRS.getConf();
       ReduceSinkDesc pConf = pRS.getConf();
-      Integer moveRSOrderTo = checkOrder(cConf.getOrder(), pConf.getOrder());
+      Integer moveRSOrderTo = checkOrder(cConf.getOrder(), pConf.getOrder(),
+              cConf.getNullOrder(), pConf.getNullOrder());
       if (moveRSOrderTo == null) {
         return null;
       }
@@ -447,7 +451,10 @@ public class ReduceSinkDeDuplication extends Transform {
     }
 
     // order of overlapping keys should be exactly the same
-    protected Integer checkOrder(String corder, String porder) {
+    protected Integer checkOrder(String corder, String porder,
+            String cNullOrder, String pNullOrder) {
+      assert corder.length() == cNullOrder.length();
+      assert porder.length() == pNullOrder.length();
       if (corder == null || corder.trim().equals("")) {
         if (porder == null || porder.trim().equals("")) {
           return 0;
@@ -459,8 +466,11 @@ public class ReduceSinkDeDuplication extends Transform {
       }
       corder = corder.trim();
       porder = porder.trim();
+      cNullOrder = cNullOrder.trim();
+      pNullOrder = pNullOrder.trim();
       int target = Math.min(corder.length(), porder.length());
-      if (!corder.substring(0, target).equals(porder.substring(0, target))) {
+      if (!corder.substring(0, target).equals(porder.substring(0, target)) ||
+              !cNullOrder.substring(0, target).equals(pNullOrder.substring(0, target))) {
         return null;
       }
       return Integer.valueOf(corder.length()).compareTo(porder.length());

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/BucketingSortingCtx.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/BucketingSortingCtx.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/BucketingSortingCtx.java
index 296fecb..ea3e179 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/BucketingSortingCtx.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/BucketingSortingCtx.java
@@ -201,14 +201,16 @@ public class BucketingSortingCtx implements NodeProcessorCtx {
     private List<Integer> indexes = new ArrayList<Integer>();
     // Sort order (+|-)
     private char sortOrder;
+    private char nullSortOrder;
 
-    public SortCol(String name, int index, char sortOrder) {
-      this(sortOrder);
+    public SortCol(String name, int index, char sortOrder, char nullSortOrder) {
+      this(sortOrder, nullSortOrder);
       addAlias(name, index);
     }
 
-    public SortCol(char sortOrder) {
+    public SortCol(char sortOrder, char nullSortOrder) {
       this.sortOrder = sortOrder;
+      this.nullSortOrder = nullSortOrder;
     }
 
 
@@ -232,11 +234,16 @@ public class BucketingSortingCtx implements NodeProcessorCtx {
       return sortOrder;
     }
 
+    public char getNullSortOrder() {
+      return nullSortOrder;
+    }
+
     @Override
     // Chooses a representative alias, index, and order to use as the String, the first is used
     // because it is set in the constructor
     public String toString() {
-      return "name: " + names.get(0) + " index: " + indexes.get(0) + " order: " + sortOrder;
+      return "name: " + names.get(0) + " index: " + indexes.get(0) + " order: " + sortOrder
+              + " nullOrder: " + nullSortOrder;
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/BucketingSortingOpProcFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/BucketingSortingOpProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/BucketingSortingOpProcFactory.java
index aa41200..9159120 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/BucketingSortingOpProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/BucketingSortingOpProcFactory.java
@@ -166,6 +166,7 @@ public class BucketingSortingOpProcFactory {
         }
 
         String sortOrder = rsDesc.getOrder();
+        String nullSortOrder = rsDesc.getNullOrder();
         List<ExprNodeDesc> keyCols = rsDesc.getKeyCols();
         List<ExprNodeDesc> valCols = ExprNodeDescUtils.backtrack(joinValues, jop, parent);
 
@@ -186,7 +187,8 @@ public class BucketingSortingOpProcFactory {
               newSortCols[keyIndex].addAlias(vname, vindex);
             } else {
               newBucketCols[keyIndex] = new BucketCol(vname, vindex);
-              newSortCols[keyIndex] = new SortCol(vname, vindex, sortOrder.charAt(keyIndex));
+              newSortCols[keyIndex] = new SortCol(vname, vindex, sortOrder.charAt(keyIndex),
+                      nullSortOrder.charAt(keyIndex));
             }
           }
         }
@@ -311,7 +313,8 @@ public class BucketingSortingOpProcFactory {
         int sortIndex = indexOfColName(sortCols, columnExpr.getColumn());
         if (sortIndex != -1) {
           if (newSortCols[sortIndex] == null) {
-            newSortCols[sortIndex] = new SortCol(sortCols.get(sortIndex).getSortOrder());
+            newSortCols[sortIndex] = new SortCol(sortCols.get(sortIndex).getSortOrder(),
+                    sortCols.get(sortIndex).getNullSortOrder());
           }
           newSortCols[sortIndex].addAlias(
               colInfos.get(colInfosIndex).getInternalName(), colInfosIndex);
@@ -436,7 +439,7 @@ public class BucketingSortingOpProcFactory {
   private static List<SortCol> getNewSortCols(List<SortCol> sortCols, List<ColumnInfo> colInfos) {
     List<SortCol> newSortCols = new ArrayList<SortCol>(sortCols.size());
     for (int i = 0; i < sortCols.size(); i++) {
-      SortCol sortCol = new SortCol(sortCols.get(i).getSortOrder());
+      SortCol sortCol = new SortCol(sortCols.get(i).getSortOrder(), sortCols.get(i).getNullSortOrder());
       for (Integer index : sortCols.get(i).getIndexes()) {
         // The only time this condition should be false is in the case of dynamic partitioning
         if (index < colInfos.size()) {
@@ -537,6 +540,7 @@ public class BucketingSortingOpProcFactory {
 
   static List<SortCol> extractSortCols(ReduceSinkOperator rop, List<ExprNodeDesc> outputValues) {
     String sortOrder = rop.getConf().getOrder();
+    String nullSortOrder = rop.getConf().getNullOrder();
     List<SortCol> sortCols = new ArrayList<SortCol>();
     ArrayList<ExprNodeDesc> keyCols = rop.getConf().getKeyCols();
     for (int i = 0; i < keyCols.size(); i++) {
@@ -548,7 +552,8 @@ public class BucketingSortingOpProcFactory {
       if (index < 0) {
         break;
       }
-      sortCols.add(new SortCol(((ExprNodeColumnDesc) keyCol).getColumn(), index, sortOrder.charAt(i)));
+      sortCols.add(new SortCol(((ExprNodeColumnDesc) keyCol).getColumn(), index,
+              sortOrder.charAt(i), nullSortOrder.charAt(i)));
     }
     // If the sorted columns can't all be found in the values then the data is only sorted on
     // the columns seen up until now
@@ -649,6 +654,7 @@ public class BucketingSortingOpProcFactory {
 
       GroupByDesc groupByDesc = gop.getConf();
       String sortOrder = rop.getConf().getOrder();
+      String nullSortOrder = rop.getConf().getNullOrder();
       List<BucketCol> bucketCols = new ArrayList<BucketCol>();
       List<SortCol> sortCols = new ArrayList<SortCol>();
       assert rop.getConf().getKeyCols().size() <= rop.getSchema().getSignature().size();
@@ -659,7 +665,7 @@ public class BucketingSortingOpProcFactory {
         }
         String colName = rop.getSchema().getSignature().get(i).getInternalName();
         bucketCols.add(new BucketCol(colName, i));
-        sortCols.add(new SortCol(colName, i, sortOrder.charAt(i)));
+        sortCols.add(new SortCol(colName, i, sortOrder.charAt(i), nullSortOrder.charAt(i)));
       }
       bctx.setBucketedCols(rop, bucketCols);
       bctx.setSortedCols(rop, sortCols);

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkReduceSinkMapJoinProc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkReduceSinkMapJoinProc.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkReduceSinkMapJoinProc.java
index f48fac1..f296a53 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkReduceSinkMapJoinProc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkReduceSinkMapJoinProc.java
@@ -18,10 +18,13 @@
 
 package org.apache.hadoop.hive.ql.optimizer.spark;
 
-import com.google.common.base.Preconditions;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Stack;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.exec.GroupByOperator;
 import org.apache.hadoop.hive.ql.exec.HashTableDummyOperator;
@@ -52,13 +55,10 @@ import org.apache.hadoop.hive.ql.plan.SparkEdgeProperty;
 import org.apache.hadoop.hive.ql.plan.SparkHashTableSinkDesc;
 import org.apache.hadoop.hive.ql.plan.SparkWork;
 import org.apache.hadoop.hive.ql.plan.TableDesc;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Stack;
+import com.google.common.base.Preconditions;
 
 public class SparkReduceSinkMapJoinProc implements NodeProcessor {
 
@@ -209,11 +209,14 @@ public class SparkReduceSinkMapJoinProc implements NodeProcessor {
     Map<Byte, List<ExprNodeDesc>> keyExprMap = mapJoinOp.getConf().getKeys();
     List<ExprNodeDesc> keyCols = keyExprMap.get(Byte.valueOf((byte) 0));
     StringBuilder keyOrder = new StringBuilder();
+    StringBuilder keyNullOrder = new StringBuilder();
     for (int i = 0; i < keyCols.size(); i++) {
       keyOrder.append("+");
+      keyNullOrder.append("a");
     }
     TableDesc keyTableDesc = PlanUtils.getReduceKeyTableDesc(PlanUtils
-        .getFieldSchemasFromColumnList(keyCols, "mapjoinkey"), keyOrder.toString());
+        .getFieldSchemasFromColumnList(keyCols, "mapjoinkey"), keyOrder.toString(),
+        keyNullOrder.toString());
     mapJoinOp.getConf().setKeyTableDesc(keyTableDesc);
 
     // let the dummy op be the parent of mapjoin op

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
index 6523288..f10a40a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
@@ -109,6 +109,8 @@ public abstract class BaseSemanticAnalyzer {
 
   public static int HIVE_COLUMN_ORDER_ASC = 1;
   public static int HIVE_COLUMN_ORDER_DESC = 0;
+  public static int HIVE_COLUMN_NULLS_FIRST = 0;
+  public static int HIVE_COLUMN_NULLS_LAST = 1;
 
   /**
    * ReadEntities that are passed to the hooks.
@@ -657,11 +659,23 @@ public abstract class BaseSemanticAnalyzer {
     for (int i = 0; i < numCh; i++) {
       ASTNode child = (ASTNode) ast.getChild(i);
       if (child.getToken().getType() == HiveParser.TOK_TABSORTCOLNAMEASC) {
-        colList.add(new Order(unescapeIdentifier(child.getChild(0).getText()).toLowerCase(),
-            HIVE_COLUMN_ORDER_ASC));
+        child = (ASTNode) child.getChild(0);
+        if (child.getToken().getType() == HiveParser.TOK_NULLS_FIRST) {
+          colList.add(new Order(unescapeIdentifier(child.getChild(0).getText()).toLowerCase(),
+              HIVE_COLUMN_ORDER_ASC, HIVE_COLUMN_NULLS_FIRST));
+        } else {
+          colList.add(new Order(unescapeIdentifier(child.getChild(0).getText()).toLowerCase(),
+              HIVE_COLUMN_ORDER_ASC, HIVE_COLUMN_NULLS_LAST));
+        }
       } else {
-        colList.add(new Order(unescapeIdentifier(child.getChild(0).getText()).toLowerCase(),
-            HIVE_COLUMN_ORDER_DESC));
+        child = (ASTNode) child.getChild(0);
+        if (child.getToken().getType() == HiveParser.TOK_NULLS_LAST) {
+          colList.add(new Order(unescapeIdentifier(child.getChild(0).getText()).toLowerCase(),
+              HIVE_COLUMN_ORDER_DESC, HIVE_COLUMN_NULLS_LAST));
+        } else {
+          colList.add(new Order(unescapeIdentifier(child.getChild(0).getText()).toLowerCase(),
+              HIVE_COLUMN_ORDER_DESC, HIVE_COLUMN_NULLS_FIRST));
+        }
       }
     }
     return colList;

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
index d056c5d..c36aa9d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
@@ -2413,6 +2413,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
 
         List<Node> obASTExprLst = obAST.getChildren();
         ASTNode obASTExpr;
+        ASTNode nullObASTExpr;
         List<Pair<ASTNode, TypeInfo>> vcASTTypePairs = new ArrayList<Pair<ASTNode, TypeInfo>>();
         RowResolver inputRR = relToHiveRR.get(srcRel);
         RowResolver outputRR = new RowResolver();
@@ -2425,9 +2426,11 @@ public class CalcitePlanner extends SemanticAnalyzer {
         for (int i = 0; i < obASTExprLst.size(); i++) {
           // 2.1 Convert AST Expr to ExprNode
           obASTExpr = (ASTNode) obASTExprLst.get(i);
+          nullObASTExpr = (ASTNode) obASTExpr.getChild(0);
+          ASTNode ref = (ASTNode) nullObASTExpr.getChild(0);
           Map<ASTNode, ExprNodeDesc> astToExprNDescMap = TypeCheckProcFactory.genExprNode(
               obASTExpr, new TypeCheckCtx(inputRR));
-          ExprNodeDesc obExprNDesc = astToExprNDescMap.get(obASTExpr.getChild(0));
+          ExprNodeDesc obExprNDesc = astToExprNDescMap.get(ref);
           if (obExprNDesc == null)
             throw new SemanticException("Invalid order by expression: " + obASTExpr.toString());
 
@@ -2442,18 +2445,26 @@ public class CalcitePlanner extends SemanticAnalyzer {
           } else {
             fieldIndex = srcRelRecordSz + newVCLst.size();
             newVCLst.add(rnd);
-            vcASTTypePairs.add(new Pair<ASTNode, TypeInfo>((ASTNode) obASTExpr.getChild(0),
-                obExprNDesc.getTypeInfo()));
+            vcASTTypePairs.add(new Pair<ASTNode, TypeInfo>(ref, obExprNDesc.getTypeInfo()));
           }
 
           // 2.4 Determine the Direction of order by
-          org.apache.calcite.rel.RelFieldCollation.Direction order = RelFieldCollation.Direction.DESCENDING;
+          RelFieldCollation.Direction order = RelFieldCollation.Direction.DESCENDING;
           if (obASTExpr.getType() == HiveParser.TOK_TABSORTCOLNAMEASC) {
             order = RelFieldCollation.Direction.ASCENDING;
           }
+          RelFieldCollation.NullDirection nullOrder;
+          if (nullObASTExpr.getType() == HiveParser.TOK_NULLS_FIRST) {
+            nullOrder = RelFieldCollation.NullDirection.FIRST;
+          } else if (nullObASTExpr.getType() == HiveParser.TOK_NULLS_LAST) {
+            nullOrder = RelFieldCollation.NullDirection.LAST;
+          } else {
+            throw new SemanticException(
+                    "Unexpected null ordering option: " + nullObASTExpr.getType());
+          }
 
           // 2.5 Add to field collations
-          fieldCollations.add(new RelFieldCollation(fieldIndex, order));
+          fieldCollations.add(new RelFieldCollation(fieldIndex, order, nullOrder));
         }
 
         // 3. Add Child Project Rel if needed, Generate Output RR, input Sel Rel
@@ -2583,8 +2594,17 @@ public class CalcitePlanner extends SemanticAnalyzer {
           ExprNodeDesc exp = genExprNodeDesc(oExpr.getExpression(), inputRR, tcCtx);
           RexNode ordExp = converter.convert(exp);
           Set<SqlKind> flags = new HashSet<SqlKind>();
-          if (oExpr.getOrder() == org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.Order.DESC)
+          if (oExpr.getOrder() == org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.Order.DESC) {
             flags.add(SqlKind.DESCENDING);
+          }
+          if (oExpr.getNullOrder() == org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.NullOrder.NULLS_FIRST) {
+            flags.add(SqlKind.NULLS_FIRST);
+          } else if (oExpr.getNullOrder() == org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.NullOrder.NULLS_LAST) {
+            flags.add(SqlKind.NULLS_LAST);
+          } else {
+            throw new SemanticException(
+                    "Unexpected null ordering option: " + oExpr.getNullOrder());
+          }
           oKeys.add(new RexFieldCollation(ordExp, flags));
         }
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
index 3f92d16..dd997f0 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
@@ -55,6 +55,8 @@ KW_EXISTS : 'EXISTS';
 
 KW_ASC : 'ASC';
 KW_DESC : 'DESC';
+KW_NULLS : 'NULLS';
+KW_LAST : 'LAST';
 KW_ORDER : 'ORDER';
 KW_GROUP : 'GROUP';
 KW_BY : 'BY';

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
index 9cca100..50c53db 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
@@ -81,6 +81,8 @@ TOK_GROUPING_SETS;
 TOK_GROUPING_SETS_EXPRESSION;
 TOK_HAVING;
 TOK_ORDERBY;
+TOK_NULLS_FIRST;
+TOK_NULLS_LAST;
 TOK_CLUSTERBY;
 TOK_DISTRIBUTEBY;
 TOK_SORTBY;
@@ -402,6 +404,8 @@ import org.apache.hadoop.hive.conf.HiveConf;
 
     xlateMap.put("KW_ASC", "ASC");
     xlateMap.put("KW_DESC", "DESC");
+    xlateMap.put("KW_NULLS", "NULLS");
+    xlateMap.put("KW_LAST", "LAST");
     xlateMap.put("KW_ORDER", "ORDER");
     xlateMap.put("KW_BY", "BY");
     xlateMap.put("KW_GROUP", "GROUP");
@@ -2014,13 +2018,34 @@ skewedValueLocationElement
       skewedColumnValue
      | skewedColumnValuePair
     ;
-    
+
+orderSpecification
+@init { pushMsg("order specification", state); }
+@after { popMsg(state); }
+    : KW_ASC | KW_DESC ;
+
+nullOrdering
+@init { pushMsg("nulls ordering", state); }
+@after { popMsg(state); }
+    : KW_NULLS KW_FIRST -> ^(TOK_NULLS_FIRST)
+    | KW_NULLS KW_LAST -> ^(TOK_NULLS_LAST)
+    ;
+
 columnNameOrder
 @init { pushMsg("column name order", state); }
 @after { popMsg(state); }
-    : identifier (asc=KW_ASC | desc=KW_DESC)?
-    -> {$desc == null}? ^(TOK_TABSORTCOLNAMEASC identifier)
-    ->                  ^(TOK_TABSORTCOLNAMEDESC identifier)
+    : identifier orderSpec=orderSpecification? nullSpec=nullOrdering?
+    -> {$orderSpec.tree == null && $nullSpec.tree == null}?
+            ^(TOK_TABSORTCOLNAMEASC ^(TOK_NULLS_FIRST identifier))
+    -> {$orderSpec.tree == null}?
+            ^(TOK_TABSORTCOLNAMEASC ^($nullSpec identifier))
+    -> {$nullSpec.tree == null && $orderSpec.tree.getType()==HiveParser.KW_ASC}?
+            ^(TOK_TABSORTCOLNAMEASC ^(TOK_NULLS_FIRST identifier))
+    -> {$nullSpec.tree == null && $orderSpec.tree.getType()==HiveParser.KW_DESC}?
+            ^(TOK_TABSORTCOLNAMEDESC ^(TOK_NULLS_LAST identifier))
+    -> {$orderSpec.tree.getType()==HiveParser.KW_ASC}?
+            ^(TOK_TABSORTCOLNAMEASC ^($nullSpec identifier))
+    -> ^(TOK_TABSORTCOLNAMEDESC ^($nullSpec identifier))
     ;
 
 columnNameCommentList
@@ -2039,9 +2064,18 @@ columnNameComment
 columnRefOrder
 @init { pushMsg("column order", state); }
 @after { popMsg(state); }
-    : expression (asc=KW_ASC | desc=KW_DESC)?
-    -> {$desc == null}? ^(TOK_TABSORTCOLNAMEASC expression)
-    ->                  ^(TOK_TABSORTCOLNAMEDESC expression)
+    : expression orderSpec=orderSpecification? nullSpec=nullOrdering?
+    -> {$orderSpec.tree == null && $nullSpec.tree == null}?
+            ^(TOK_TABSORTCOLNAMEASC ^(TOK_NULLS_FIRST expression))
+    -> {$orderSpec.tree == null}?
+            ^(TOK_TABSORTCOLNAMEASC ^($nullSpec expression))
+    -> {$nullSpec.tree == null && $orderSpec.tree.getType()==HiveParser.KW_ASC}?
+            ^(TOK_TABSORTCOLNAMEASC ^(TOK_NULLS_FIRST expression))
+    -> {$nullSpec.tree == null && $orderSpec.tree.getType()==HiveParser.KW_DESC}?
+            ^(TOK_TABSORTCOLNAMEDESC ^(TOK_NULLS_LAST expression))
+    -> {$orderSpec.tree.getType()==HiveParser.KW_ASC}?
+            ^(TOK_TABSORTCOLNAMEASC ^($nullSpec expression))
+    -> ^(TOK_TABSORTCOLNAMEDESC ^($nullSpec expression))
     ;
 
 columnNameType

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g b/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
index 61bd10c..a192fa7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
@@ -653,8 +653,8 @@ nonReserved
     | KW_ENABLE | KW_ESCAPED | KW_EXCLUSIVE | KW_EXPLAIN | KW_EXPORT | KW_FIELDS | KW_FILE | KW_FILEFORMAT
     | KW_FIRST | KW_FORMAT | KW_FORMATTED | KW_FUNCTIONS | KW_HOLD_DDLTIME | KW_HOUR | KW_IDXPROPERTIES | KW_IGNORE
     | KW_INDEX | KW_INDEXES | KW_INPATH | KW_INPUTDRIVER | KW_INPUTFORMAT | KW_ITEMS | KW_JAR
-    | KW_KEYS | KW_KEY_TYPE | KW_LIMIT | KW_OFFSET | KW_LINES | KW_LOAD | KW_LOCATION | KW_LOCK | KW_LOCKS | KW_LOGICAL | KW_LONG
-    | KW_MAPJOIN | KW_MATERIALIZED | KW_METADATA | KW_MINUS | KW_MINUTE | KW_MONTH | KW_MSCK | KW_NOSCAN | KW_NO_DROP | KW_OFFLINE
+    | KW_KEYS | KW_KEY_TYPE | KW_LAST | KW_LIMIT | KW_OFFSET | KW_LINES | KW_LOAD | KW_LOCATION | KW_LOCK | KW_LOCKS | KW_LOGICAL | KW_LONG
+    | KW_MAPJOIN | KW_MATERIALIZED | KW_METADATA | KW_MINUS | KW_MINUTE | KW_MONTH | KW_MSCK | KW_NOSCAN | KW_NO_DROP | KW_NULLS | KW_OFFLINE
     | KW_OPTION | KW_OUTPUTDRIVER | KW_OUTPUTFORMAT | KW_OVERWRITE | KW_OWNER | KW_PARTITIONED | KW_PARTITIONS | KW_PLUS | KW_PRETTY
     | KW_PRINCIPALS | KW_PROTECTION | KW_PURGE | KW_READ | KW_READONLY | KW_REBUILD | KW_RECORDREADER | KW_RECORDWRITER
     | KW_RELOAD | KW_RENAME | KW_REPAIR | KW_REPLACE | KW_REPLICATION | KW_RESTRICT | KW_REWRITE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/java/org/apache/hadoop/hive/ql/parse/PTFInvocationSpec.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/PTFInvocationSpec.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/PTFInvocationSpec.java
index a8980eb..ecf3cfc 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/PTFInvocationSpec.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/PTFInvocationSpec.java
@@ -508,18 +508,27 @@ public class PTFInvocationSpec {
     DESC;
   }
 
+  public static enum NullOrder
+  {
+    NULLS_FIRST,
+    NULLS_LAST;
+  }
+
   public static class OrderExpression extends PartitionExpression
   {
     Order order;
+    NullOrder nullOrder;
 
     public OrderExpression() {
       order = Order.ASC;
+      nullOrder = NullOrder.NULLS_FIRST;
     }
 
     public OrderExpression(PartitionExpression peSpec)
     {
       super(peSpec);
       order = Order.ASC;
+      nullOrder = NullOrder.NULLS_FIRST;
     }
 
     public Order getOrder()
@@ -532,12 +541,23 @@ public class PTFInvocationSpec {
       this.order = order;
     }
 
+    public NullOrder getNullOrder()
+    {
+      return nullOrder;
+    }
+
+    public void setNullOrder(NullOrder nullOrder)
+    {
+      this.nullOrder = nullOrder;
+    }
+
     @Override
     public int hashCode()
     {
       final int prime = 31;
       int result = super.hashCode();
       result = prime * result + ((order == null) ? 0 : order.hashCode());
+      result = prime * result + ((nullOrder == null) ? 0 : nullOrder.hashCode());
       return result;
     }
 
@@ -557,13 +577,16 @@ public class PTFInvocationSpec {
       if (order != other.order) {
         return false;
       }
+      if (nullOrder != other.nullOrder) {
+        return false;
+      }
       return true;
     }
 
     @Override
     public String toString()
     {
-      return String.format("%s %s", super.toString(), order);
+      return String.format("%s %s %s", super.toString(), order, nullOrder);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/java/org/apache/hadoop/hive/ql/parse/PTFTranslator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/PTFTranslator.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/PTFTranslator.java
index 2370ec0..9921b21 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/PTFTranslator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/PTFTranslator.java
@@ -499,6 +499,7 @@ public class PTFTranslator {
       throws SemanticException {
     OrderExpressionDef oexpDef = new OrderExpressionDef();
     oexpDef.setOrder(oExpr.getOrder());
+    oexpDef.setNullOrder(oExpr.getNullOrder());
     try {
       PTFExpressionDef expDef = buildExpressionDef(inpShape, oExpr.getExpression());
       oexpDef.setExpressionTreeString(expDef.getExpressionTreeString());


[04/18] hive git commit: HIVE-12994: Implement support for NULLS FIRST/NULLS LAST (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/tez/vectorized_ptf.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/vectorized_ptf.q.out b/ql/src/test/results/clientpositive/tez/vectorized_ptf.q.out
index 431e334..3d1f22f 100644
--- a/ql/src/test/results/clientpositive/tez/vectorized_ptf.q.out
+++ b/ql/src/test/results/clientpositive/tez/vectorized_ptf.q.out
@@ -157,8 +157,9 @@ TOK_QUERY
                   p_mfgr
             TOK_ORDERBY
                TOK_TABSORTCOLNAMEASC
-                  TOK_TABLE_OR_COL
-                     p_name
+                  TOK_NULLS_FIRST
+                     TOK_TABLE_OR_COL
+                        p_name
    TOK_INSERT
       TOK_DESTINATION
          TOK_DIR
@@ -183,8 +184,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             r
          TOK_SELEXPR
             TOK_FUNCTION
@@ -196,8 +198,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             dr
          TOK_SELEXPR
             TOK_FUNCTION
@@ -211,8 +214,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
                   TOK_WINDOWRANGE
                      preceding
                         unbounded
@@ -241,6 +245,7 @@ STAGE PLANS:
                   GatherStats: false
                   Reduce Output Operator
                     key expressions: p_mfgr (type: string), p_name (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: p_mfgr (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -313,13 +318,14 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
                   Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col2 (type: string), _col1 (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: _col2 (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -342,7 +348,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -511,10 +517,11 @@ TOK_QUERY
                   p_mfgr
             TOK_SORTBY
                TOK_TABSORTCOLNAMEASC
-                  .
-                     TOK_TABLE_OR_COL
-                        j
-                     p_name
+                  TOK_NULLS_FIRST
+                     .
+                        TOK_TABLE_OR_COL
+                           j
+                        p_name
    TOK_INSERT
       TOK_DESTINATION
          TOK_DIR
@@ -547,8 +554,9 @@ TOK_QUERY
                               p_mfgr
                         TOK_ORDERBY
                            TOK_TABSORTCOLNAMEASC
-                              TOK_TABLE_OR_COL
-                                 p_name
+                              TOK_NULLS_FIRST
+                                 TOK_TABLE_OR_COL
+                                    p_name
             deltaSz
 
 
@@ -578,6 +586,7 @@ STAGE PLANS:
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: p_partkey (type: int)
+                      null sort order: a
                       sort order: +
                       Map-reduce partition columns: p_partkey (type: int)
                       Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -646,6 +655,7 @@ STAGE PLANS:
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: p_partkey (type: int)
+                      null sort order: a
                       sort order: +
                       Map-reduce partition columns: p_partkey (type: int)
                       Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -715,6 +725,7 @@ STAGE PLANS:
                 Statistics: Num rows: 28 Data size: 17646 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col2 (type: string), _col1 (type: string)
+                  null sort order: aa
                   sort order: ++
                   Map-reduce partition columns: _col2 (type: string)
                   Statistics: Num rows: 28 Data size: 17646 Basic stats: COMPLETE Column stats: NONE
@@ -737,13 +748,14 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
                   Statistics: Num rows: 28 Data size: 17646 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col2 (type: string), _col1 (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: _col2 (type: string)
                     Statistics: Num rows: 28 Data size: 17646 Basic stats: COMPLETE Column stats: NONE
@@ -766,7 +778,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -886,8 +898,9 @@ TOK_QUERY
                   p_mfgr
             TOK_ORDERBY
                TOK_TABSORTCOLNAMEASC
-                  TOK_TABLE_OR_COL
-                     p_name
+                  TOK_NULLS_FIRST
+                     TOK_TABLE_OR_COL
+                        p_name
    TOK_INSERT
       TOK_DESTINATION
          TOK_DIR
@@ -924,6 +937,7 @@ STAGE PLANS:
                   GatherStats: false
                   Reduce Output Operator
                     key expressions: p_mfgr (type: string), p_name (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: p_mfgr (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -996,7 +1010,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
@@ -1114,8 +1128,9 @@ TOK_QUERY
                   p_mfgr
             TOK_ORDERBY
                TOK_TABSORTCOLNAMEASC
-                  TOK_TABLE_OR_COL
-                     p_name
+                  TOK_NULLS_FIRST
+                     TOK_TABLE_OR_COL
+                        p_name
    TOK_INSERT
       TOK_DESTINATION
          TOK_DIR
@@ -1140,8 +1155,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             r
          TOK_SELEXPR
             TOK_FUNCTION
@@ -1153,8 +1169,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             dr
          TOK_SELEXPR
             TOK_FUNCTION
@@ -1168,8 +1185,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
                   TOK_WINDOWRANGE
                      preceding
                         unbounded
@@ -1198,6 +1216,7 @@ STAGE PLANS:
                   GatherStats: false
                   Reduce Output Operator
                     key expressions: p_mfgr (type: string), p_name (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: p_mfgr (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -1270,13 +1289,14 @@ STAGE PLANS:
                       Partition table definition
                         input alias: abc
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
                   Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col2 (type: string), _col1 (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: _col2 (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -1299,7 +1319,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -1444,8 +1464,9 @@ TOK_QUERY
                   p_mfgr
             TOK_ORDERBY
                TOK_TABSORTCOLNAMEASC
-                  TOK_TABLE_OR_COL
-                     p_name
+                  TOK_NULLS_FIRST
+                     TOK_TABLE_OR_COL
+                        p_name
    TOK_INSERT
       TOK_DESTINATION
          TOK_DIR
@@ -1470,8 +1491,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             r
          TOK_SELEXPR
             TOK_FUNCTION
@@ -1483,8 +1505,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             dr
          TOK_SELEXPR
             TOK_TABLE_OR_COL
@@ -1507,8 +1530,9 @@ TOK_QUERY
                               p_mfgr
                         TOK_ORDERBY
                            TOK_TABSORTCOLNAMEASC
-                              TOK_TABLE_OR_COL
-                                 p_name
+                              TOK_NULLS_FIRST
+                                 TOK_TABLE_OR_COL
+                                    p_name
             deltaSz
 
 
@@ -1533,6 +1557,7 @@ STAGE PLANS:
                   GatherStats: false
                   Reduce Output Operator
                     key expressions: p_mfgr (type: string), p_name (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: p_mfgr (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -1605,13 +1630,14 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
                   Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col2 (type: string), _col1 (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: _col2 (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -1634,7 +1660,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -1782,8 +1808,9 @@ TOK_QUERY
                   p_mfgr
             TOK_ORDERBY
                TOK_TABSORTCOLNAMEASC
-                  TOK_TABLE_OR_COL
-                     p_name
+                  TOK_NULLS_FIRST
+                     TOK_TABLE_OR_COL
+                        p_name
    TOK_INSERT
       TOK_DESTINATION
          TOK_DIR
@@ -1808,8 +1835,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             r
          TOK_SELEXPR
             TOK_FUNCTION
@@ -1821,8 +1849,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             dr
          TOK_SELEXPR
             TOK_TABLE_OR_COL
@@ -1845,8 +1874,9 @@ TOK_QUERY
                               p_mfgr
                         TOK_ORDERBY
                            TOK_TABSORTCOLNAMEASC
-                              TOK_TABLE_OR_COL
-                                 p_name
+                              TOK_NULLS_FIRST
+                                 TOK_TABLE_OR_COL
+                                    p_name
             deltaSz
       TOK_GROUPBY
          TOK_TABLE_OR_COL
@@ -1878,6 +1908,7 @@ STAGE PLANS:
                   GatherStats: false
                   Reduce Output Operator
                     key expressions: p_mfgr (type: string), p_name (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: p_mfgr (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -1950,7 +1981,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
@@ -1966,6 +1997,7 @@ STAGE PLANS:
                       Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: int)
+                        null sort order: aaa
                         sort order: +++
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -1988,7 +2020,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col0
                         raw input shape:
                         window functions:
@@ -2131,8 +2163,9 @@ TOK_QUERY
                      p_mfgr
                TOK_ORDERBY
                   TOK_TABSORTCOLNAMEASC
-                     TOK_TABLE_OR_COL
-                        p_name
+                     TOK_NULLS_FIRST
+                        TOK_TABLE_OR_COL
+                           p_name
          TOK_TABREF
             TOK_TABNAME
                part_orc
@@ -2178,6 +2211,7 @@ STAGE PLANS:
                   GatherStats: false
                   Reduce Output Operator
                     key expressions: p_mfgr (type: string), p_name (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: p_mfgr (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -2246,6 +2280,7 @@ STAGE PLANS:
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: p_partkey (type: int)
+                      null sort order: a
                       sort order: +
                       Map-reduce partition columns: p_partkey (type: int)
                       Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -2317,7 +2352,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: abc
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col0: int, _col1: string, _col2: string, _col3: string, _col4: string, _col5: int, _col6: string, _col7: double, _col8: string
                         partition by: _col2
                         raw input shape:
@@ -2328,6 +2363,7 @@ STAGE PLANS:
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: int)
+                      null sort order: a
                       sort order: +
                       Map-reduce partition columns: _col0 (type: int)
                       Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -2458,8 +2494,9 @@ TOK_QUERY
                      p_mfgr
                TOK_ORDERBY
                   TOK_TABSORTCOLNAMEASC
-                     TOK_TABLE_OR_COL
-                        p_name
+                     TOK_NULLS_FIRST
+                        TOK_TABLE_OR_COL
+                           p_name
          =
             .
                TOK_TABLE_OR_COL
@@ -2505,6 +2542,7 @@ STAGE PLANS:
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: p_partkey (type: int)
+                      null sort order: a
                       sort order: +
                       Map-reduce partition columns: p_partkey (type: int)
                       Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -2568,6 +2606,7 @@ STAGE PLANS:
                   GatherStats: false
                   Reduce Output Operator
                     key expressions: p_mfgr (type: string), p_name (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: p_mfgr (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -2678,7 +2717,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: abc
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col0: int, _col1: string, _col2: string, _col3: string, _col4: string, _col5: int, _col6: string, _col7: double, _col8: string
                         partition by: _col2
                         raw input shape:
@@ -2689,6 +2728,7 @@ STAGE PLANS:
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: int)
+                      null sort order: a
                       sort order: +
                       Map-reduce partition columns: _col0 (type: int)
                       Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -2779,11 +2819,13 @@ TOK_QUERY
                   p_mfgr
             TOK_ORDERBY
                TOK_TABSORTCOLNAMEASC
-                  TOK_TABLE_OR_COL
-                     p_name
+                  TOK_NULLS_FIRST
+                     TOK_TABLE_OR_COL
+                        p_name
                TOK_TABSORTCOLNAMEDESC
-                  TOK_TABLE_OR_COL
-                     p_size
+                  TOK_NULLS_LAST
+                     TOK_TABLE_OR_COL
+                        p_size
    TOK_INSERT
       TOK_DESTINATION
          TOK_DIR
@@ -2808,11 +2850,13 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
                         TOK_TABSORTCOLNAMEDESC
-                           TOK_TABLE_OR_COL
-                              p_size
+                           TOK_NULLS_LAST
+                              TOK_TABLE_OR_COL
+                                 p_size
             r
 
 
@@ -2844,7 +2888,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_1
                           name: noopwithmap
-                          order by: p_name, p_size(DESC)
+                          order by: p_name ASC NULLS FIRST, p_size DESC NULLS LAST
                           output shape: p_name: string, p_mfgr: string, p_size: int
                           partition by: p_mfgr
                           raw input shape:
@@ -2853,6 +2897,7 @@ STAGE PLANS:
                     Map-side function: true
                     Reduce Output Operator
                       key expressions: p_mfgr (type: string), p_name (type: string), p_size (type: int)
+                      null sort order: aaz
                       sort order: ++-
                       Map-reduce partition columns: p_mfgr (type: string)
                       Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -2923,7 +2968,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopwithmap
-                        order by: _col1, _col5(DESC)
+                        order by: _col1 ASC NULLS FIRST, _col5 DESC NULLS LAST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
@@ -2931,6 +2976,7 @@ STAGE PLANS:
                   Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col2 (type: string), _col1 (type: string), _col5 (type: int)
+                    null sort order: aaz
                     sort order: ++-
                     Map-reduce partition columns: _col2 (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -2952,7 +2998,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1, _col5(DESC)
+                        order by: _col1 ASC NULLS FIRST, _col5 DESC NULLS LAST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -3076,8 +3122,9 @@ TOK_QUERY
                   p_mfgr
             TOK_ORDERBY
                TOK_TABSORTCOLNAMEASC
-                  TOK_TABLE_OR_COL
-                     p_name
+                  TOK_NULLS_FIRST
+                     TOK_TABLE_OR_COL
+                        p_name
    TOK_INSERT
       TOK_DESTINATION
          TOK_DIR
@@ -3102,8 +3149,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             r
          TOK_SELEXPR
             TOK_FUNCTION
@@ -3115,8 +3163,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             dr
          TOK_SELEXPR
             TOK_FUNCTION
@@ -3130,8 +3179,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
                   TOK_WINDOWRANGE
                      preceding
                         unbounded
@@ -3167,7 +3217,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_1
                           name: noopwithmap
-                          order by: p_name
+                          order by: p_name ASC NULLS FIRST
                           output shape: p_name: string, p_mfgr: string, p_size: int, p_retailprice: double
                           partition by: p_mfgr
                           raw input shape:
@@ -3176,6 +3226,7 @@ STAGE PLANS:
                     Map-side function: true
                     Reduce Output Operator
                       key expressions: p_mfgr (type: string), p_name (type: string)
+                      null sort order: aa
                       sort order: ++
                       Map-reduce partition columns: p_mfgr (type: string)
                       Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -3247,7 +3298,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopwithmap
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -3255,6 +3306,7 @@ STAGE PLANS:
                   Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col2 (type: string), _col1 (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: _col2 (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -3277,7 +3329,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -3418,8 +3470,9 @@ TOK_QUERY
                   p_mfgr
             TOK_ORDERBY
                TOK_TABSORTCOLNAMEASC
-                  TOK_TABLE_OR_COL
-                     p_name
+                  TOK_NULLS_FIRST
+                     TOK_TABLE_OR_COL
+                        p_name
    TOK_INSERT
       TOK_DESTINATION
          TOK_DIR
@@ -3444,8 +3497,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             r
          TOK_SELEXPR
             TOK_FUNCTION
@@ -3457,8 +3511,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             dr
          TOK_SELEXPR
             TOK_FUNCTION
@@ -3472,8 +3527,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
                   TOK_WINDOWRANGE
                      preceding
                         unbounded
@@ -3502,6 +3558,7 @@ STAGE PLANS:
                   GatherStats: false
                   Reduce Output Operator
                     key expressions: p_mfgr (type: string), p_name (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: p_mfgr (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -3574,13 +3631,14 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
                   Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col2 (type: string), _col1 (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: _col2 (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -3603,7 +3661,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -3750,11 +3808,13 @@ TOK_QUERY
                         p_mfgr
                   TOK_ORDERBY
                      TOK_TABSORTCOLNAMEASC
-                        TOK_TABLE_OR_COL
-                           p_mfgr
+                        TOK_NULLS_FIRST
+                           TOK_TABLE_OR_COL
+                              p_mfgr
                      TOK_TABSORTCOLNAMEASC
-                        TOK_TABLE_OR_COL
-                           p_name
+                        TOK_NULLS_FIRST
+                           TOK_TABLE_OR_COL
+                              p_name
    TOK_INSERT
       TOK_DESTINATION
          TOK_DIR
@@ -3779,8 +3839,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             r
          TOK_SELEXPR
             TOK_FUNCTION
@@ -3792,8 +3853,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             dr
          TOK_SELEXPR
             TOK_FUNCTION
@@ -3807,8 +3869,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
                   TOK_WINDOWRANGE
                      preceding
                         unbounded
@@ -3838,6 +3901,7 @@ STAGE PLANS:
                   GatherStats: false
                   Reduce Output Operator
                     key expressions: p_mfgr (type: string), p_name (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: p_mfgr (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -3910,7 +3974,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -3924,7 +3988,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_1
                           name: noopwithmap
-                          order by: _col2, _col1
+                          order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                           output shape: _col1: string, _col2: string, _col5: int, _col7: double
                           partition by: _col2
                           raw input shape:
@@ -3932,7 +3996,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_2
                           name: noop
-                          order by: _col2, _col1
+                          order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                           output shape: _col1: string, _col2: string, _col5: int, _col7: double
                           partition by: _col2
                           raw input shape:
@@ -3940,6 +4004,7 @@ STAGE PLANS:
                     Map-side function: true
                     Reduce Output Operator
                       key expressions: _col2 (type: string), _col1 (type: string)
+                      null sort order: aa
                       sort order: ++
                       Map-reduce partition columns: _col2 (type: string)
                       Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -3962,7 +4027,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopwithmap
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -3970,13 +4035,14 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_2
                         name: noop
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
                   Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col2 (type: string), _col1 (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: _col2 (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -3999,7 +4065,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -4153,8 +4219,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             TOK_INSERT
                TOK_DESTINATION
                   TOK_DIR
@@ -4178,8 +4245,9 @@ TOK_QUERY
                                     p_mfgr
                               TOK_ORDERBY
                                  TOK_TABSORTCOLNAMEASC
-                                    TOK_TABLE_OR_COL
-                                       p_name
+                                    TOK_NULLS_FIRST
+                                       TOK_TABLE_OR_COL
+                                          p_name
                      cd
                   TOK_SELEXPR
                      TOK_TABLE_OR_COL
@@ -4202,8 +4270,9 @@ TOK_QUERY
                                  p_mfgr
                            TOK_ORDERBY
                               TOK_TABSORTCOLNAMEASC
-                                 TOK_TABLE_OR_COL
-                                    p_name
+                                 TOK_NULLS_FIRST
+                                    TOK_TABLE_OR_COL
+                                       p_name
                         TOK_WINDOWRANGE
                            preceding
                               2
@@ -4254,6 +4323,7 @@ STAGE PLANS:
                   GatherStats: false
                   Reduce Output Operator
                     key expressions: p_mfgr (type: string), p_name (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: p_mfgr (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -4326,13 +4396,14 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
                   Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col2 (type: string), _col1 (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: _col2 (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -4355,7 +4426,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -4504,8 +4575,9 @@ TOK_QUERY
                      p_mfgr
                TOK_ORDERBY
                   TOK_TABSORTCOLNAMEASC
-                     TOK_TABLE_OR_COL
-                        p_name
+                     TOK_NULLS_FIRST
+                        TOK_TABLE_OR_COL
+                           p_name
          TOK_TABREF
             TOK_TABNAME
                part_orc
@@ -4546,10 +4618,11 @@ TOK_QUERY
                            p_mfgr
                      TOK_SORTBY
                         TOK_TABSORTCOLNAMEASC
-                           .
-                              TOK_TABLE_OR_COL
-                                 abc
-                              p_name
+                           TOK_NULLS_FIRST
+                              .
+                                 TOK_TABLE_OR_COL
+                                    abc
+                                 p_name
             r
          TOK_SELEXPR
             TOK_FUNCTION
@@ -4563,10 +4636,11 @@ TOK_QUERY
                            p_mfgr
                      TOK_SORTBY
                         TOK_TABSORTCOLNAMEASC
-                           .
-                              TOK_TABLE_OR_COL
-                                 abc
-                              p_name
+                           TOK_NULLS_FIRST
+                              .
+                                 TOK_TABLE_OR_COL
+                                    abc
+                                 p_name
             dr
          TOK_SELEXPR
             TOK_FUNCTION
@@ -4584,10 +4658,11 @@ TOK_QUERY
                            p_mfgr
                      TOK_SORTBY
                         TOK_TABSORTCOLNAMEASC
-                           .
-                              TOK_TABLE_OR_COL
-                                 abc
-                              p_name
+                           TOK_NULLS_FIRST
+                              .
+                                 TOK_TABLE_OR_COL
+                                    abc
+                                 p_name
             cd
          TOK_SELEXPR
             .
@@ -4610,10 +4685,11 @@ TOK_QUERY
                            p_mfgr
                      TOK_SORTBY
                         TOK_TABSORTCOLNAMEASC
-                           .
-                              TOK_TABLE_OR_COL
-                                 abc
-                              p_name
+                           TOK_NULLS_FIRST
+                              .
+                                 TOK_TABLE_OR_COL
+                                    abc
+                                 p_name
                   TOK_WINDOWRANGE
                      preceding
                         unbounded
@@ -4650,10 +4726,11 @@ TOK_QUERY
                               p_mfgr
                         TOK_SORTBY
                            TOK_TABSORTCOLNAMEASC
-                              .
-                                 TOK_TABLE_OR_COL
-                                    abc
-                                 p_name
+                              TOK_NULLS_FIRST
+                                 .
+                                    TOK_TABLE_OR_COL
+                                       abc
+                                    p_name
             deltaSz
 
 
@@ -4679,6 +4756,7 @@ STAGE PLANS:
                   GatherStats: false
                   Reduce Output Operator
                     key expressions: p_mfgr (type: string), p_name (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: p_mfgr (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -4747,6 +4825,7 @@ STAGE PLANS:
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: p_partkey (type: int)
+                      null sort order: a
                       sort order: +
                       Map-reduce partition columns: p_partkey (type: int)
                       Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -4818,7 +4897,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: abc
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col0: int, _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -4829,6 +4908,7 @@ STAGE PLANS:
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: int)
+                      null sort order: a
                       sort order: +
                       Map-reduce partition columns: _col0 (type: int)
                       Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -4849,6 +4929,7 @@ STAGE PLANS:
                 Statistics: Num rows: 28 Data size: 17646 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col2 (type: string), _col1 (type: string)
+                  null sort order: aa
                   sort order: ++
                   Map-reduce partition columns: _col2 (type: string)
                   Statistics: Num rows: 28 Data size: 17646 Basic stats: COMPLETE Column stats: NONE
@@ -4871,7 +4952,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -5027,8 +5108,9 @@ TOK_QUERY
                   p_mfgr
             TOK_ORDERBY
                TOK_TABSORTCOLNAMEASC
-                  TOK_TABLE_OR_COL
-                     p_name
+                  TOK_NULLS_FIRST
+                     TOK_TABLE_OR_COL
+                        p_name
    TOK_INSERT
       TOK_DESTINATION
          TOK_DIR
@@ -5066,6 +5148,7 @@ STAGE PLANS:
                   GatherStats: false
                   Reduce Output Operator
                     key expressions: p_mfgr (type: string), p_name (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: p_mfgr (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -5138,7 +5221,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
@@ -5154,6 +5237,7 @@ STAGE PLANS:
                       Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: int)
+                        null sort order: aaa
                         sort order: +++
                         Map-reduce partition columns: _col0 (type: string), _col1 (type: string), _col2 (type: int)
                         Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -5287,8 +5371,9 @@ TOK_QUERY
                   p_mfgr
             TOK_ORDERBY
                TOK_TABSORTCOLNAMEASC
-                  TOK_TABLE_OR_COL
-                     p_mfgr
+                  TOK_NULLS_FIRST
+                     TOK_TABLE_OR_COL
+                        p_mfgr
    TOK_INSERT
       TOK_DESTINATION
          TOK_DIR
@@ -5321,8 +5406,9 @@ TOK_QUERY
                         p_mfgr
                   TOK_ORDERBY
                      TOK_TABSORTCOLNAMEASC
-                        TOK_TABLE_OR_COL
-                           p_brand
+                        TOK_NULLS_FIRST
+                           TOK_TABLE_OR_COL
+                              p_brand
                TOK_WINDOWRANGE
                   preceding
                      2
@@ -5360,6 +5446,7 @@ STAGE PLANS:
                       Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string), _col1 (type: string)
+                        null sort order: aa
                         sort order: ++
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -5434,13 +5521,14 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col0
+                        order by: _col0 ASC NULLS FIRST
                         output shape: _col0: string, _col1: string, _col2: double
                         partition by: _col0
                         raw input shape:
                   Statistics: Num rows: 13 Data size: 8021 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col0 (type: string), _col1 (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: _col0 (type: string)
                     Statistics: Num rows: 13 Data size: 8021 Basic stats: COMPLETE Column stats: NONE
@@ -5463,7 +5551,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col0
                         raw input shape:
                         window functions:
@@ -5636,8 +5724,9 @@ TOK_QUERY
                   p_mfgr
             TOK_ORDERBY
                TOK_TABSORTCOLNAMEASC
-                  TOK_TABLE_OR_COL
-                     p_name
+                  TOK_NULLS_FIRST
+                     TOK_TABLE_OR_COL
+                        p_name
    TOK_INSERT
       TOK_DESTINATION
          TOK_TAB
@@ -5663,8 +5752,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_SORTBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             r
          TOK_SELEXPR
             TOK_FUNCTION
@@ -5676,8 +5766,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_SORTBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             dr
          TOK_SELEXPR
             TOK_FUNCTION
@@ -5691,8 +5782,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_SORTBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
                   TOK_WINDOWRANGE
                      preceding
                         unbounded
@@ -5727,8 +5819,9 @@ TOK_QUERY
                               p_mfgr
                         TOK_SORTBY
                            TOK_TABSORTCOLNAMEASC
-                              TOK_TABLE_OR_COL
-                                 p_size
+                              TOK_NULLS_FIRST
+                                 TOK_TABLE_OR_COL
+                                    p_size
                      TOK_WINDOWVALUES
                         preceding
                            5
@@ -5745,11 +5838,13 @@ TOK_QUERY
                            p_mfgr
                      TOK_SORTBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_mfgr
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_mfgr
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             r
          TOK_SELEXPR
             TOK_FUNCTION
@@ -5761,11 +5856,13 @@ TOK_QUERY
                            p_mfgr
                      TOK_SORTBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_mfgr
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_mfgr
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             dr
          TOK_SELEXPR
             TOK_FUNCTION
@@ -5777,11 +5874,13 @@ TOK_QUERY
                            p_mfgr
                      TOK_SORTBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_mfgr
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_mfgr
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             cud
          TOK_SELEXPR
             TOK_FUNCTION
@@ -5802,11 +5901,13 @@ TOK_QUERY
                         p_mfgr
                   TOK_SORTBY
                      TOK_TABSORTCOLNAMEASC
-                        TOK_TABLE_OR_COL
-                           p_mfgr
+                        TOK_NULLS_FIRST
+                           TOK_TABLE_OR_COL
+                              p_mfgr
                      TOK_TABSORTCOLNAMEASC
-                        TOK_TABLE_OR_COL
-                           p_name
+                        TOK_NULLS_FIRST
+                           TOK_TABLE_OR_COL
+                              p_name
                TOK_WINDOWRANGE
                   preceding
                      2
@@ -5841,6 +5942,7 @@ STAGE PLANS:
                   GatherStats: false
                   Reduce Output Operator
                     key expressions: p_mfgr (type: string), p_name (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: p_mfgr (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -5913,13 +6015,14 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
                   Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col2 (type: string), _col1 (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: _col2 (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -5928,6 +6031,7 @@ STAGE PLANS:
                     auto parallelism: true
                   Reduce Output Operator
                     key expressions: _col2 (type: string), _col5 (type: int)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: _col2 (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -5950,7 +6054,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -6021,7 +6125,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col5
+                        order by: _col5 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -6038,6 +6142,7 @@ STAGE PLANS:
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col2 (type: string), _col1 (type: string)
+                      null sort order: aa
                       sort order: ++
                       Map-reduce partition columns: _col2 (type: string)
                       Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -6060,7 +6165,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col3, _col2
+                        order by: _col3 ASC NULLS FIRST, _col2 ASC NULLS FIRST
                         partition by: _col3
                         raw input shape:
                         window functions:
@@ -6355,8 +6460,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_mfgr
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_mfgr
             TOK_PARTITIONINGSPEC
                TOK_DISTRIBUTEBY
                   TOK_TABLE_OR_COL
@@ -6365,11 +6471,13 @@ TOK_QUERY
                      p_name
                TOK_ORDERBY
                   TOK_TABSORTCOLNAMEASC
-                     TOK_TABLE_OR_COL
-                        p_mfgr
+                     TOK_NULLS_FIRST
+                        TOK_TABLE_OR_COL
+                           p_mfgr
                   TOK_TABSORTCOLNAMEASC
-                     TOK_TABLE_OR_COL
-                        p_name
+                     TOK_NULLS_FIRST
+                        TOK_TABLE_OR_COL
+                           p_name
          TOK_PARTITIONINGSPEC
             TOK_DISTRIBUTEBY
                TOK_TABLE_OR_COL
@@ -6378,11 +6486,13 @@ TOK_QUERY
                   p_name
             TOK_ORDERBY
                TOK_TABSORTCOLNAMEASC
-                  TOK_TABLE_OR_COL
-                     p_mfgr
+                  TOK_NULLS_FIRST
+                     TOK_TABLE_OR_COL
+                        p_mfgr
                TOK_TABSORTCOLNAMEASC
-                  TOK_TABLE_OR_COL
-                     p_name
+                  TOK_NULLS_FIRST
+                     TOK_TABLE_OR_COL
+                        p_name
    TOK_INSERT
       TOK_DESTINATION
          TOK_DIR
@@ -6460,6 +6570,7 @@ STAGE PLANS:
                   GatherStats: false
                   Reduce Output Operator
                     key expressions: p_mfgr (type: string)
+                    null sort order: a
                     sort order: +
                     Map-reduce partition columns: p_mfgr (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -6532,14 +6643,14 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col2
+                        order by: _col2 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
                       Partition table definition
                         input alias: ptf_2
                         name: noop
-                        order by: _col2
+                        order by: _col2 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
@@ -6553,7 +6664,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_1
                           name: noopwithmap
-                          order by: _col2, _col1
+                          order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                           output shape: _col1: string, _col2: string, _col5: int
                           partition by: _col2, _col1
                           raw input shape:
@@ -6561,7 +6672,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_2
                           name: noop
-                          order by: _col2, _col1
+                          order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                           output shape: _col1: string, _col2: string, _col5: int
                           partition by: _col2, _col1
                           raw input shape:
@@ -6569,6 +6680,7 @@ STAGE PLANS:
                     Map-side function: true
                     Reduce Output Operator
                       key expressions: _col2 (type: string), _col1 (type: string)
+                      null sort order: aa
                       sort order: ++
                       Map-reduce partition columns: _col2 (type: string), _col1 (type: string)
                       Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -6591,7 +6703,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopwithmap
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
@@ -6599,13 +6711,14 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_2
                         name: noop
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
                   Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col2 (type: string), _col1 (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: _col2 (type: string), _col1 (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -6628,7 +6741,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         partition by: _col2, _col1
                         raw input shape:
                         window functions:
@@ -6807,8 +6920,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_mfgr
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_mfgr
             TOK_PARTITIONINGSPEC
                TOK_DISTRIBUTEBY
                   TOK_TABLE_OR_COL
@@ -6817,19 +6931,22 @@ TOK_QUERY
                      p_name
                TOK_ORDERBY
                   TOK_TABSORTCOLNAMEASC
-                     TOK_TABLE_OR_COL
-                        p_mfgr
+                     TOK_NULLS_FIRST
+                        TOK_TABLE_OR_COL
+                           p_mfgr
                   TOK_TABSORTCOLNAMEASC
-                     TOK_TABLE_OR_COL
-                        p_name
+                     TOK_NULLS_FIRST
+                        TOK_TABLE_OR_COL
+                           p_name
          TOK_PARTITIONINGSPEC
             TOK_DISTRIBUTEBY
                TOK_TABLE_OR_COL
                   p_mfgr
             TOK_ORDERBY
                TOK_TABSORTCOLNAMEASC
-                  TOK_TABLE_OR_COL
-                     p_mfgr
+                  TOK_NULLS_FIRST
+                     TOK_TABLE_OR_COL
+                        p_mfgr
    TOK_INSERT
       TOK_DESTINATION
          TOK_DIR
@@ -6851,8 +6968,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             r
          TOK_SELEXPR
             TOK_FUNCTION
@@ -6864,8 +6982,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             dr
          TOK_SELEXPR
             TOK_TABLE_OR_COL
@@ -6882,8 +7001,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
                   TOK_WINDOWRANGE
                      preceding
                         unbounded
@@ -6914,6 +7034,7 @@ STAGE PLANS:
                   GatherStats: false
                   Reduce Output Operator
                     key expressions: p_mfgr (type: string)
+                    null sort order: a
                     sort order: +
                     Map-reduce partition columns: p_mfgr (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -6986,20 +7107,21 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col2
+                        order by: _col2 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
                       Partition table definition
                         input alias: ptf_2
                         name: noop
-                        order by: _col2
+                        order by: _col2 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
                   Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col2 (type: string), _col1 (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: _col2 (type: string), _col1 (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -7022,13 +7144,14 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
                   Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col2 (type: string)
+                    null sort order: a
                     sort order: +
                     Map-reduce partition columns: _col2 (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -7051,13 +7174,14 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col2
+                        order by: _col2 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
                   Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col2 (type: string), _col1 (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: _col2 (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -7080,7 +7204,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -7257,19 +7381,22 @@ TOK_QUERY
                            p_name
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_mfgr
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_mfgr
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             TOK_PARTITIONINGSPEC
                TOK_DISTRIBUTEBY
                   TOK_TABLE_OR_COL
                      p_mfgr
                TOK_ORDERBY
                   TOK_TABSORTCOLNAMEASC
-                     TOK_TABLE_OR_COL
-                        p_mfgr
+                     TOK_NULLS_FIRST
+                        TOK_TABLE_OR_COL
+                           p_mfgr
    TOK_INSERT
       TOK_DESTINATION
          TOK_DIR
@@ -7291,8 +7418,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             r
          TOK_SELEXPR
             TOK_FUNCTION
@@ -7304,8 +7432,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             dr
          TOK_SELEXPR
             TOK_TABLE_OR_COL
@@ -7322,8 +7451,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             s1
 
 
@@ -7349,6 +7479,7 @@ STAGE PLANS:
                   GatherStats: false
                   Reduce Output Operator
                     key expressions: p_mfgr (type: string), p_name (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: p_mfgr (type: string), p_name (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -7421,20 +7552,21 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
                       Partition table definition
                         input alias: ptf_2
                         name: noop
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
                   Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col2 (type: string)
+                    null sort order: a
                     sort order: +
                     Map-reduce partition columns: _col2 (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -7457,20 +7589,21 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col2
+                        order by: _col2 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
                       Partition table definition
                         input alias: ptf_2
                         name: noop
-                        order by: _col2
+                        order by: _col2 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
                   Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col2 (type: string), _col1 (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: _col2 (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -7493,7 +7626,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -7670,19 +7803,22 @@ TOK_QUERY
                            p_name
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_mfgr
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_mfgr
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             TOK_PARTITIONINGSPEC
                TOK_DISTRIBUTEBY
                   TOK_TABLE_OR_COL
                      p_mfgr
                TOK_ORDERBY
                   TOK_TABSORTCOLNAMEASC
-                     TOK_TABLE_OR_COL
-                        p_mfgr
+                     TOK_NULLS_FIRST
+                        TOK_TABLE_OR_COL
+                           p_mfgr
          TOK_PARTITIONINGSPEC
             TOK_DISTRIBUTEBY
                TOK_TABLE_OR_COL
@@ -7691,11 +7827,13 @@ TOK_QUERY
                   p_name
             TOK_ORDERBY
                TOK_TABSORTCOLNAMEASC
-                  TOK_TABLE_OR_COL
-                     p_mfgr
+                  TOK_NULLS_FIRST
+                     TOK_TABLE_OR_COL
+                        p_mfgr
                TOK_TABSORTCOLNAMEASC
-                  TOK_TABLE_OR_COL
-                     p_name
+                  TOK_NULLS_FIRST
+                     TOK_TABLE_OR_COL
+                        p_name
    TOK_INSERT
       TOK_DESTINATION
          TOK_DIR
@@ -7774,6 +7912,7 @@ STAGE PLANS:
                   GatherStats: false
                   Reduce Output Operator
                     key expressions: p_mfgr (type: string), p_name (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: p_mfgr (type: string), p_name (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -7846,20 +7985,21 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
                       Partition table definition
                         input alias: ptf_2
                         name: noop
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
                   Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col2 (type: string)
+                    null sort order: a
                     sort order: +
                     Map-reduce partition columns: _col2 (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -7882,7 +8022,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col2
+                        order by: _col2 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
@@ -7896,7 +8036,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_1
                           name: noopwithmap
-                          order by: _col2, _col1
+                          order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                           output shape: _col1: string, _col2: string, _col5: int
                           partition by: _col2, _col1
                           raw input shape:
@@ -7905,6 +8045,7 @@ STAGE PLANS:
                     Map-side function: true
                     Reduce Output Operator
                       key expressions: _col2 (type: string), _col1 (type: string)
+                      null sort order: aa
                       sort order: ++
                       Map-reduce partition columns: _col2 (type: string), _col1 (type: string)
                       Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -7927,7 +8068,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopwithmap
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
@@ -7935,6 +8076,7 @@ STAGE PLANS:
                   Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col2 (type: string), _col1 (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: _col2 (type: string), _col1 (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -7957,7 +8099,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         partition by: _col2, _col1
                         raw input shape:
                         window functions:
@@ -8134,19 +8276,22 @@ TOK_QUERY
                         p_name
                   TOK_ORDERBY
                      TOK_TABSORTCOLNAMEASC
-                        TOK_TABLE_OR_COL
-                           p_mfgr
+                        TOK_NULLS_FIRST
+                           TOK_TABLE_OR_COL
+                              p_mfgr
                      TOK_TABSORTCOLNAMEASC
-                        TOK_TABLE_OR_COL
-                           p_name
+                        TOK_NULLS_FIRST
+                           TOK_TABLE_OR_COL
+                              p_name
             TOK_PARTITIONINGSPEC
                

<TRUNCATED>

[12/18] hive git commit: HIVE-12994: Implement support for NULLS FIRST/NULLS LAST (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/ptf.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/ptf.q.out b/ql/src/test/results/clientpositive/ptf.q.out
index c8d37df..cdac02a 100644
--- a/ql/src/test/results/clientpositive/ptf.q.out
+++ b/ql/src/test/results/clientpositive/ptf.q.out
@@ -56,7 +56,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noop
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int, _col7: double
                   partition by: _col2
                   raw input shape:
@@ -92,7 +92,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:
@@ -272,7 +272,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noop
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2
                   raw input shape:
@@ -308,7 +308,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:
@@ -427,7 +427,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noop
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2
                   raw input shape:
@@ -544,7 +544,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: abc
                   name: noop
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int, _col7: double
                   partition by: _col2
                   raw input shape:
@@ -580,7 +580,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:
@@ -725,7 +725,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noop
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2
                   raw input shape:
@@ -761,7 +761,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:
@@ -909,7 +909,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noop
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2
                   raw input shape:
@@ -954,7 +954,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col0
                   raw input shape:
                   window functions:
@@ -1095,7 +1095,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: abc
                   name: noop
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   output shape: _col0: int, _col1: string, _col2: string, _col3: string, _col4: string, _col5: int, _col6: string, _col7: double, _col8: string
                   partition by: _col2
                   raw input shape:
@@ -1246,7 +1246,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: abc
                   name: noop
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   output shape: _col0: int, _col1: string, _col2: string, _col3: string, _col4: string, _col5: int, _col6: string, _col7: double, _col8: string
                   partition by: _col2
                   raw input shape:
@@ -1390,7 +1390,7 @@ STAGE PLANS:
                   Partition table definition
                     input alias: ptf_1
                     name: noopwithmap
-                    order by: p_name, p_size(DESC)
+                    order by: p_name ASC NULLS FIRST, p_size DESC NULLS LAST
                     output shape: p_name: string, p_mfgr: string, p_size: int
                     partition by: p_mfgr
                     raw input shape:
@@ -1416,7 +1416,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noopwithmap
-                  order by: _col1, _col5(DESC)
+                  order by: _col1 ASC NULLS FIRST, _col5 DESC NULLS LAST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2
                   raw input shape:
@@ -1452,7 +1452,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1, _col5(DESC)
+                  order by: _col1 ASC NULLS FIRST, _col5 DESC NULLS LAST
                   partition by: _col2
                   raw input shape:
                   window functions:
@@ -1565,7 +1565,7 @@ STAGE PLANS:
                   Partition table definition
                     input alias: ptf_1
                     name: noopwithmap
-                    order by: p_name
+                    order by: p_name ASC NULLS FIRST
                     output shape: p_name: string, p_mfgr: string, p_size: int, p_retailprice: double
                     partition by: p_mfgr
                     raw input shape:
@@ -1592,7 +1592,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noopwithmap
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int, _col7: double
                   partition by: _col2
                   raw input shape:
@@ -1629,7 +1629,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:
@@ -1770,7 +1770,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noop
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int, _col7: double
                   partition by: _col2
                   raw input shape:
@@ -1806,7 +1806,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:
@@ -1950,7 +1950,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noop
-                  order by: _col2(DESC), _col1
+                  order by: _col2 DESC NULLS LAST, _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int, _col7: double
                   partition by: _col2
                   raw input shape:
@@ -1964,7 +1964,7 @@ STAGE PLANS:
                   Partition table definition
                     input alias: ptf_1
                     name: noopwithmap
-                    order by: _col2(DESC), _col1
+                    order by: _col2 DESC NULLS LAST, _col1 ASC NULLS FIRST
                     output shape: _col1: string, _col2: string, _col5: int, _col7: double
                     partition by: _col2
                     raw input shape:
@@ -1972,7 +1972,7 @@ STAGE PLANS:
                   Partition table definition
                     input alias: ptf_2
                     name: noop
-                    order by: _col2(DESC), _col1
+                    order by: _col2 DESC NULLS LAST, _col1 ASC NULLS FIRST
                     output shape: _col1: string, _col2: string, _col5: int, _col7: double
                     partition by: _col2
                     raw input shape:
@@ -2009,7 +2009,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noopwithmap
-                  order by: _col2(DESC), _col1
+                  order by: _col2 DESC NULLS LAST, _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int, _col7: double
                   partition by: _col2
                   raw input shape:
@@ -2017,7 +2017,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_2
                   name: noop
-                  order by: _col2(DESC), _col1
+                  order by: _col2 DESC NULLS LAST, _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int, _col7: double
                   partition by: _col2
                   raw input shape:
@@ -2053,7 +2053,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:
@@ -2204,7 +2204,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noop
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int, _col7: double
                   partition by: _col2
                   raw input shape:
@@ -2240,7 +2240,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:
@@ -2388,7 +2388,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: abc
                   name: noop
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   output shape: _col0: int, _col1: string, _col2: string, _col5: int, _col7: double
                   partition by: _col2
                   raw input shape:
@@ -2464,7 +2464,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:
@@ -2620,7 +2620,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noop
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2
                   raw input shape:
@@ -2789,7 +2789,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noop
-                  order by: _col0
+                  order by: _col0 ASC NULLS FIRST
                   output shape: _col0: string, _col1: string, _col2: double
                   partition by: _col0
                   raw input shape:
@@ -2825,7 +2825,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col0
                   raw input shape:
                   window functions:
@@ -3005,7 +3005,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noop
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int, _col7: double
                   partition by: _col2
                   raw input shape:
@@ -3047,7 +3047,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:
@@ -3122,7 +3122,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col5
+                  order by: _col5 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:
@@ -3168,7 +3168,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col3, _col2
+                  order by: _col3 ASC NULLS FIRST, _col2 ASC NULLS FIRST
                   partition by: _col3
                   raw input shape:
                   window functions:
@@ -3413,14 +3413,14 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noop
-                  order by: _col2
+                  order by: _col2 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2
                   raw input shape:
                 Partition table definition
                   input alias: ptf_2
                   name: noop
-                  order by: _col2
+                  order by: _col2 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2
                   raw input shape:
@@ -3434,7 +3434,7 @@ STAGE PLANS:
                   Partition table definition
                     input alias: ptf_1
                     name: noopwithmap
-                    order by: _col2, _col1
+                    order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                     output shape: _col1: string, _col2: string, _col5: int
                     partition by: _col2, _col1
                     raw input shape:
@@ -3442,7 +3442,7 @@ STAGE PLANS:
                   Partition table definition
                     input alias: ptf_2
                     name: noop
-                    order by: _col2, _col1
+                    order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                     output shape: _col1: string, _col2: string, _col5: int
                     partition by: _col2, _col1
                     raw input shape:
@@ -3479,7 +3479,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noopwithmap
-                  order by: _col2, _col1
+                  order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2, _col1
                   raw input shape:
@@ -3487,7 +3487,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_2
                   name: noop
-                  order by: _col2, _col1
+                  order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2, _col1
                   raw input shape:
@@ -3523,7 +3523,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col2, _col1
+                  order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   partition by: _col2, _col1
                   raw input shape:
                   window functions:
@@ -3698,14 +3698,14 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noop
-                  order by: _col2
+                  order by: _col2 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2
                   raw input shape:
                 Partition table definition
                   input alias: ptf_2
                   name: noop
-                  order by: _col2
+                  order by: _col2 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2
                   raw input shape:
@@ -3741,7 +3741,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noop
-                  order by: _col2, _col1
+                  order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2, _col1
                   raw input shape:
@@ -3777,7 +3777,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noop
-                  order by: _col2
+                  order by: _col2 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2
                   raw input shape:
@@ -3813,7 +3813,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:
@@ -3983,14 +3983,14 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noop
-                  order by: _col2, _col1
+                  order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2, _col1
                   raw input shape:
                 Partition table definition
                   input alias: ptf_2
                   name: noop
-                  order by: _col2, _col1
+                  order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2, _col1
                   raw input shape:
@@ -4026,14 +4026,14 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noop
-                  order by: _col2
+                  order by: _col2 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2
                   raw input shape:
                 Partition table definition
                   input alias: ptf_2
                   name: noop
-                  order by: _col2
+                  order by: _col2 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2
                   raw input shape:
@@ -4069,7 +4069,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:
@@ -4240,14 +4240,14 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noop
-                  order by: _col2, _col1
+                  order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2, _col1
                   raw input shape:
                 Partition table definition
                   input alias: ptf_2
                   name: noop
-                  order by: _col2, _col1
+                  order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2, _col1
                   raw input shape:
@@ -4283,7 +4283,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noop
-                  order by: _col2
+                  order by: _col2 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2
                   raw input shape:
@@ -4297,7 +4297,7 @@ STAGE PLANS:
                   Partition table definition
                     input alias: ptf_1
                     name: noopwithmap
-                    order by: _col2, _col1
+                    order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                     output shape: _col1: string, _col2: string, _col5: int
                     partition by: _col2, _col1
                     raw input shape:
@@ -4335,7 +4335,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noopwithmap
-                  order by: _col2, _col1
+                  order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2, _col1
                   raw input shape:
@@ -4372,7 +4372,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col2, _col1
+                  order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   partition by: _col2, _col1
                   raw input shape:
                   window functions:
@@ -4544,7 +4544,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noop
-                  order by: _col2, _col1
+                  order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2, _col1
                   raw input shape:
@@ -4558,7 +4558,7 @@ STAGE PLANS:
                   Partition table definition
                     input alias: ptf_1
                     name: noopwithmap
-                    order by: _col2
+                    order by: _col2 ASC NULLS FIRST
                     output shape: _col1: string, _col2: string, _col5: int
                     partition by: _col2
                     raw input shape:
@@ -4566,7 +4566,7 @@ STAGE PLANS:
                   Partition table definition
                     input alias: ptf_2
                     name: noop
-                    order by: _col2
+                    order by: _col2 ASC NULLS FIRST
                     output shape: _col1: string, _col2: string, _col5: int
                     partition by: _col2
                     raw input shape:
@@ -4603,7 +4603,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noopwithmap
-                  order by: _col2
+                  order by: _col2 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2
                   raw input shape:
@@ -4611,7 +4611,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_2
                   name: noop
-                  order by: _col2
+                  order by: _col2 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2
                   raw input shape:
@@ -4647,7 +4647,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col2, _col1
+                  order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   partition by: _col2, _col1
                   raw input shape:
                   window functions:
@@ -4813,14 +4813,14 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noop
-                  order by: _col2, _col1
+                  order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2, _col1
                   raw input shape:
                 Partition table definition
                   input alias: ptf_2
                   name: noop
-                  order by: _col2, _col1
+                  order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2, _col1
                   raw input shape:
@@ -4834,7 +4834,7 @@ STAGE PLANS:
                   Partition table definition
                     input alias: ptf_1
                     name: noopwithmap
-                    order by: _col2, _col1
+                    order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                     output shape: _col1: string, _col2: string, _col5: int
                     partition by: _col2, _col1
                     raw input shape:
@@ -4872,7 +4872,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noopwithmap
-                  order by: _col2, _col1
+                  order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2, _col1
                   raw input shape:
@@ -4909,7 +4909,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/ptf_matchpath.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/ptf_matchpath.q.out b/ql/src/test/results/clientpositive/ptf_matchpath.q.out
index 042982f..beb4e28 100644
--- a/ql/src/test/results/clientpositive/ptf_matchpath.q.out
+++ b/ql/src/test/results/clientpositive/ptf_matchpath.q.out
@@ -94,7 +94,7 @@ STAGE PLANS:
                   input alias: ptf_1
                   arguments: 'LATE.LATE+', 'LATE', (_col5 > 15.0), 'origin_city_name, fl_num, year, month, day_of_month, size(tpath) as sz, tpath[0].day_of_month as tpath'
                   name: matchpath
-                  order by: _col2, _col3, _col4
+                  order by: _col2 ASC NULLS FIRST, _col3 ASC NULLS FIRST, _col4 ASC NULLS FIRST
                   output shape: tpath: int
                   partition by: _col6
                   raw input shape:
@@ -214,7 +214,7 @@ STAGE PLANS:
                   input alias: ptf_1
                   arguments: 'LATE.LATE+', 'LATE', (_col5 > 15.0), 'origin_city_name, fl_num, year, month, day_of_month, size(tpath) as sz, tpath[0].day_of_month as tpath'
                   name: matchpath
-                  order by: _col6, _col2, _col3, _col4
+                  order by: _col6 ASC NULLS FIRST, _col2 ASC NULLS FIRST, _col3 ASC NULLS FIRST, _col4 ASC NULLS FIRST
                   output shape: tpath: int
                   partition by: 0
                   raw input shape:
@@ -331,7 +331,7 @@ STAGE PLANS:
                   input alias: ptf_1
                   arguments: 'LATE.LATE+', 'LATE', (_col5 > 15.0), 'origin_city_name, fl_num, year, month, day_of_month, size(tpath) as sz, tpath[0].day_of_month as tpath'
                   name: matchpath
-                  order by: _col6, _col2, _col3, _col4
+                  order by: _col6 ASC NULLS FIRST, _col2 ASC NULLS FIRST, _col3 ASC NULLS FIRST, _col4 ASC NULLS FIRST
                   output shape: tpath: int
                   partition by: 0
                   raw input shape:

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/ptf_streaming.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/ptf_streaming.q.out b/ql/src/test/results/clientpositive/ptf_streaming.q.out
index 35c6d04..d1b49db 100644
--- a/ql/src/test/results/clientpositive/ptf_streaming.q.out
+++ b/ql/src/test/results/clientpositive/ptf_streaming.q.out
@@ -56,7 +56,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noopstreaming
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int, _col7: double
                   partition by: _col2
                   raw input shape:
@@ -92,7 +92,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:
@@ -272,7 +272,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noopstreaming
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2
                   raw input shape:
@@ -308,7 +308,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:
@@ -430,7 +430,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: abc
                   name: noopstreaming
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   output shape: _col0: int, _col1: string, _col2: string, _col3: string, _col4: string, _col5: int, _col6: string, _col7: double, _col8: string
                   partition by: _col2
                   raw input shape:
@@ -570,7 +570,7 @@ STAGE PLANS:
                   Partition table definition
                     input alias: ptf_1
                     name: noopwithmapstreaming
-                    order by: p_name, p_size(DESC)
+                    order by: p_name ASC NULLS FIRST, p_size DESC NULLS LAST
                     output shape: p_name: string, p_mfgr: string, p_size: int
                     partition by: p_mfgr
                     raw input shape:
@@ -596,7 +596,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noopwithmapstreaming
-                  order by: _col1, _col5(DESC)
+                  order by: _col1 ASC NULLS FIRST, _col5 DESC NULLS LAST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2
                   raw input shape:
@@ -632,7 +632,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1, _col5(DESC)
+                  order by: _col1 ASC NULLS FIRST, _col5 DESC NULLS LAST
                   partition by: _col2
                   raw input shape:
                   window functions:
@@ -745,7 +745,7 @@ STAGE PLANS:
                   Partition table definition
                     input alias: ptf_1
                     name: noopwithmapstreaming
-                    order by: p_name
+                    order by: p_name ASC NULLS FIRST
                     output shape: p_name: string, p_mfgr: string, p_size: int, p_retailprice: double
                     partition by: p_mfgr
                     raw input shape:
@@ -772,7 +772,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noopwithmapstreaming
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int, _col7: double
                   partition by: _col2
                   raw input shape:
@@ -809,7 +809,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:
@@ -953,7 +953,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noopstreaming
-                  order by: _col2, _col1
+                  order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int, _col7: double
                   partition by: _col2
                   raw input shape:
@@ -967,7 +967,7 @@ STAGE PLANS:
                   Partition table definition
                     input alias: ptf_1
                     name: noopwithmapstreaming
-                    order by: _col2, _col1
+                    order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                     output shape: _col1: string, _col2: string, _col5: int, _col7: double
                     partition by: _col2
                     raw input shape:
@@ -975,7 +975,7 @@ STAGE PLANS:
                   Partition table definition
                     input alias: ptf_2
                     name: noopstreaming
-                    order by: _col2, _col1
+                    order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                     output shape: _col1: string, _col2: string, _col5: int, _col7: double
                     partition by: _col2
                     raw input shape:
@@ -1012,7 +1012,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noopwithmapstreaming
-                  order by: _col2, _col1
+                  order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int, _col7: double
                   partition by: _col2
                   raw input shape:
@@ -1020,7 +1020,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_2
                   name: noopstreaming
-                  order by: _col2, _col1
+                  order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int, _col7: double
                   partition by: _col2
                   raw input shape:
@@ -1056,7 +1056,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:
@@ -1202,7 +1202,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noopstreaming
-                  order by: _col2, _col1
+                  order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int, _col7: double
                   partition by: _col2
                   raw input shape:
@@ -1216,7 +1216,7 @@ STAGE PLANS:
                   Partition table definition
                     input alias: ptf_1
                     name: noopwithmap
-                    order by: _col2, _col1
+                    order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                     output shape: _col1: string, _col2: string, _col5: int, _col7: double
                     partition by: _col2
                     raw input shape:
@@ -1224,7 +1224,7 @@ STAGE PLANS:
                   Partition table definition
                     input alias: ptf_2
                     name: noopstreaming
-                    order by: _col2, _col1
+                    order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                     output shape: _col1: string, _col2: string, _col5: int, _col7: double
                     partition by: _col2
                     raw input shape:
@@ -1261,7 +1261,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noopwithmap
-                  order by: _col2, _col1
+                  order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int, _col7: double
                   partition by: _col2
                   raw input shape:
@@ -1269,7 +1269,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_2
                   name: noopstreaming
-                  order by: _col2, _col1
+                  order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int, _col7: double
                   partition by: _col2
                   raw input shape:
@@ -1305,7 +1305,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:
@@ -1451,7 +1451,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noopstreaming
-                  order by: _col2, _col1
+                  order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int, _col7: double
                   partition by: _col2
                   raw input shape:
@@ -1465,7 +1465,7 @@ STAGE PLANS:
                   Partition table definition
                     input alias: ptf_1
                     name: noopwithmapstreaming
-                    order by: _col2, _col1
+                    order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                     output shape: _col1: string, _col2: string, _col5: int, _col7: double
                     partition by: _col2
                     raw input shape:
@@ -1473,7 +1473,7 @@ STAGE PLANS:
                   Partition table definition
                     input alias: ptf_2
                     name: noop
-                    order by: _col2, _col1
+                    order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                     output shape: _col1: string, _col2: string, _col5: int, _col7: double
                     partition by: _col2
                     raw input shape:
@@ -1510,7 +1510,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noopwithmapstreaming
-                  order by: _col2, _col1
+                  order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int, _col7: double
                   partition by: _col2
                   raw input shape:
@@ -1518,7 +1518,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_2
                   name: noop
-                  order by: _col2, _col1
+                  order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int, _col7: double
                   partition by: _col2
                   raw input shape:
@@ -1554,7 +1554,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:
@@ -1704,7 +1704,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: abc
                   name: noopstreaming
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   output shape: _col0: int, _col1: string, _col2: string, _col5: int, _col7: double
                   partition by: _col2
                   raw input shape:
@@ -1780,7 +1780,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:
@@ -1959,14 +1959,14 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noopstreaming
-                  order by: _col2
+                  order by: _col2 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2
                   raw input shape:
                 Partition table definition
                   input alias: ptf_2
                   name: noop
-                  order by: _col2
+                  order by: _col2 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2
                   raw input shape:
@@ -1980,7 +1980,7 @@ STAGE PLANS:
                   Partition table definition
                     input alias: ptf_1
                     name: noopwithmap
-                    order by: _col2, _col1
+                    order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                     output shape: _col1: string, _col2: string, _col5: int
                     partition by: _col2, _col1
                     raw input shape:
@@ -1988,7 +1988,7 @@ STAGE PLANS:
                   Partition table definition
                     input alias: ptf_2
                     name: noopstreaming
-                    order by: _col2, _col1
+                    order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                     output shape: _col1: string, _col2: string, _col5: int
                     partition by: _col2, _col1
                     raw input shape:
@@ -2025,7 +2025,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noopwithmap
-                  order by: _col2, _col1
+                  order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2, _col1
                   raw input shape:
@@ -2033,7 +2033,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_2
                   name: noopstreaming
-                  order by: _col2, _col1
+                  order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2, _col1
                   raw input shape:
@@ -2069,7 +2069,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col2, _col1
+                  order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   partition by: _col2, _col1
                   raw input shape:
                   window functions:
@@ -2244,14 +2244,14 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noopstreaming
-                  order by: _col2
+                  order by: _col2 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2
                   raw input shape:
                 Partition table definition
                   input alias: ptf_2
                   name: noop
-                  order by: _col2
+                  order by: _col2 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2
                   raw input shape:
@@ -2287,7 +2287,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noopstreaming
-                  order by: _col2, _col1
+                  order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2, _col1
                   raw input shape:
@@ -2323,7 +2323,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noop
-                  order by: _col2
+                  order by: _col2 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2
                   raw input shape:
@@ -2359,7 +2359,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:
@@ -2527,14 +2527,14 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noopstreaming
-                  order by: _col2, _col1
+                  order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2, _col1
                   raw input shape:
                 Partition table definition
                   input alias: ptf_2
                   name: noop
-                  order by: _col2, _col1
+                  order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2, _col1
                   raw input shape:
@@ -2548,7 +2548,7 @@ STAGE PLANS:
                   Partition table definition
                     input alias: ptf_1
                     name: noopwithmapstreaming
-                    order by: _col2, _col1
+                    order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                     output shape: _col1: string, _col2: string, _col5: int
                     partition by: _col2, _col1
                     raw input shape:
@@ -2586,7 +2586,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noopwithmapstreaming
-                  order by: _col2, _col1
+                  order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2, _col1
                   raw input shape:
@@ -2623,7 +2623,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/ptfgroupbyjoin.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/ptfgroupbyjoin.q.out b/ql/src/test/results/clientpositive/ptfgroupbyjoin.q.out
index cddee56..7efc492 100644
--- a/ql/src/test/results/clientpositive/ptfgroupbyjoin.q.out
+++ b/ql/src/test/results/clientpositive/ptfgroupbyjoin.q.out
@@ -237,7 +237,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col0, _col1
+                  order by: _col0 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   partition by: _col0, _col1
                   raw input shape:
                   window functions:

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/push_or.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/push_or.q.out b/ql/src/test/results/clientpositive/push_or.q.out
index 83680e1..bc5c930 100644
--- a/ql/src/test/results/clientpositive/push_or.q.out
+++ b/ql/src/test/results/clientpositive/push_or.q.out
@@ -67,11 +67,13 @@ TOK_QUERY
                5
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               ds
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  ds
 
 
 STAGE DEPENDENCIES:
@@ -96,6 +98,7 @@ STAGE PLANS:
                 Statistics: Num rows: 40 Data size: 320 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int), _col2 (type: string)
+                  null sort order: aa
                   sort order: ++
                   Statistics: Num rows: 40 Data size: 320 Basic stats: COMPLETE Column stats: NONE
                   tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/quotedid_basic.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/quotedid_basic.q.out b/ql/src/test/results/clientpositive/quotedid_basic.q.out
index 052e4d0..9ca3f6a 100644
--- a/ql/src/test/results/clientpositive/quotedid_basic.q.out
+++ b/ql/src/test/results/clientpositive/quotedid_basic.q.out
@@ -214,7 +214,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:
@@ -322,7 +322,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/reduce_deduplicate.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/reduce_deduplicate.q.out b/ql/src/test/results/clientpositive/reduce_deduplicate.q.out
index e82f31a..96b51d7 100644
--- a/ql/src/test/results/clientpositive/reduce_deduplicate.q.out
+++ b/ql/src/test/results/clientpositive/reduce_deduplicate.q.out
@@ -53,6 +53,7 @@ STAGE PLANS:
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
                 key expressions: _col0 (type: string)
+                null sort order: a
                 sort order: +
                 Map-reduce partition columns: _col0 (type: string)
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -410,6 +411,7 @@ STAGE PLANS:
                   Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col1 (type: string)
+                    null sort order: a
                     sort order: +
                     Map-reduce partition columns: _col1 (type: string)
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/regexp_extract.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/regexp_extract.q.out b/ql/src/test/results/clientpositive/regexp_extract.q.out
index d2a0099..6622278 100644
--- a/ql/src/test/results/clientpositive/regexp_extract.q.out
+++ b/ql/src/test/results/clientpositive/regexp_extract.q.out
@@ -121,6 +121,7 @@ STAGE PLANS:
                   Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col0 (type: string)
+                    null sort order: a
                     sort order: +
                     Map-reduce partition columns: _col0 (type: string)
                     Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
@@ -437,6 +438,7 @@ STAGE PLANS:
                   Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col0 (type: string)
+                    null sort order: a
                     sort order: +
                     Map-reduce partition columns: _col0 (type: string)
                     Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/router_join_ppr.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/router_join_ppr.q.out b/ql/src/test/results/clientpositive/router_join_ppr.q.out
index c9432c7..8a10129 100644
--- a/ql/src/test/results/clientpositive/router_join_ppr.q.out
+++ b/ql/src/test/results/clientpositive/router_join_ppr.q.out
@@ -126,6 +126,7 @@ STAGE PLANS:
                 Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
@@ -146,6 +147,7 @@ STAGE PLANS:
                 Statistics: Num rows: 222 Data size: 2358 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 222 Data size: 2358 Basic stats: COMPLETE Column stats: NONE
@@ -603,6 +605,7 @@ STAGE PLANS:
                 Statistics: Num rows: 111 Data size: 1179 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 111 Data size: 1179 Basic stats: COMPLETE Column stats: NONE
@@ -623,6 +626,7 @@ STAGE PLANS:
                 Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
@@ -981,6 +985,7 @@ STAGE PLANS:
                 Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
@@ -1001,6 +1006,7 @@ STAGE PLANS:
                 Statistics: Num rows: 111 Data size: 1179 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 111 Data size: 1179 Basic stats: COMPLETE Column stats: NONE
@@ -1355,6 +1361,7 @@ STAGE PLANS:
                 Statistics: Num rows: 12 Data size: 127 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 12 Data size: 127 Basic stats: COMPLETE Column stats: NONE
@@ -1375,6 +1382,7 @@ STAGE PLANS:
                 Statistics: Num rows: 6 Data size: 63 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 6 Data size: 63 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/sample10.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/sample10.q.out b/ql/src/test/results/clientpositive/sample10.q.out
index 8ccb577..bf0cb59 100644
--- a/ql/src/test/results/clientpositive/sample10.q.out
+++ b/ql/src/test/results/clientpositive/sample10.q.out
@@ -77,8 +77,9 @@ TOK_QUERY
             ds
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               ds
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  ds
 
 
 STAGE DEPENDENCIES:
@@ -111,6 +112,7 @@ STAGE PLANS:
                   Statistics: Num rows: 20 Data size: 120 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col0 (type: string)
+                    null sort order: a
                     sort order: +
                     Map-reduce partition columns: _col0 (type: string)
                     Statistics: Num rows: 20 Data size: 120 Basic stats: COMPLETE Column stats: NONE
@@ -350,6 +352,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col0 (type: string)
+              null sort order: a
               sort order: +
               Statistics: Num rows: 10 Data size: 60 Basic stats: COMPLETE Column stats: NONE
               tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/sample6.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/sample6.q.out b/ql/src/test/results/clientpositive/sample6.q.out
index 38e21de..c2be9d6 100644
--- a/ql/src/test/results/clientpositive/sample6.q.out
+++ b/ql/src/test/results/clientpositive/sample6.q.out
@@ -624,11 +624,13 @@ TOK_QUERY
                   s
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               value
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  value
 
 
 STAGE DEPENDENCIES:
@@ -654,6 +656,7 @@ STAGE PLANS:
                 Statistics: Num rows: 500 Data size: 5301 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int), _col1 (type: string)
+                  null sort order: aa
                   sort order: ++
                   Statistics: Num rows: 500 Data size: 5301 Basic stats: COMPLETE Column stats: NONE
                   tag: -1
@@ -1023,11 +1026,13 @@ TOK_QUERY
                   s
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               value
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  value
 
 
 STAGE DEPENDENCIES:
@@ -1053,6 +1058,7 @@ STAGE PLANS:
                 Statistics: Num rows: 500 Data size: 5301 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int), _col1 (type: string)
+                  null sort order: aa
                   sort order: ++
                   Statistics: Num rows: 500 Data size: 5301 Basic stats: COMPLETE Column stats: NONE
                   tag: -1
@@ -1676,11 +1682,13 @@ TOK_QUERY
                   s
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               value
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  value
 
 
 STAGE DEPENDENCIES:
@@ -1705,6 +1713,7 @@ STAGE PLANS:
                 Statistics: Num rows: 500 Data size: 5301 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int), _col1 (type: string)
+                  null sort order: aa
                   sort order: ++
                   Statistics: Num rows: 500 Data size: 5301 Basic stats: COMPLETE Column stats: NONE
                   tag: -1
@@ -2171,11 +2180,13 @@ TOK_QUERY
                   s
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               value
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  value
 
 
 STAGE DEPENDENCIES:
@@ -2200,6 +2211,7 @@ STAGE PLANS:
                 Statistics: Num rows: 500 Data size: 5301 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int), _col1 (type: string)
+                  null sort order: aa
                   sort order: ++
                   Statistics: Num rows: 500 Data size: 5301 Basic stats: COMPLETE Column stats: NONE
                   tag: -1
@@ -2652,11 +2664,13 @@ TOK_QUERY
                   s
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               value
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  value
 
 
 STAGE DEPENDENCIES:
@@ -2682,6 +2696,7 @@ STAGE PLANS:
                 Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int), _col1 (type: string)
+                  null sort order: aa
                   sort order: ++
                   Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
                   tag: -1
@@ -2981,11 +2996,13 @@ TOK_QUERY
                   s
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               value
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  value
 
 
 STAGE DEPENDENCIES:
@@ -3011,6 +3028,7 @@ STAGE PLANS:
                 Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int), _col1 (type: string)
+                  null sort order: aa
                   sort order: ++
                   Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
                   tag: -1
@@ -3187,11 +3205,13 @@ TOK_QUERY
                   s
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               value
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  value
 
 
 STAGE DEPENDENCIES:
@@ -3217,6 +3237,7 @@ STAGE PLANS:
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int), _col1 (type: string)
+                  null sort order: aa
                   sort order: ++
                   Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                   tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/sample8.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/sample8.q.out b/ql/src/test/results/clientpositive/sample8.q.out
index 63eeb35..a5ae1ef 100644
--- a/ql/src/test/results/clientpositive/sample8.q.out
+++ b/ql/src/test/results/clientpositive/sample8.q.out
@@ -102,6 +102,7 @@ STAGE PLANS:
               Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
                 key expressions: key (type: string), value (type: string)
+                null sort order: aa
                 sort order: ++
                 Map-reduce partition columns: key (type: string), value (type: string)
                 Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
@@ -117,6 +118,7 @@ STAGE PLANS:
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
                 key expressions: key (type: string), value (type: string)
+                null sort order: aa
                 sort order: ++
                 Map-reduce partition columns: key (type: string), value (type: string)
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/semijoin2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/semijoin2.q.out b/ql/src/test/results/clientpositive/semijoin2.q.out
index 3fe0a89..62e1961 100644
--- a/ql/src/test/results/clientpositive/semijoin2.q.out
+++ b/ql/src/test/results/clientpositive/semijoin2.q.out
@@ -154,7 +154,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: (_col98 + _col16), floor(_col21)(DESC)
+                  order by: (_col98 + _col16) ASC NULLS FIRST, floor(_col21) DESC NULLS LAST
                   partition by: (_col98 + _col16)
                   raw input shape:
                   window functions:
@@ -201,7 +201,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: (_col99 + _col17)(DESC), floor(_col22)(DESC)
+                  order by: (_col99 + _col17) DESC NULLS LAST, floor(_col22) DESC NULLS LAST
                   partition by: (_col99 + _col17)
                   raw input shape:
                   window functions:

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/semijoin4.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/semijoin4.q.out b/ql/src/test/results/clientpositive/semijoin4.q.out
index 4dca08b..2aaf7ea 100644
--- a/ql/src/test/results/clientpositive/semijoin4.q.out
+++ b/ql/src/test/results/clientpositive/semijoin4.q.out
@@ -172,7 +172,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: (UDFToShort(_col2) + _col0), floor(_col1)(DESC)
+                  order by: (UDFToShort(_col2) + _col0) ASC NULLS FIRST, floor(_col1) DESC NULLS LAST
                   partition by: (UDFToShort(_col2) + _col0)
                   raw input shape:
                   window functions:

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/show_create_table_alter.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/show_create_table_alter.q.out b/ql/src/test/results/clientpositive/show_create_table_alter.q.out
index 32819ea..78a34de 100644
--- a/ql/src/test/results/clientpositive/show_create_table_alter.q.out
+++ b/ql/src/test/results/clientpositive/show_create_table_alter.q.out
@@ -24,7 +24,7 @@ CREATE EXTERNAL TABLE `tmp_showcrt1`(
 CLUSTERED BY ( 
   key) 
 SORTED BY ( 
-  value DESC) 
+  value DESC NULLS LAST) 
 INTO 5 BUCKETS
 ROW FORMAT SERDE 
   'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' 
@@ -59,7 +59,7 @@ COMMENT 'temporary table'
 CLUSTERED BY ( 
   key) 
 SORTED BY ( 
-  value DESC) 
+  value DESC NULLS LAST) 
 INTO 5 BUCKETS
 ROW FORMAT SERDE 
   'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' 
@@ -98,7 +98,7 @@ COMMENT 'changed comment'
 CLUSTERED BY ( 
   key) 
 SORTED BY ( 
-  value DESC) 
+  value DESC NULLS LAST) 
 INTO 5 BUCKETS
 ROW FORMAT SERDE 
   'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' 
@@ -136,7 +136,7 @@ COMMENT 'changed comment'
 CLUSTERED BY ( 
   key) 
 SORTED BY ( 
-  value DESC) 
+  value DESC NULLS LAST) 
 INTO 5 BUCKETS
 ROW FORMAT SERDE 
   'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' 
@@ -174,7 +174,7 @@ COMMENT 'changed comment'
 CLUSTERED BY ( 
   key) 
 SORTED BY ( 
-  value DESC) 
+  value DESC NULLS LAST) 
 INTO 5 BUCKETS
 ROW FORMAT SERDE 
   'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' 

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/smb_mapjoin_11.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/smb_mapjoin_11.q.out b/ql/src/test/results/clientpositive/smb_mapjoin_11.q.out
index a05dd6f..b407402 100644
--- a/ql/src/test/results/clientpositive/smb_mapjoin_11.q.out
+++ b/ql/src/test/results/clientpositive/smb_mapjoin_11.q.out
@@ -149,6 +149,7 @@ STAGE PLANS:
                   expressions: _col0 (type: int), _col7 (type: string)
                   outputColumnNames: _col0, _col1
                   Reduce Output Operator
+                    null sort order: 
                     sort order: 
                     Map-reduce partition columns: _col0 (type: int)
                     tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/smb_mapjoin_13.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/smb_mapjoin_13.q.out b/ql/src/test/results/clientpositive/smb_mapjoin_13.q.out
index fc12e35..a5a60e4 100644
--- a/ql/src/test/results/clientpositive/smb_mapjoin_13.q.out
+++ b/ql/src/test/results/clientpositive/smb_mapjoin_13.q.out
@@ -112,10 +112,11 @@ TOK_QUERY
             TOK_ALLCOLREF
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            .
-               TOK_TABLE_OR_COL
-                  a
-               key
+            TOK_NULLS_FIRST
+               .
+                  TOK_TABLE_OR_COL
+                     a
+                  key
       TOK_LIMIT
          10
 
@@ -150,6 +151,7 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1, _col2, _col3
                   Reduce Output Operator
                     key expressions: _col0 (type: int)
+                    null sort order: a
                     sort order: +
                     tag: -1
                     TopN: 10
@@ -310,10 +312,11 @@ TOK_QUERY
             TOK_ALLCOLREF
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            .
-               TOK_TABLE_OR_COL
-                  a
-               key
+            TOK_NULLS_FIRST
+               .
+                  TOK_TABLE_OR_COL
+                     a
+                  key
       TOK_LIMIT
          10
 
@@ -372,6 +375,7 @@ STAGE PLANS:
                   Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col0 (type: int)
+                    null sort order: a
                     sort order: +
                     Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
                     tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/smb_mapjoin_15.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/smb_mapjoin_15.q.out b/ql/src/test/results/clientpositive/smb_mapjoin_15.q.out
index 6f6d80b..cdf2359 100644
--- a/ql/src/test/results/clientpositive/smb_mapjoin_15.q.out
+++ b/ql/src/test/results/clientpositive/smb_mapjoin_15.q.out
@@ -84,10 +84,11 @@ TOK_QUERY
             TOK_ALLCOLREF
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            .
-               TOK_TABLE_OR_COL
-                  a
-               key
+            TOK_NULLS_FIRST
+               .
+                  TOK_TABLE_OR_COL
+                     a
+                  key
       TOK_LIMIT
          10
 
@@ -122,6 +123,7 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1, _col2, _col3
                   Reduce Output Operator
                     key expressions: _col0 (type: int)
+                    null sort order: a
                     sort order: +
                     tag: -1
                     TopN: 10
@@ -344,10 +346,11 @@ TOK_QUERY
             TOK_ALLCOLREF
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            .
-               TOK_TABLE_OR_COL
-                  a
-               key
+            TOK_NULLS_FIRST
+               .
+                  TOK_TABLE_OR_COL
+                     a
+                  key
       TOK_LIMIT
          10
 
@@ -382,6 +385,7 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
                   Reduce Output Operator
                     key expressions: _col0 (type: int)
+                    null sort order: a
                     sort order: +
                     tag: -1
                     TopN: 10
@@ -552,10 +556,11 @@ TOK_QUERY
             TOK_ALLCOLREF
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            .
-               TOK_TABLE_OR_COL
-                  a
-               key
+            TOK_NULLS_FIRST
+               .
+                  TOK_TABLE_OR_COL
+                     a
+                  key
       TOK_LIMIT
          10
 
@@ -590,6 +595,7 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
                   Reduce Output Operator
                     key expressions: _col0 (type: int)
+                    null sort order: a
                     sort order: +
                     tag: -1
                     TopN: 10
@@ -760,10 +766,11 @@ TOK_QUERY
             TOK_ALLCOLREF
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            .
-               TOK_TABLE_OR_COL
-                  a
-               key
+            TOK_NULLS_FIRST
+               .
+                  TOK_TABLE_OR_COL
+                     a
+                  key
       TOK_LIMIT
          10
 
@@ -830,6 +837,7 @@ STAGE PLANS:
                   Statistics: Num rows: 550 Data size: 7939 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col0 (type: int)
+                    null sort order: a
                     sort order: +
                     Statistics: Num rows: 550 Data size: 7939 Basic stats: COMPLETE Column stats: NONE
                     tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/sort_merge_join_desc_5.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/sort_merge_join_desc_5.q.out b/ql/src/test/results/clientpositive/sort_merge_join_desc_5.q.out
index 48767bd..29dee15 100644
--- a/ql/src/test/results/clientpositive/sort_merge_join_desc_5.q.out
+++ b/ql/src/test/results/clientpositive/sort_merge_join_desc_5.q.out
@@ -140,6 +140,7 @@ STAGE PLANS:
                   mode: hash
                   outputColumnNames: _col0
                   Reduce Output Operator
+                    null sort order: 
                     sort order: 
                     tag: -1
                     value expressions: _col0 (type: bigint)

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/sort_merge_join_desc_6.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/sort_merge_join_desc_6.q.out b/ql/src/test/results/clientpositive/sort_merge_join_desc_6.q.out
index dd2f849..b1f99e5 100644
--- a/ql/src/test/results/clientpositive/sort_merge_join_desc_6.q.out
+++ b/ql/src/test/results/clientpositive/sort_merge_join_desc_6.q.out
@@ -218,6 +218,7 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
+                    null sort order: 
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                     tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/sort_merge_join_desc_7.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/sort_merge_join_desc_7.q.out b/ql/src/test/results/clientpositive/sort_merge_join_desc_7.q.out
index 76efef5..f788fec 100644
--- a/ql/src/test/results/clientpositive/sort_merge_join_desc_7.q.out
+++ b/ql/src/test/results/clientpositive/sort_merge_join_desc_7.q.out
@@ -291,6 +291,7 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
+                    null sort order: 
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                     tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/auto_join_reordering_values.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/auto_join_reordering_values.q.out b/ql/src/test/results/clientpositive/spark/auto_join_reordering_values.q.out
index 73f62e9..d6c5ae3 100644
--- a/ql/src/test/results/clientpositive/spark/auto_join_reordering_values.q.out
+++ b/ql/src/test/results/clientpositive/spark/auto_join_reordering_values.q.out
@@ -197,6 +197,7 @@ STAGE PLANS:
                       Statistics: Num rows: 1 Data size: 36 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col1 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col1 (type: string)
                         Statistics: Num rows: 1 Data size: 36 Basic stats: COMPLETE Column stats: NONE
@@ -268,6 +269,7 @@ STAGE PLANS:
                       Statistics: Num rows: 1 Data size: 36 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 1 Data size: 36 Basic stats: COMPLETE Column stats: NONE
@@ -338,6 +340,7 @@ STAGE PLANS:
                       Statistics: Num rows: 1 Data size: 36 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: int)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 1 Data size: 36 Basic stats: COMPLETE Column stats: NONE
@@ -408,6 +411,7 @@ STAGE PLANS:
                       Statistics: Num rows: 1 Data size: 36 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: int)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 1 Data size: 36 Basic stats: COMPLETE Column stats: NONE
@@ -478,6 +482,7 @@ STAGE PLANS:
                       Statistics: Num rows: 100 Data size: 288 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: int)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 100 Data size: 288 Basic stats: COMPLETE Column stats: NONE
@@ -545,6 +550,7 @@ STAGE PLANS:
                 Statistics: Num rows: 1 Data size: 39 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 1 Data size: 39 Basic stats: COMPLETE Column stats: NONE
@@ -564,6 +570,7 @@ STAGE PLANS:
                 Statistics: Num rows: 1 Data size: 42 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col2 (type: int)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col2 (type: int)
                   Statistics: Num rows: 1 Data size: 42 Basic stats: COMPLETE Column stats: NONE
@@ -583,6 +590,7 @@ STAGE PLANS:
                 Statistics: Num rows: 1 Data size: 46 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col3 (type: int)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col3 (type: int)
                   Statistics: Num rows: 1 Data size: 46 Basic stats: COMPLETE Column stats: NONE


[07/18] hive git commit: HIVE-12994: Implement support for NULLS FIRST/NULLS LAST (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/subquery_in.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/subquery_in.q.out b/ql/src/test/results/clientpositive/subquery_in.q.out
index e59e192..58f5618 100644
--- a/ql/src/test/results/clientpositive/subquery_in.q.out
+++ b/ql/src/test/results/clientpositive/subquery_in.q.out
@@ -274,7 +274,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col5
+                  order by: _col5 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:
@@ -458,7 +458,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col5
+                  order by: _col5 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/subquery_in_having.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/subquery_in_having.q.out b/ql/src/test/results/clientpositive/subquery_in_having.q.out
index d27304f..e623299 100644
--- a/ql/src/test/results/clientpositive/subquery_in_having.q.out
+++ b/ql/src/test/results/clientpositive/subquery_in_having.q.out
@@ -1336,7 +1336,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col5
+                  order by: _col5 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/subquery_notin.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/subquery_notin.q.out b/ql/src/test/results/clientpositive/subquery_notin.q.out
index f1fceba..c600b7f 100644
--- a/ql/src/test/results/clientpositive/subquery_notin.q.out
+++ b/ql/src/test/results/clientpositive/subquery_notin.q.out
@@ -342,7 +342,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col5
+                  order by: _col5 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:
@@ -498,7 +498,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col5
+                  order by: _col5 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:
@@ -626,7 +626,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col5
+                  order by: _col5 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:
@@ -793,7 +793,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col5
+                  order by: _col5 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:
@@ -954,7 +954,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col5
+                  order by: _col5 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:
@@ -1146,7 +1146,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col5
+                  order by: _col5 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/subquery_unqualcolumnrefs.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/subquery_unqualcolumnrefs.q.out b/ql/src/test/results/clientpositive/subquery_unqualcolumnrefs.q.out
index 7f70e39..7853737 100644
--- a/ql/src/test/results/clientpositive/subquery_unqualcolumnrefs.q.out
+++ b/ql/src/test/results/clientpositive/subquery_unqualcolumnrefs.q.out
@@ -230,7 +230,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col5
+                  order by: _col5 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:
@@ -402,7 +402,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col5
+                  order by: _col5 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:
@@ -838,7 +838,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col5
+                  order by: _col5 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:
@@ -994,7 +994,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col5
+                  order by: _col5 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/temp_table_display_colstats_tbllvl.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/temp_table_display_colstats_tbllvl.q.out b/ql/src/test/results/clientpositive/temp_table_display_colstats_tbllvl.q.out
index 0762dce..532e65a 100644
--- a/ql/src/test/results/clientpositive/temp_table_display_colstats_tbllvl.q.out
+++ b/ql/src/test/results/clientpositive/temp_table_display_colstats_tbllvl.q.out
@@ -145,6 +145,7 @@ STAGE PLANS:
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
                 Reduce Output Operator
+                  null sort order: 
                   sort order: 
                   tag: -1
                   value expressions: _col0 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:string,numbitvectors:int>), _col1 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:string,numbitvectors:int>), _col2 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,bitvector:string,numbitvectors:int>)

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_1.q.out b/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_1.q.out
index c175580..bd6db7f 100644
--- a/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_1.q.out
+++ b/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_1.q.out
@@ -170,6 +170,7 @@ STAGE PLANS:
                       Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE
@@ -257,6 +258,7 @@ STAGE PLANS:
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           Reduce Output Operator
+                            null sort order: 
                             sort order: 
                             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                             tag: -1
@@ -493,6 +495,7 @@ STAGE PLANS:
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           Reduce Output Operator
+                            null sort order: 
                             sort order: 
                             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                             tag: -1
@@ -610,6 +613,7 @@ STAGE PLANS:
                       Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE
@@ -800,6 +804,7 @@ STAGE PLANS:
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           Reduce Output Operator
+                            null sort order: 
                             sort order: 
                             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                             tag: -1
@@ -917,6 +922,7 @@ STAGE PLANS:
                       Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_11.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_11.q.out b/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_11.q.out
index e4692b5..04b48a2 100644
--- a/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_11.q.out
+++ b/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_11.q.out
@@ -166,6 +166,7 @@ STAGE PLANS:
                       Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE
@@ -252,6 +253,7 @@ STAGE PLANS:
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           Reduce Output Operator
+                            null sort order: 
                             sort order: 
                             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                             tag: -1
@@ -478,6 +480,7 @@ STAGE PLANS:
                       Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE
@@ -564,6 +567,7 @@ STAGE PLANS:
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           Reduce Output Operator
+                            null sort order: 
                             sort order: 
                             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                             tag: -1
@@ -785,6 +789,7 @@ STAGE PLANS:
                     Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: key (type: string)
+                      null sort order: a
                       sort order: +
                       Map-reduce partition columns: key (type: string)
                       Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE
@@ -867,6 +872,7 @@ STAGE PLANS:
                         outputColumnNames: _col0
                         Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
+                          null sort order: 
                           sort order: 
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           tag: -1
@@ -1103,6 +1109,7 @@ STAGE PLANS:
                     Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: key (type: string)
+                      null sort order: a
                       sort order: +
                       Map-reduce partition columns: key (type: string)
                       Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE
@@ -1188,6 +1195,7 @@ STAGE PLANS:
                         outputColumnNames: _col0
                         Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
+                          null sort order: 
                           sort order: 
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           tag: -1
@@ -1299,6 +1307,7 @@ STAGE PLANS:
                     Statistics: Num rows: 116 Data size: 11624 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: key (type: string)
+                      null sort order: a
                       sort order: +
                       Map-reduce partition columns: key (type: string)
                       Statistics: Num rows: 116 Data size: 11624 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_12.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_12.q.out b/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_12.q.out
index a219e59..8664435 100644
--- a/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_12.q.out
+++ b/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_12.q.out
@@ -232,6 +232,7 @@ STAGE PLANS:
                       Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 1 Data size: 114 Basic stats: COMPLETE Column stats: NONE
@@ -303,6 +304,7 @@ STAGE PLANS:
                       Statistics: Num rows: 1 Data size: 170 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 1 Data size: 170 Basic stats: COMPLETE Column stats: NONE
@@ -404,6 +406,7 @@ STAGE PLANS:
                             outputColumnNames: _col0
                             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                             Reduce Output Operator
+                              null sort order: 
                               sort order: 
                               Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                               tag: -1
@@ -514,6 +517,7 @@ STAGE PLANS:
                   Select Operator
                     Statistics: Num rows: 1 Data size: 170 Basic stats: PARTIAL Column stats: NONE
                     Reduce Output Operator
+                      null sort order: 
                       sort order: 
                       Statistics: Num rows: 1 Data size: 170 Basic stats: PARTIAL Column stats: NONE
                       tag: 1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_2.q.out b/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_2.q.out
index 78fe2e9..a0f2be1 100644
--- a/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_2.q.out
+++ b/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_2.q.out
@@ -166,6 +166,7 @@ STAGE PLANS:
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           Reduce Output Operator
+                            null sort order: 
                             sort order: 
                             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                             tag: -1
@@ -283,6 +284,7 @@ STAGE PLANS:
                       Statistics: Num rows: 2 Data size: 226 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 2 Data size: 226 Basic stats: COMPLETE Column stats: NONE
@@ -475,6 +477,7 @@ STAGE PLANS:
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           Reduce Output Operator
+                            null sort order: 
                             sort order: 
                             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                             tag: -1
@@ -592,6 +595,7 @@ STAGE PLANS:
                       Statistics: Num rows: 2 Data size: 226 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 2 Data size: 226 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_3.q.out b/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_3.q.out
index c67c816..e6fb5dc 100644
--- a/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_3.q.out
+++ b/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_3.q.out
@@ -150,6 +150,7 @@ STAGE PLANS:
                       Statistics: Num rows: 2 Data size: 228 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 2 Data size: 228 Basic stats: COMPLETE Column stats: NONE
@@ -283,6 +284,7 @@ STAGE PLANS:
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           Reduce Output Operator
+                            null sort order: 
                             sort order: 
                             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                             tag: -1
@@ -473,6 +475,7 @@ STAGE PLANS:
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           Reduce Output Operator
+                            null sort order: 
                             sort order: 
                             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                             tag: -1
@@ -544,6 +547,7 @@ STAGE PLANS:
                       Statistics: Num rows: 2 Data size: 228 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 2 Data size: 228 Basic stats: COMPLETE Column stats: NONE
@@ -780,6 +784,7 @@ STAGE PLANS:
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           Reduce Output Operator
+                            null sort order: 
                             sort order: 
                             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                             tag: -1
@@ -851,6 +856,7 @@ STAGE PLANS:
                       Statistics: Num rows: 2 Data size: 228 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 2 Data size: 228 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_4.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_4.q.out b/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_4.q.out
index 33cb0ae..b24c2769 100644
--- a/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_4.q.out
+++ b/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_4.q.out
@@ -166,6 +166,7 @@ STAGE PLANS:
                       Statistics: Num rows: 4 Data size: 452 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 4 Data size: 452 Basic stats: COMPLETE Column stats: NONE
@@ -299,6 +300,7 @@ STAGE PLANS:
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           Reduce Output Operator
+                            null sort order: 
                             sort order: 
                             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                             tag: -1
@@ -489,6 +491,7 @@ STAGE PLANS:
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           Reduce Output Operator
+                            null sort order: 
                             sort order: 
                             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                             tag: -1
@@ -560,6 +563,7 @@ STAGE PLANS:
                       Statistics: Num rows: 4 Data size: 452 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 4 Data size: 452 Basic stats: COMPLETE Column stats: NONE
@@ -796,6 +800,7 @@ STAGE PLANS:
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           Reduce Output Operator
+                            null sort order: 
                             sort order: 
                             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                             tag: -1
@@ -867,6 +872,7 @@ STAGE PLANS:
                       Statistics: Num rows: 4 Data size: 452 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 4 Data size: 452 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_5.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_5.q.out b/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_5.q.out
index 7bb05f1..db7b805 100644
--- a/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_5.q.out
+++ b/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_5.q.out
@@ -207,6 +207,7 @@ STAGE PLANS:
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           Reduce Output Operator
+                            null sort order: 
                             sort order: 
                             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                             tag: -1
@@ -441,6 +442,7 @@ STAGE PLANS:
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           Reduce Output Operator
+                            null sort order: 
                             sort order: 
                             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                             tag: -1
@@ -621,6 +623,7 @@ STAGE PLANS:
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           Reduce Output Operator
+                            null sort order: 
                             sort order: 
                             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                             tag: -1
@@ -689,6 +692,7 @@ STAGE PLANS:
                       Statistics: Num rows: 2 Data size: 226 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 2 Data size: 226 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_7.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_7.q.out b/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_7.q.out
index dbb6ecb..faf20fa 100644
--- a/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_7.q.out
+++ b/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_7.q.out
@@ -183,6 +183,7 @@ STAGE PLANS:
                       Statistics: Num rows: 4 Data size: 452 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 4 Data size: 452 Basic stats: COMPLETE Column stats: NONE
@@ -316,6 +317,7 @@ STAGE PLANS:
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           Reduce Output Operator
+                            null sort order: 
                             sort order: 
                             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                             tag: -1
@@ -554,6 +556,7 @@ STAGE PLANS:
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           Reduce Output Operator
+                            null sort order: 
                             sort order: 
                             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                             tag: -1
@@ -671,6 +674,7 @@ STAGE PLANS:
                       Statistics: Num rows: 4 Data size: 452 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 4 Data size: 452 Basic stats: COMPLETE Column stats: NONE
@@ -909,6 +913,7 @@ STAGE PLANS:
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           Reduce Output Operator
+                            null sort order: 
                             sort order: 
                             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                             tag: -1
@@ -1026,6 +1031,7 @@ STAGE PLANS:
                       Statistics: Num rows: 4 Data size: 452 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 4 Data size: 452 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_8.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_8.q.out b/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_8.q.out
index 89034de..7b51ffc 100644
--- a/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_8.q.out
+++ b/ql/src/test/results/clientpositive/tez/auto_sortmerge_join_8.q.out
@@ -183,6 +183,7 @@ STAGE PLANS:
                       Statistics: Num rows: 2 Data size: 228 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 2 Data size: 228 Basic stats: COMPLETE Column stats: NONE
@@ -316,6 +317,7 @@ STAGE PLANS:
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           Reduce Output Operator
+                            null sort order: 
                             sort order: 
                             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                             tag: -1
@@ -554,6 +556,7 @@ STAGE PLANS:
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           Reduce Output Operator
+                            null sort order: 
                             sort order: 
                             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                             tag: -1
@@ -671,6 +674,7 @@ STAGE PLANS:
                       Statistics: Num rows: 2 Data size: 228 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 2 Data size: 228 Basic stats: COMPLETE Column stats: NONE
@@ -911,6 +915,7 @@ STAGE PLANS:
                           outputColumnNames: _col0
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                           Reduce Output Operator
+                            null sort order: 
                             sort order: 
                             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                             tag: -1
@@ -1028,6 +1033,7 @@ STAGE PLANS:
                       Statistics: Num rows: 2 Data size: 228 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 2 Data size: 228 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/tez/bucket2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/bucket2.q.out b/ql/src/test/results/clientpositive/tez/bucket2.q.out
index 952fe13..151b0ce 100644
--- a/ql/src/test/results/clientpositive/tez/bucket2.q.out
+++ b/ql/src/test/results/clientpositive/tez/bucket2.q.out
@@ -60,6 +60,7 @@ STAGE PLANS:
                     outputColumnNames: _col0, _col1
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
+                      null sort order: 
                       sort order: 
                       Map-reduce partition columns: UDFToInteger(_col0) (type: int)
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/tez/bucket3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/bucket3.q.out b/ql/src/test/results/clientpositive/tez/bucket3.q.out
index b63ebbd..3308a8a 100644
--- a/ql/src/test/results/clientpositive/tez/bucket3.q.out
+++ b/ql/src/test/results/clientpositive/tez/bucket3.q.out
@@ -64,6 +64,7 @@ STAGE PLANS:
                     outputColumnNames: _col0, _col1
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
+                      null sort order: 
                       sort order: 
                       Map-reduce partition columns: UDFToInteger(_col0) (type: int)
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/tez/bucket4.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/bucket4.q.out b/ql/src/test/results/clientpositive/tez/bucket4.q.out
index 5f4c2e9..f2030e8 100644
--- a/ql/src/test/results/clientpositive/tez/bucket4.q.out
+++ b/ql/src/test/results/clientpositive/tez/bucket4.q.out
@@ -57,6 +57,7 @@ STAGE PLANS:
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: UDFToInteger(_col0) (type: int)
+                      null sort order: a
                       sort order: +
                       Map-reduce partition columns: UDFToInteger(_col0) (type: int)
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/tez/ctas.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/ctas.q.out b/ql/src/test/results/clientpositive/tez/ctas.q.out
index 74e8b98..0259c02 100644
--- a/ql/src/test/results/clientpositive/tez/ctas.q.out
+++ b/ql/src/test/results/clientpositive/tez/ctas.q.out
@@ -722,11 +722,13 @@ TOK_CREATETABLE
                   value
          TOK_SORTBY
             TOK_TABSORTCOLNAMEASC
-               TOK_TABLE_OR_COL
-                  key
+               TOK_NULLS_FIRST
+                  TOK_TABLE_OR_COL
+                     key
             TOK_TABSORTCOLNAMEASC
-               TOK_TABLE_OR_COL
-                  value
+               TOK_NULLS_FIRST
+                  TOK_TABLE_OR_COL
+                     value
          TOK_LIMIT
             10
 
@@ -759,6 +761,7 @@ STAGE PLANS:
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: string), _col1 (type: string)
+                      null sort order: aa
                       sort order: ++
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                       tag: -1
@@ -826,6 +829,7 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 100 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col0 (type: string), _col1 (type: string)
+                    null sort order: aa
                     sort order: ++
                     Statistics: Num rows: 10 Data size: 100 Basic stats: COMPLETE Column stats: NONE
                     tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/tez/disable_merge_for_bucketing.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/disable_merge_for_bucketing.q.out b/ql/src/test/results/clientpositive/tez/disable_merge_for_bucketing.q.out
index 7b59dea..65f750b 100644
--- a/ql/src/test/results/clientpositive/tez/disable_merge_for_bucketing.q.out
+++ b/ql/src/test/results/clientpositive/tez/disable_merge_for_bucketing.q.out
@@ -56,6 +56,7 @@ STAGE PLANS:
                     outputColumnNames: _col0, _col1
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
+                      null sort order: 
                       sort order: 
                       Map-reduce partition columns: UDFToInteger(_col0) (type: int)
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/tez/dynpart_sort_opt_vectorization.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/dynpart_sort_opt_vectorization.q.out b/ql/src/test/results/clientpositive/tez/dynpart_sort_opt_vectorization.q.out
index a90e3f6..f7a94ac 100644
--- a/ql/src/test/results/clientpositive/tez/dynpart_sort_opt_vectorization.q.out
+++ b/ql/src/test/results/clientpositive/tez/dynpart_sort_opt_vectorization.q.out
@@ -1215,7 +1215,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	4                   	 
 Bucket Columns:     	[si]                	 
-Sort Columns:       	[Order(col:f, order:1)]	 
+Sort Columns:       	[Order(col:f, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: desc formatted over1k_part_buck_sort_orc partition(t="__HIVE_DEFAULT_PARTITION__")
@@ -1256,7 +1256,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	4                   	 
 Bucket Columns:     	[si]                	 
-Sort Columns:       	[Order(col:f, order:1)]	 
+Sort Columns:       	[Order(col:f, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: select count(*) from over1k_part_orc
@@ -2255,7 +2255,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[si]                	 
-Sort Columns:       	[Order(col:f, order:1)]	 
+Sort Columns:       	[Order(col:f, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: desc formatted over1k_part_buck_sort2_orc partition(t="__HIVE_DEFAULT_PARTITION__")
@@ -2296,7 +2296,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[si]                	 
-Sort Columns:       	[Order(col:f, order:1)]	 
+Sort Columns:       	[Order(col:f, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: explain select * from over1k_part_buck_sort2_orc
@@ -2471,7 +2471,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[si]                	 
-Sort Columns:       	[Order(col:f, order:1)]	 
+Sort Columns:       	[Order(col:f, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: desc formatted over1k_part_buck_sort2_orc partition(t="__HIVE_DEFAULT_PARTITION__")
@@ -2512,7 +2512,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[si]                	 
-Sort Columns:       	[Order(col:f, order:1)]	 
+Sort Columns:       	[Order(col:f, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: explain select * from over1k_part_buck_sort2_orc

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/tez/dynpart_sort_optimization.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/dynpart_sort_optimization.q.out b/ql/src/test/results/clientpositive/tez/dynpart_sort_optimization.q.out
index 5292106..01dc4ae 100644
--- a/ql/src/test/results/clientpositive/tez/dynpart_sort_optimization.q.out
+++ b/ql/src/test/results/clientpositive/tez/dynpart_sort_optimization.q.out
@@ -1128,7 +1128,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	4                   	 
 Bucket Columns:     	[si]                	 
-Sort Columns:       	[Order(col:f, order:1)]	 
+Sort Columns:       	[Order(col:f, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: desc formatted over1k_part_buck_sort partition(t="__HIVE_DEFAULT_PARTITION__")
@@ -1169,7 +1169,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	4                   	 
 Bucket Columns:     	[si]                	 
-Sort Columns:       	[Order(col:f, order:1)]	 
+Sort Columns:       	[Order(col:f, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: select count(*) from over1k_part
@@ -2152,7 +2152,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[si]                	 
-Sort Columns:       	[Order(col:f, order:1)]	 
+Sort Columns:       	[Order(col:f, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: desc formatted over1k_part_buck_sort2 partition(t="__HIVE_DEFAULT_PARTITION__")
@@ -2193,7 +2193,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[si]                	 
-Sort Columns:       	[Order(col:f, order:1)]	 
+Sort Columns:       	[Order(col:f, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: select * from over1k_part_buck_sort2
@@ -2295,7 +2295,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[si]                	 
-Sort Columns:       	[Order(col:f, order:1)]	 
+Sort Columns:       	[Order(col:f, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: desc formatted over1k_part_buck_sort2 partition(t="__HIVE_DEFAULT_PARTITION__")
@@ -2336,7 +2336,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[si]                	 
-Sort Columns:       	[Order(col:f, order:1)]	 
+Sort Columns:       	[Order(col:f, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: select * from over1k_part_buck_sort2

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/tez/explainuser_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/explainuser_1.q.out b/ql/src/test/results/clientpositive/tez/explainuser_1.q.out
index 9942b8c..70a1322 100644
--- a/ql/src/test/results/clientpositive/tez/explainuser_1.q.out
+++ b/ql/src/test/results/clientpositive/tez/explainuser_1.q.out
@@ -2302,7 +2302,7 @@ Stage-0
                 Filter Operator [FIL_25] (rows=26 width=491)
                   predicate:first_value_window_0 is not null
                   PTF Operator [PTF_10] (rows=26 width=491)
-                    Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col5","partition by:":"_col2"}]
+                    Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col5 ASC NULLS FIRST","partition by:":"_col2"}]
                     Select Operator [SEL_9] (rows=26 width=491)
                       Output:["_col1","_col2","_col5"]
                     <-Map 4 [SIMPLE_EDGE]
@@ -2671,7 +2671,7 @@ Stage-0
         Select Operator [SEL_4] (rows=20 width=52)
           Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9"]
           PTF Operator [PTF_3] (rows=20 width=459)
-            Function definitions:[{},{"name:":"windowingtablefunction","order by:":"0","partition by:":"0"}]
+            Function definitions:[{},{"name:":"windowingtablefunction","order by:":"0 ASC NULLS FIRST","partition by:":"0"}]
             Select Operator [SEL_2] (rows=20 width=459)
               Output:["_col2","_col3"]
             <-Map 1 [SIMPLE_EDGE]
@@ -2698,7 +2698,7 @@ Stage-0
         Select Operator [SEL_4] (rows=20 width=52)
           Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9"]
           PTF Operator [PTF_3] (rows=20 width=459)
-            Function definitions:[{},{"name:":"windowingtablefunction","order by:":"0","partition by:":"0"}]
+            Function definitions:[{},{"name:":"windowingtablefunction","order by:":"0 ASC NULLS FIRST","partition by:":"0"}]
             Select Operator [SEL_2] (rows=20 width=459)
               Output:["_col2","_col3"]
             <-Map 1 [SIMPLE_EDGE]
@@ -2725,7 +2725,7 @@ Stage-0
         Select Operator [SEL_4] (rows=20 width=64)
           Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10"]
           PTF Operator [PTF_3] (rows=20 width=612)
-            Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col1","partition by:":"_col0"}]
+            Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col1 ASC NULLS FIRST","partition by:":"_col0"}]
             Select Operator [SEL_2] (rows=20 width=612)
               Output:["_col0","_col1","_col2","_col3"]
             <-Map 1 [SIMPLE_EDGE]
@@ -2752,7 +2752,7 @@ Stage-0
         Select Operator [SEL_4] (rows=25 width=179)
           Output:["_col0","_col1","_col2"]
           PTF Operator [PTF_3] (rows=25 width=443)
-            Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col1","partition by:":"_col0"}]
+            Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col1 ASC NULLS FIRST","partition by:":"_col0"}]
             Select Operator [SEL_2] (rows=25 width=443)
               Output:["_col0","_col1"]
             <-Map 1 [SIMPLE_EDGE]
@@ -4326,14 +4326,14 @@ Stage-0
         Select Operator [SEL_7] (rows=26 width=239)
           Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
           PTF Operator [PTF_6] (rows=26 width=499)
-            Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col1","partition by:":"_col2"}]
+            Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col1 ASC NULLS FIRST","partition by:":"_col2"}]
             Select Operator [SEL_5] (rows=26 width=499)
               Output:["_col1","_col2","_col5","_col7"]
             <-Reducer 2 [SIMPLE_EDGE]
               SHUFFLE [RS_4]
                 PartitionCols:_col2
                 PTF Operator [PTF_3] (rows=26 width=499)
-                  Function definitions:[{},{"Partition table definition":{"name:":"noop","order by:":"_col1","partition by:":"_col2"}}]
+                  Function definitions:[{},{"Partition table definition":{"name:":"noop","order by:":"_col1 ASC NULLS FIRST","partition by:":"_col2"}}]
                   Select Operator [SEL_2] (rows=26 width=499)
                     Output:["_col1","_col2","_col5","_col7"]
                   <-Map 1 [SIMPLE_EDGE]
@@ -4372,14 +4372,14 @@ Stage-0
         Select Operator [SEL_14] (rows=29 width=227)
           Output:["_col0","_col1","_col2","_col3"]
           PTF Operator [PTF_13] (rows=29 width=223)
-            Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col1","partition by:":"_col2"}]
+            Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col1 ASC NULLS FIRST","partition by:":"_col2"}]
             Select Operator [SEL_12] (rows=29 width=223)
               Output:["_col1","_col2","_col5"]
             <-Reducer 2 [SIMPLE_EDGE]
               SHUFFLE [RS_11]
                 PartitionCols:_col2
                 PTF Operator [PTF_10] (rows=29 width=223)
-                  Function definitions:[{},{"Partition table definition":{"name:":"noop","order by:":"_col1","partition by:":"_col2"}}]
+                  Function definitions:[{},{"Partition table definition":{"name:":"noop","order by:":"_col1 ASC NULLS FIRST","partition by:":"_col2"}}]
                   Select Operator [SEL_9] (rows=29 width=223)
                     Output:["_col1","_col2","_col5"]
                   <-Map 1 [SIMPLE_EDGE]
@@ -4434,14 +4434,14 @@ Stage-0
         Select Operator [SEL_7] (rows=26 width=239)
           Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
           PTF Operator [PTF_6] (rows=26 width=499)
-            Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col1","partition by:":"_col2"}]
+            Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col1 ASC NULLS FIRST","partition by:":"_col2"}]
             Select Operator [SEL_5] (rows=26 width=499)
               Output:["_col1","_col2","_col5","_col7"]
             <-Reducer 2 [SIMPLE_EDGE]
               SHUFFLE [RS_4]
                 PartitionCols:_col2
                 PTF Operator [PTF_3] (rows=26 width=499)
-                  Function definitions:[{},{"Partition table definition":{"name:":"noop","order by:":"_col1","partition by:":"_col2"}}]
+                  Function definitions:[{},{"Partition table definition":{"name:":"noop","order by:":"_col1 ASC NULLS FIRST","partition by:":"_col2"}}]
                   Select Operator [SEL_2] (rows=26 width=499)
                     Output:["_col1","_col2","_col5","_col7"]
                   <-Map 1 [SIMPLE_EDGE]
@@ -4485,14 +4485,14 @@ Stage-0
         Select Operator [SEL_7] (rows=26 width=239)
           Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
           PTF Operator [PTF_6] (rows=26 width=491)
-            Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col1","partition by:":"_col2"}]
+            Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col1 ASC NULLS FIRST","partition by:":"_col2"}]
             Select Operator [SEL_5] (rows=26 width=491)
               Output:["_col1","_col2","_col5"]
             <-Reducer 2 [SIMPLE_EDGE]
               SHUFFLE [RS_4]
                 PartitionCols:_col2
                 PTF Operator [PTF_3] (rows=26 width=491)
-                  Function definitions:[{},{"Partition table definition":{"name:":"noop","order by:":"_col1","partition by:":"_col2"}}]
+                  Function definitions:[{},{"Partition table definition":{"name:":"noop","order by:":"_col1 ASC NULLS FIRST","partition by:":"_col2"}}]
                   Select Operator [SEL_2] (rows=26 width=491)
                     Output:["_col1","_col2","_col5"]
                   <-Map 1 [SIMPLE_EDGE]
@@ -4539,7 +4539,7 @@ Stage-0
         Select Operator [SEL_12] (rows=26 width=239)
           Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
           PTF Operator [PTF_11] (rows=26 width=223)
-            Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col1","partition by:":"_col0"}]
+            Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col1 ASC NULLS FIRST","partition by:":"_col0"}]
             Group By Operator [GBY_8] (rows=26 width=223)
               Output:["_col0","_col1","_col2"],keys:KEY._col0, KEY._col1, KEY._col2
             <-Reducer 3 [SIMPLE_EDGE]
@@ -4553,7 +4553,7 @@ Stage-0
                     Select Operator [SEL_4] (rows=26 width=491)
                       Output:["_col1","_col2","_col5"]
                       PTF Operator [PTF_3] (rows=26 width=491)
-                        Function definitions:[{},{"Partition table definition":{"name:":"noop","order by:":"_col1","partition by:":"_col2"}}]
+                        Function definitions:[{},{"Partition table definition":{"name:":"noop","order by:":"_col1 ASC NULLS FIRST","partition by:":"_col2"}}]
                         Select Operator [SEL_2] (rows=26 width=491)
                           Output:["_col1","_col2","_col5"]
                         <-Map 1 [SIMPLE_EDGE]
@@ -4599,7 +4599,7 @@ Stage-0
         <-Filter Operator [FIL_14] (rows=26 width=887)
             predicate:_col0 is not null
             PTF Operator [PTF_4] (rows=26 width=887)
-              Function definitions:[{},{"Partition table definition":{"name:":"noop","order by:":"_col1","partition by:":"_col2"}}]
+              Function definitions:[{},{"Partition table definition":{"name:":"noop","order by:":"_col1 ASC NULLS FIRST","partition by:":"_col2"}}]
               Select Operator [SEL_3] (rows=26 width=887)
                 Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"]
               <-Map 1 [SIMPLE_EDGE]
@@ -4637,21 +4637,21 @@ Stage-0
         Select Operator [SEL_8] (rows=26 width=227)
           Output:["_col0","_col1","_col2","_col3"]
           PTF Operator [PTF_7] (rows=26 width=491)
-            Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col1, _col5(DESC)","partition by:":"_col2"}]
+            Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col1 ASC NULLS FIRST, _col5 DESC NULLS LAST","partition by:":"_col2"}]
             Select Operator [SEL_6] (rows=26 width=491)
               Output:["_col1","_col2","_col5"]
             <-Reducer 2 [SIMPLE_EDGE]
               SHUFFLE [RS_5]
                 PartitionCols:_col2
                 PTF Operator [PTF_4] (rows=26 width=491)
-                  Function definitions:[{},{"Partition table definition":{"name:":"noopwithmap","order by:":"_col1, _col5(DESC)","partition by:":"_col2"}}]
+                  Function definitions:[{},{"Partition table definition":{"name:":"noopwithmap","order by:":"_col1 ASC NULLS FIRST, _col5 DESC NULLS LAST","partition by:":"_col2"}}]
                   Select Operator [SEL_3] (rows=26 width=491)
                     Output:["_col1","_col2","_col5"]
                   <-Map 1 [SIMPLE_EDGE]
                     SHUFFLE [RS_2]
                       PartitionCols:p_mfgr
                       PTF Operator [PTF_1] (rows=26 width=223)
-                        Function definitions:[{},{"Partition table definition":{"name:":"noopwithmap","order by:":"p_name, p_size(DESC)","partition by:":"p_mfgr"}}]
+                        Function definitions:[{},{"Partition table definition":{"name:":"noopwithmap","order by:":"p_name ASC NULLS FIRST, p_size DESC NULLS LAST","partition by:":"p_mfgr"}}]
                         TableScan [TS_0] (rows=26 width=223)
                           default@part,part,Tbl:COMPLETE,Col:COMPLETE,Output:["p_name","p_mfgr","p_size"]
 
@@ -4688,21 +4688,21 @@ Stage-0
         Select Operator [SEL_8] (rows=26 width=239)
           Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
           PTF Operator [PTF_7] (rows=26 width=499)
-            Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col1","partition by:":"_col2"}]
+            Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col1 ASC NULLS FIRST","partition by:":"_col2"}]
             Select Operator [SEL_6] (rows=26 width=499)
               Output:["_col1","_col2","_col5","_col7"]
             <-Reducer 2 [SIMPLE_EDGE]
               SHUFFLE [RS_5]
                 PartitionCols:_col2
                 PTF Operator [PTF_4] (rows=26 width=499)
-                  Function definitions:[{},{"Partition table definition":{"name:":"noopwithmap","order by:":"_col1","partition by:":"_col2"}}]
+                  Function definitions:[{},{"Partition table definition":{"name:":"noopwithmap","order by:":"_col1 ASC NULLS FIRST","partition by:":"_col2"}}]
                   Select Operator [SEL_3] (rows=26 width=499)
                     Output:["_col1","_col2","_col5","_col7"]
                   <-Map 1 [SIMPLE_EDGE]
                     SHUFFLE [RS_2]
                       PartitionCols:p_mfgr
                       PTF Operator [PTF_1] (rows=26 width=231)
-                        Function definitions:[{},{"Partition table definition":{"name:":"noopwithmap","order by:":"p_name","partition by:":"p_mfgr"}}]
+                        Function definitions:[{},{"Partition table definition":{"name:":"noopwithmap","order by:":"p_name ASC NULLS FIRST","partition by:":"p_mfgr"}}]
                         TableScan [TS_0] (rows=26 width=231)
                           default@part,part,Tbl:COMPLETE,Col:COMPLETE,Output:["p_name","p_mfgr","p_size","p_retailprice"]
 
@@ -4739,14 +4739,14 @@ Stage-0
         Select Operator [SEL_7] (rows=26 width=239)
           Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
           PTF Operator [PTF_6] (rows=26 width=499)
-            Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col1","partition by:":"_col2"}]
+            Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col1 ASC NULLS FIRST","partition by:":"_col2"}]
             Select Operator [SEL_5] (rows=26 width=499)
               Output:["_col1","_col2","_col5","_col7"]
             <-Reducer 2 [SIMPLE_EDGE]
               SHUFFLE [RS_4]
                 PartitionCols:_col2
                 PTF Operator [PTF_3] (rows=26 width=499)
-                  Function definitions:[{},{"Partition table definition":{"name:":"noop","order by:":"_col1","partition by:":"_col2"}}]
+                  Function definitions:[{},{"Partition table definition":{"name:":"noop","order by:":"_col1 ASC NULLS FIRST","partition by:":"_col2"}}]
                   Select Operator [SEL_2] (rows=26 width=499)
                     Output:["_col1","_col2","_col5","_col7"]
                   <-Map 1 [SIMPLE_EDGE]
@@ -4791,23 +4791,23 @@ Stage-0
         Select Operator [SEL_11] (rows=26 width=239)
           Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
           PTF Operator [PTF_10] (rows=26 width=499)
-            Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col1","partition by:":"_col2"}]
+            Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col1 ASC NULLS FIRST","partition by:":"_col2"}]
             Select Operator [SEL_9] (rows=26 width=499)
               Output:["_col1","_col2","_col5","_col7"]
             <-Reducer 3 [SIMPLE_EDGE]
               SHUFFLE [RS_8]
                 PartitionCols:_col2
                 PTF Operator [PTF_7] (rows=26 width=499)
-                  Function definitions:[{},{"Partition table definition":{"name:":"noopwithmap","order by:":"_col2(DESC), _col1","partition by:":"_col2"}},{"Partition table definition":{"name:":"noop","order by:":"_col2(DESC), _col1","partition by:":"_col2"}}]
+                  Function definitions:[{},{"Partition table definition":{"name:":"noopwithmap","order by:":"_col2 DESC NULLS LAST, _col1 ASC NULLS FIRST","partition by:":"_col2"}},{"Partition table definition":{"name:":"noop","order by:":"_col2 DESC NULLS LAST, _col1 ASC NULLS FIRST","partition by:":"_col2"}}]
                   Select Operator [SEL_6] (rows=26 width=499)
                     Output:["_col1","_col2","_col5","_col7"]
                   <-Reducer 2 [SIMPLE_EDGE]
                     SHUFFLE [RS_5]
                       PartitionCols:_col2
                       PTF Operator [PTF_4] (rows=26 width=499)
-                        Function definitions:[{},{"Partition table definition":{"name:":"noopwithmap","order by:":"_col2(DESC), _col1","partition by:":"_col2"}},{"Partition table definition":{"name:":"noop","order by:":"_col2(DESC), _col1","partition by:":"_col2"}}]
+                        Function definitions:[{},{"Partition table definition":{"name:":"noopwithmap","order by:":"_col2 DESC NULLS LAST, _col1 ASC NULLS FIRST","partition by:":"_col2"}},{"Partition table definition":{"name:":"noop","order by:":"_col2 DESC NULLS LAST, _col1 ASC NULLS FIRST","partition by:":"_col2"}}]
                         PTF Operator [PTF_3] (rows=26 width=499)
-                          Function definitions:[{},{"Partition table definition":{"name:":"noop","order by:":"_col2(DESC), _col1","partition by:":"_col2"}}]
+                          Function definitions:[{},{"Partition table definition":{"name:":"noop","order by:":"_col2 DESC NULLS LAST, _col1 ASC NULLS FIRST","partition by:":"_col2"}}]
                           Select Operator [SEL_2] (rows=26 width=499)
                             Output:["_col1","_col2","_col5","_col7"]
                           <-Map 1 [SIMPLE_EDGE]
@@ -4857,14 +4857,14 @@ Stage-0
         Select Operator [SEL_7] (rows=26 width=235)
           Output:["_col0","_col1","_col2","_col3"]
           PTF Operator [PTF_6] (rows=26 width=499)
-            Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col1","partition by:":"_col2"}]
+            Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col1 ASC NULLS FIRST","partition by:":"_col2"}]
             Select Operator [SEL_5] (rows=26 width=499)
               Output:["_col1","_col2","_col5","_col7"]
             <-Reducer 2 [SIMPLE_EDGE]
               SHUFFLE [RS_4]
                 PartitionCols:_col2
                 PTF Operator [PTF_3] (rows=26 width=499)
-                  Function definitions:[{},{"Partition table definition":{"name:":"noop","order by:":"_col1","partition by:":"_col2"}}]
+                  Function definitions:[{},{"Partition table definition":{"name:":"noop","order by:":"_col1 ASC NULLS FIRST","partition by:":"_col2"}}]
                   Select Operator [SEL_2] (rows=26 width=499)
                     Output:["_col1","_col2","_col5","_col7"]
                   <-Map 1 [SIMPLE_EDGE]
@@ -4912,7 +4912,7 @@ Stage-0
         Select Operator [SEL_13] (rows=29 width=259)
           Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8"]
           PTF Operator [PTF_12] (rows=29 width=767)
-            Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col1","partition by:":"_col2"}]
+            Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col1 ASC NULLS FIRST","partition by:":"_col2"}]
             Select Operator [SEL_11] (rows=29 width=767)
               Output:["_col1","_col2","_col5","_col7"]
             <-Reducer 2 [SIMPLE_EDGE]
@@ -4930,7 +4930,7 @@ Stage-0
                 <-Filter Operator [FIL_18] (rows=26 width=503)
                     predicate:_col0 is not null
                     PTF Operator [PTF_4] (rows=26 width=503)
-                      Function definitions:[{},{"Partition table definition":{"name:":"noop","order by:":"_col1","partition by:":"_col2"}}]
+                      Function definitions:[{},{"Partition table definition":{"name:":"noop","order by:":"_col1 ASC NULLS FIRST","partition by:":"_col2"}}]
                       Select Operator [SEL_3] (rows=26 width=503)
                         Output:["_col0","_col1","_col2","_col5","_col7"]
                       <-Map 1 [SIMPLE_EDGE]
@@ -5058,14 +5058,14 @@ Stage-4
                 Select Operator [SEL_7] (rows=26 width=239)
                   Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
                   PTF Operator [PTF_6] (rows=26 width=499)
-                    Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col1","partition by:":"_col2"}]
+                    Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col1 ASC NULLS FIRST","partition by:":"_col2"}]
                     Select Operator [SEL_5] (rows=26 width=499)
                       Output:["_col1","_col2","_col5","_col7"]
                     <-Reducer 2 [SIMPLE_EDGE]
                       SHUFFLE [RS_4]
                         PartitionCols:_col2
                         PTF Operator [PTF_3] (rows=26 width=499)
-                          Function definitions:[{},{"Partition table definition":{"name:":"noop","order by:":"_col1","partition by:":"_col2"}}]
+                          Function definitions:[{},{"Partition table definition":{"name:":"noop","order by:":"_col1 ASC NULLS FIRST","partition by:":"_col2"}}]
                           Select Operator [SEL_2] (rows=26 width=499)
                             Output:["_col1","_col2","_col5","_col7"]
                           <-Map 1 [SIMPLE_EDGE]
@@ -5079,7 +5079,7 @@ Stage-4
                 Select Operator [SEL_17] (rows=26 width=247)
                   Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7"]
                   PTF Operator [PTF_16] (rows=26 width=499)
-                    Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col3, _col2","partition by:":"_col3"}]
+                    Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col3 ASC NULLS FIRST, _col2 ASC NULLS FIRST","partition by:":"_col3"}]
                     Select Operator [SEL_15] (rows=26 width=499)
                       Output:["_col0","_col2","_col3","_col6"]
                     <-Reducer 4 [SIMPLE_EDGE]
@@ -5088,7 +5088,7 @@ Stage-4
                         Select Operator [SEL_13] (rows=26 width=491)
                           Output:["_col1","_col2","_col5","sum_window_0"]
                           PTF Operator [PTF_12] (rows=26 width=491)
-                            Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col5","partition by:":"_col2"}]
+                            Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col5 ASC NULLS FIRST","partition by:":"_col2"}]
                             Select Operator [SEL_11] (rows=26 width=491)
                               Output:["_col1","_col2","_col5"]
                             <-Reducer 2 [SIMPLE_EDGE]
@@ -5152,23 +5152,23 @@ Stage-0
         Select Operator [SEL_11] (rows=26 width=239)
           Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
           PTF Operator [PTF_10] (rows=26 width=491)
-            Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col2, _col1","partition by:":"_col2, _col1"}]
+            Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST","partition by:":"_col2, _col1"}]
             Select Operator [SEL_9] (rows=26 width=491)
               Output:["_col1","_col2","_col5"]
             <-Reducer 3 [SIMPLE_EDGE]
               SHUFFLE [RS_8]
                 PartitionCols:_col2, _col1
                 PTF Operator [PTF_7] (rows=26 width=491)
-                  Function definitions:[{},{"Partition table definition":{"name:":"noopwithmap","order by:":"_col2, _col1","partition by:":"_col2, _col1"}},{"Partition table definition":{"name:":"noop","order by:":"_col2, _col1","partition by:":"_col2, _col1"}}]
+                  Function definitions:[{},{"Partition table definition":{"name:":"noopwithmap","order by:":"_col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST","partition by:":"_col2, _col1"}},{"Partition table definition":{"name:":"noop","order by:":"_col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST","partition by:":"_col2, _col1"}}]
                   Select Operator [SEL_6] (rows=26 width=491)
                     Output:["_col1","_col2","_col5"]
                   <-Reducer 2 [SIMPLE_EDGE]
                     SHUFFLE [RS_5]
                       PartitionCols:_col2, _col1
                       PTF Operator [PTF_4] (rows=26 width=491)
-                        Function definitions:[{},{"Partition table definition":{"name:":"noopwithmap","order by:":"_col2, _col1","partition by:":"_col2, _col1"}},{"Partition table definition":{"name:":"noop","order by:":"_col2, _col1","partition by:":"_col2, _col1"}}]
+                        Function definitions:[{},{"Partition table definition":{"name:":"noopwithmap","order by:":"_col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST","partition by:":"_col2, _col1"}},{"Partition table definition":{"name:":"noop","order by:":"_col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST","partition by:":"_col2, _col1"}}]
                         PTF Operator [PTF_3] (rows=26 width=491)
-                          Function definitions:[{},{"Partition table definition":{"name:":"noop","order by:":"_col2","partition by:":"_col2"}},{"Partition table definition":{"name:":"noop","order by:":"_col2","partition by:":"_col2"}}]
+                          Function definitions:[{},{"Partition table definition":{"name:":"noop","order by:":"_col2 ASC NULLS FIRST","partition by:":"_col2"}},{"Partition table definition":{"name:":"noop","order by:":"_col2 ASC NULLS FIRST","partition by:":"_col2"}}]
                           Select Operator [SEL_2] (rows=26 width=491)
                             Output:["_col1","_col2","_col5"]
                           <-Map 1 [SIMPLE_EDGE]
@@ -5228,28 +5228,28 @@ Stage-0
         Select Operator [SEL_13] (rows=26 width=239)
           Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
           PTF Operator [PTF_12] (rows=26 width=491)
-            Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col1","partition by:":"_col2"}]
+            Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col1 ASC NULLS FIRST","partition by:":"_col2"}]
             Select Operator [SEL_11] (rows=26 width=491)
               Output:["_col1","_col2","_col5"]
             <-Reducer 4 [SIMPLE_EDGE]
               SHUFFLE [RS_10]
                 PartitionCols:_col2
                 PTF Operator [PTF_9] (rows=26 width=491)
-                  Function definitions:[{},{"Partition table definition":{"name:":"noop","order by:":"_col2","partition by:":"_col2"}}]
+                  Function definitions:[{},{"Partition table definition":{"name:":"noop","order by:":"_col2 ASC NULLS FIRST","partition by:":"_col2"}}]
                   Select Operator [SEL_8] (rows=26 width=491)
                     Output:["_col1","_col2","_col5"]
                   <-Reducer 3 [SIMPLE_EDGE]
                     SHUFFLE [RS_7]
                       PartitionCols:_col2
                       PTF Operator [PTF_6] (rows=26 width=491)
-                        Function definitions:[{},{"Partition table definition":{"name:":"noop","order by:":"_col2, _col1","partition by:":"_col2, _col1"}}]
+                        Function definitions:[{},{"Partition table definition":{"name:":"noop","order by:":"_col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST","partition by:":"_col2, _col1"}}]
                         Select Operator [SEL_5] (rows=26 width=491)
                           Output:["_col1","_col2","_col5"]
                         <-Reducer 2 [SIMPLE_EDGE]
                           SHUFFLE [RS_4]
                             PartitionCols:_col2, _col1
                             PTF Operator [PTF_3] (rows=26 width=491)
-                              Function definitions:[{},{"Partition table definition":{"name:":"noop","order by:":"_col2","partition by:":"_col2"}},{"Partition table definition":{"name:":"noop","order by:":"_col2","partition by:":"_col2"}}]
+                              Function definitions:[{},{"Partition table definition":{"name:":"noop","order by:":"_col2 ASC NULLS FIRST","partition by:":"_col2"}},{"Partition table definition":{"name:":"noop","order by:":"_col2 ASC NULLS FIRST","partition by:":"_col2"}}]
                               Select Operator [SEL_2] (rows=26 width=491)
                                 Output:["_col1","_col2","_col5"]
                               <-Map 1 [SIMPLE_EDGE]
@@ -5304,21 +5304,21 @@ Stage-0
         Select Operator [SEL_10] (rows=26 width=239)
           Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
           PTF Operator [PTF_9] (rows=26 width=491)
-            Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col1","partition by:":"_col2"}]
+            Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col1 ASC NULLS FIRST","partition by:":"_col2"}]
             Select Operator [SEL_8] (rows=26 width=491)
               Output:["_col1","_col2","_col5"]
             <-Reducer 3 [SIMPLE_EDGE]
               SHUFFLE [RS_7]
                 PartitionCols:_col2
                 PTF Operator [PTF_6] (rows=26 width=491)
-                  Function definitions:[{},{"Partition table definition":{"name:":"noop","order by:":"_col2","partition by:":"_col2"}},{"Partition table definition":{"name:":"noop","order by:":"_col2","partition by:":"_col2"}}]
+                  Function definitions:[{},{"Partition table definition":{"name:":"noop","order by:":"_col2 ASC NULLS FIRST","partition by:":"_col2"}},{"Partition table definition":{"name:":"noop","order by:":"_col2 ASC NULLS FIRST","partition by:":"_col2"}}]
                   Select Operator [SEL_5] (rows=26 width=491)
                     Output:["_col1","_col2","_col5"]
                   <-Reducer 2 [SIMPLE_EDGE]
                     SHUFFLE [RS_4]
                       PartitionCols:_col2
                       PTF Operator [PTF_3] (rows=26 width=491)
-                        Function definitions:[{},{"Partition table definition":{"name:":"noop","order by:":"_col2, _col1","partition by:":"_col2, _col1"}},{"Partition table definition":{"name:":"noop","order by:":"_col2, _col1","partition by:":"_col2, _col1"}}]
+                        Function definitions:[{},{"Partition table definition":{"name:":"noop","order by:":"_col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST","partition by:":"_col2, _col1"}},{"Partition table definition":{"name:":"noop","order by:":"_col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST","partition by:":"_col2, _col1"}}]
                         Select Operator [SEL_2] (rows=26 width=491)
                           Output:["_col1","_col2","_col5"]
                         <-Map 1 [SIMPLE_EDGE]

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/tez/filter_join_breaktask.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/filter_join_breaktask.q.out b/ql/src/test/results/clientpositive/tez/filter_join_breaktask.q.out
index 83c4734..15cad97 100644
--- a/ql/src/test/results/clientpositive/tez/filter_join_breaktask.q.out
+++ b/ql/src/test/results/clientpositive/tez/filter_join_breaktask.q.out
@@ -163,6 +163,7 @@ STAGE PLANS:
                       Statistics: Num rows: 25 Data size: 211 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: int)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 25 Data size: 211 Basic stats: COMPLETE Column stats: NONE
@@ -234,6 +235,7 @@ STAGE PLANS:
                       Statistics: Num rows: 25 Data size: 211 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: int)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 25 Data size: 211 Basic stats: COMPLETE Column stats: NONE
@@ -306,6 +308,7 @@ STAGE PLANS:
                       Statistics: Num rows: 25 Data size: 211 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 25 Data size: 211 Basic stats: COMPLETE Column stats: NONE
@@ -375,6 +378,7 @@ STAGE PLANS:
                 Statistics: Num rows: 27 Data size: 232 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col3 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col3 (type: string)
                   Statistics: Num rows: 27 Data size: 232 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/tez/mapjoin_mapjoin.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/mapjoin_mapjoin.q.out b/ql/src/test/results/clientpositive/tez/mapjoin_mapjoin.q.out
index efe4ebf..d03b1cc 100644
--- a/ql/src/test/results/clientpositive/tez/mapjoin_mapjoin.q.out
+++ b/ql/src/test/results/clientpositive/tez/mapjoin_mapjoin.q.out
@@ -337,6 +337,7 @@ STAGE PLANS:
                       Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
@@ -407,6 +408,7 @@ STAGE PLANS:
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE


[03/18] hive git commit: HIVE-12994: Implement support for NULLS FIRST/NULLS LAST (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/tez/windowing_gby.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/windowing_gby.q.out b/ql/src/test/results/clientpositive/tez/windowing_gby.q.out
index 2af3e58..d058d7a 100644
--- a/ql/src/test/results/clientpositive/tez/windowing_gby.q.out
+++ b/ql/src/test/results/clientpositive/tez/windowing_gby.q.out
@@ -26,7 +26,7 @@ Stage-0
         Select Operator [SEL_17] (rows=6758 width=215)
           Output:["_col0"]
           PTF Operator [PTF_16] (rows=6758 width=215)
-            Function definitions:[{},{"name:":"windowingtablefunction","order by:":"(UDFToDouble(_col1) / UDFToDouble(_col2))","partition by:":"0"}]
+            Function definitions:[{},{"name:":"windowingtablefunction","order by:":"(UDFToDouble(_col1) / UDFToDouble(_col2)) ASC NULLS FIRST","partition by:":"0"}]
             Select Operator [SEL_15] (rows=6758 width=215)
               Output:["_col1","_col2"]
             <-Reducer 3 [SIMPLE_EDGE]

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/transform_ppr1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/transform_ppr1.q.out b/ql/src/test/results/clientpositive/transform_ppr1.q.out
index a7032cb..a5c5dbe 100644
--- a/ql/src/test/results/clientpositive/transform_ppr1.q.out
+++ b/ql/src/test/results/clientpositive/transform_ppr1.q.out
@@ -130,6 +130,7 @@ STAGE PLANS:
                   Statistics: Num rows: 333 Data size: 3537 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col1 (type: string)
+                    null sort order: a
                     sort order: +
                     Map-reduce partition columns: _col1 (type: string)
                     Statistics: Num rows: 333 Data size: 3537 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/transform_ppr2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/transform_ppr2.q.out b/ql/src/test/results/clientpositive/transform_ppr2.q.out
index 156fcc9..5224787 100644
--- a/ql/src/test/results/clientpositive/transform_ppr2.q.out
+++ b/ql/src/test/results/clientpositive/transform_ppr2.q.out
@@ -132,6 +132,7 @@ STAGE PLANS:
                   Statistics: Num rows: 333 Data size: 3537 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col1 (type: string)
+                    null sort order: a
                     sort order: +
                     Map-reduce partition columns: _col1 (type: string)
                     Statistics: Num rows: 333 Data size: 3537 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/udf_explode.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/udf_explode.q.out b/ql/src/test/results/clientpositive/udf_explode.q.out
index b12c280..91b9bbe 100644
--- a/ql/src/test/results/clientpositive/udf_explode.q.out
+++ b/ql/src/test/results/clientpositive/udf_explode.q.out
@@ -217,6 +217,7 @@ STAGE PLANS:
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: int)
+                    null sort order: a
                     sort order: +
                     Map-reduce partition columns: _col0 (type: int)
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -568,6 +569,7 @@ STAGE PLANS:
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: int), _col1 (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: _col0 (type: int), _col1 (type: string)
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/udtf_explode.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/udtf_explode.q.out b/ql/src/test/results/clientpositive/udtf_explode.q.out
index c369225..62d8e9f 100644
--- a/ql/src/test/results/clientpositive/udtf_explode.q.out
+++ b/ql/src/test/results/clientpositive/udtf_explode.q.out
@@ -213,6 +213,7 @@ STAGE PLANS:
                   Number of rows: 3
                   Statistics: Num rows: 3 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
+                    null sort order: 
                     sort order: 
                     Statistics: Num rows: 3 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE
                     tag: -1
@@ -307,6 +308,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col0 (type: int)
+              null sort order: a
               sort order: +
               Map-reduce partition columns: _col0 (type: int)
               Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
@@ -536,6 +538,7 @@ STAGE PLANS:
                   Number of rows: 3
                   Statistics: Num rows: 3 Data size: 1557 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
+                    null sort order: 
                     sort order: 
                     Statistics: Num rows: 3 Data size: 1557 Basic stats: COMPLETE Column stats: COMPLETE
                     tag: -1
@@ -630,6 +633,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col0 (type: int), _col1 (type: string)
+              null sort order: aa
               sort order: ++
               Map-reduce partition columns: _col0 (type: int), _col1 (type: string)
               Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/union22.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/union22.q.out b/ql/src/test/results/clientpositive/union22.q.out
index 83ec81b..5309c71 100644
--- a/ql/src/test/results/clientpositive/union22.q.out
+++ b/ql/src/test/results/clientpositive/union22.q.out
@@ -653,6 +653,7 @@ STAGE PLANS:
                 Statistics: Num rows: 166 Data size: 3693 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 166 Data size: 3693 Basic stats: COMPLETE Column stats: NONE
@@ -673,6 +674,7 @@ STAGE PLANS:
                 Statistics: Num rows: 55 Data size: 1862 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 55 Data size: 1862 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/union24.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/union24.q.out b/ql/src/test/results/clientpositive/union24.q.out
index 246581c..9147d4f 100644
--- a/ql/src/test/results/clientpositive/union24.q.out
+++ b/ql/src/test/results/clientpositive/union24.q.out
@@ -208,6 +208,7 @@ STAGE PLANS:
                   Statistics: Num rows: 103 Data size: 494 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col0 (type: string)
+                    null sort order: a
                     sort order: +
                     Map-reduce partition columns: _col0 (type: string)
                     Statistics: Num rows: 103 Data size: 494 Basic stats: COMPLETE Column stats: NONE
@@ -799,6 +800,7 @@ STAGE PLANS:
                 Statistics: Num rows: 103 Data size: 494 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 103 Data size: 494 Basic stats: COMPLETE Column stats: NONE
@@ -818,6 +820,7 @@ STAGE PLANS:
                 Statistics: Num rows: 103 Data size: 494 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 103 Data size: 494 Basic stats: COMPLETE Column stats: NONE
@@ -1374,6 +1377,7 @@ STAGE PLANS:
                 Statistics: Num rows: 103 Data size: 494 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 103 Data size: 494 Basic stats: COMPLETE Column stats: NONE
@@ -1393,6 +1397,7 @@ STAGE PLANS:
                 Statistics: Num rows: 103 Data size: 494 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 103 Data size: 494 Basic stats: COMPLETE Column stats: NONE
@@ -1533,6 +1538,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col0 (type: string)
+              null sort order: a
               sort order: +
               Map-reduce partition columns: _col0 (type: string)
               Statistics: Num rows: 113 Data size: 543 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/unionDistinct_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/unionDistinct_1.q.out b/ql/src/test/results/clientpositive/unionDistinct_1.q.out
index a0b8a6e..54fd1e5 100644
--- a/ql/src/test/results/clientpositive/unionDistinct_1.q.out
+++ b/ql/src/test/results/clientpositive/unionDistinct_1.q.out
@@ -7227,6 +7227,7 @@ STAGE PLANS:
                     Statistics: Num rows: 348 Data size: 9684 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string)
+                      null sort order: aaaa
                       sort order: ++++
                       Map-reduce partition columns: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string)
                       Statistics: Num rows: 348 Data size: 9684 Basic stats: COMPLETE Column stats: NONE
@@ -7243,6 +7244,7 @@ STAGE PLANS:
                 Statistics: Num rows: 348 Data size: 9684 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string)
+                  null sort order: aaaa
                   sort order: ++++
                   Map-reduce partition columns: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string)
                   Statistics: Num rows: 348 Data size: 9684 Basic stats: COMPLETE Column stats: NONE
@@ -7403,6 +7405,7 @@ STAGE PLANS:
                 Statistics: Num rows: 166 Data size: 3693 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 166 Data size: 3693 Basic stats: COMPLETE Column stats: NONE
@@ -7423,6 +7426,7 @@ STAGE PLANS:
                 Statistics: Num rows: 55 Data size: 1862 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 55 Data size: 1862 Basic stats: COMPLETE Column stats: NONE
@@ -8628,6 +8632,7 @@ STAGE PLANS:
                     Statistics: Num rows: 206 Data size: 988 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: string), _col1 (type: bigint)
+                      null sort order: aa
                       sort order: ++
                       Map-reduce partition columns: _col0 (type: string), _col1 (type: bigint)
                       Statistics: Num rows: 206 Data size: 988 Basic stats: COMPLETE Column stats: NONE
@@ -8654,6 +8659,7 @@ STAGE PLANS:
                     Statistics: Num rows: 206 Data size: 988 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: string), _col1 (type: bigint)
+                      null sort order: aa
                       sort order: ++
                       Map-reduce partition columns: _col0 (type: string), _col1 (type: bigint)
                       Statistics: Num rows: 206 Data size: 988 Basic stats: COMPLETE Column stats: NONE
@@ -8792,6 +8798,7 @@ STAGE PLANS:
                 Statistics: Num rows: 206 Data size: 988 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string), _col1 (type: bigint)
+                  null sort order: aa
                   sort order: ++
                   Map-reduce partition columns: _col0 (type: string), _col1 (type: bigint)
                   Statistics: Num rows: 206 Data size: 988 Basic stats: COMPLETE Column stats: NONE
@@ -8818,6 +8825,7 @@ STAGE PLANS:
                     Statistics: Num rows: 206 Data size: 988 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: string), _col1 (type: bigint)
+                      null sort order: aa
                       sort order: ++
                       Map-reduce partition columns: _col0 (type: string), _col1 (type: bigint)
                       Statistics: Num rows: 206 Data size: 988 Basic stats: COMPLETE Column stats: NONE
@@ -8932,6 +8940,7 @@ STAGE PLANS:
                 Statistics: Num rows: 154 Data size: 738 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string), _col1 (type: bigint)
+                  null sort order: aa
                   sort order: ++
                   Map-reduce partition columns: _col0 (type: string), _col1 (type: bigint)
                   Statistics: Num rows: 154 Data size: 738 Basic stats: COMPLETE Column stats: NONE
@@ -8948,6 +8957,7 @@ STAGE PLANS:
                 Statistics: Num rows: 154 Data size: 738 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string), _col1 (type: bigint)
+                  null sort order: aa
                   sort order: ++
                   Map-reduce partition columns: _col0 (type: string), _col1 (type: bigint)
                   Statistics: Num rows: 154 Data size: 738 Basic stats: COMPLETE Column stats: NONE
@@ -9051,6 +9061,7 @@ STAGE PLANS:
                   Statistics: Num rows: 103 Data size: 494 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col0 (type: string)
+                    null sort order: a
                     sort order: +
                     Map-reduce partition columns: _col0 (type: string)
                     Statistics: Num rows: 103 Data size: 494 Basic stats: COMPLETE Column stats: NONE
@@ -9358,6 +9369,7 @@ STAGE PLANS:
                     Statistics: Num rows: 206 Data size: 988 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: string), _col1 (type: bigint)
+                      null sort order: aa
                       sort order: ++
                       Map-reduce partition columns: _col0 (type: string), _col1 (type: bigint)
                       Statistics: Num rows: 206 Data size: 988 Basic stats: COMPLETE Column stats: NONE
@@ -9384,6 +9396,7 @@ STAGE PLANS:
                     Statistics: Num rows: 206 Data size: 988 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: string), _col1 (type: bigint)
+                      null sort order: aa
                       sort order: ++
                       Map-reduce partition columns: _col0 (type: string), _col1 (type: bigint)
                       Statistics: Num rows: 206 Data size: 988 Basic stats: COMPLETE Column stats: NONE
@@ -9548,6 +9561,7 @@ STAGE PLANS:
                 Statistics: Num rows: 216 Data size: 1037 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string), _col1 (type: bigint)
+                  null sort order: aa
                   sort order: ++
                   Map-reduce partition columns: _col0 (type: string), _col1 (type: bigint)
                   Statistics: Num rows: 216 Data size: 1037 Basic stats: COMPLETE Column stats: NONE
@@ -9587,6 +9601,7 @@ STAGE PLANS:
                         Statistics: Num rows: 216 Data size: 1037 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
                           key expressions: _col0 (type: string), _col1 (type: bigint)
+                          null sort order: aa
                           sort order: ++
                           Map-reduce partition columns: _col0 (type: string), _col1 (type: bigint)
                           Statistics: Num rows: 216 Data size: 1037 Basic stats: COMPLETE Column stats: NONE
@@ -9963,6 +9978,7 @@ STAGE PLANS:
                     Statistics: Num rows: 206 Data size: 988 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: string), _col1 (type: bigint)
+                      null sort order: aa
                       sort order: ++
                       Map-reduce partition columns: _col0 (type: string), _col1 (type: bigint)
                       Statistics: Num rows: 206 Data size: 988 Basic stats: COMPLETE Column stats: NONE
@@ -9989,6 +10005,7 @@ STAGE PLANS:
                     Statistics: Num rows: 206 Data size: 988 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: string), _col1 (type: bigint)
+                      null sort order: aa
                       sort order: ++
                       Map-reduce partition columns: _col0 (type: string), _col1 (type: bigint)
                       Statistics: Num rows: 206 Data size: 988 Basic stats: COMPLETE Column stats: NONE
@@ -10127,6 +10144,7 @@ STAGE PLANS:
                 Statistics: Num rows: 159 Data size: 763 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string), _col1 (type: bigint)
+                  null sort order: aa
                   sort order: ++
                   Map-reduce partition columns: _col0 (type: string), _col1 (type: bigint)
                   Statistics: Num rows: 159 Data size: 763 Basic stats: COMPLETE Column stats: NONE
@@ -10143,6 +10161,7 @@ STAGE PLANS:
                 Statistics: Num rows: 159 Data size: 763 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string), _col1 (type: bigint)
+                  null sort order: aa
                   sort order: ++
                   Map-reduce partition columns: _col0 (type: string), _col1 (type: bigint)
                   Statistics: Num rows: 159 Data size: 763 Basic stats: COMPLETE Column stats: NONE
@@ -10281,6 +10300,7 @@ STAGE PLANS:
                     Statistics: Num rows: 113 Data size: 543 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: string)
+                      null sort order: a
                       sort order: +
                       Map-reduce partition columns: _col0 (type: string)
                       Statistics: Num rows: 113 Data size: 543 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/union_ppr.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/union_ppr.q.out b/ql/src/test/results/clientpositive/union_ppr.q.out
index b1a3bcf..11c6dce 100644
--- a/ql/src/test/results/clientpositive/union_ppr.q.out
+++ b/ql/src/test/results/clientpositive/union_ppr.q.out
@@ -87,25 +87,29 @@ TOK_QUERY
             '2008-04-08'
       TOK_SORTBY
          TOK_TABSORTCOLNAMEASC
-            .
-               TOK_TABLE_OR_COL
-                  A
-               key
+            TOK_NULLS_FIRST
+               .
+                  TOK_TABLE_OR_COL
+                     A
+                  key
          TOK_TABSORTCOLNAMEASC
-            .
-               TOK_TABLE_OR_COL
-                  A
-               value
+            TOK_NULLS_FIRST
+               .
+                  TOK_TABLE_OR_COL
+                     A
+                  value
          TOK_TABSORTCOLNAMEASC
-            .
-               TOK_TABLE_OR_COL
-                  A
-               ds
+            TOK_NULLS_FIRST
+               .
+                  TOK_TABLE_OR_COL
+                     A
+                  ds
          TOK_TABSORTCOLNAMEASC
-            .
-               TOK_TABLE_OR_COL
-                  A
-               hr
+            TOK_NULLS_FIRST
+               .
+                  TOK_TABLE_OR_COL
+                     A
+                  hr
 
 
 STAGE DEPENDENCIES:
@@ -136,6 +140,7 @@ STAGE PLANS:
                     Statistics: Num rows: 666 Data size: 7074 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: string), _col1 (type: string), '2008-04-08' (type: string), _col3 (type: string)
+                      null sort order: aaaa
                       sort order: ++++
                       Statistics: Num rows: 666 Data size: 7074 Basic stats: COMPLETE Column stats: NONE
                       tag: -1
@@ -160,6 +165,7 @@ STAGE PLANS:
                     Statistics: Num rows: 666 Data size: 7074 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: string), _col1 (type: string), '2008-04-08' (type: string), _col3 (type: string)
+                      null sort order: aaaa
                       sort order: ++++
                       Statistics: Num rows: 666 Data size: 7074 Basic stats: COMPLETE Column stats: NONE
                       tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/union_remove_6_subq.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/union_remove_6_subq.q.out b/ql/src/test/results/clientpositive/union_remove_6_subq.q.out
index fe58686..59ccf49 100644
--- a/ql/src/test/results/clientpositive/union_remove_6_subq.q.out
+++ b/ql/src/test/results/clientpositive/union_remove_6_subq.q.out
@@ -536,7 +536,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col0
                   raw input shape:
                   window functions:


[06/18] hive git commit: HIVE-12994: Implement support for NULLS FIRST/NULLS LAST (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/tez/metadataonly1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/metadataonly1.q.out b/ql/src/test/results/clientpositive/tez/metadataonly1.q.out
index 3b9b51d..1961d1b 100644
--- a/ql/src/test/results/clientpositive/tez/metadataonly1.q.out
+++ b/ql/src/test/results/clientpositive/tez/metadataonly1.q.out
@@ -57,6 +57,7 @@ STAGE PLANS:
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
+                        null sort order: 
                         sort order: 
                         Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE
                         tag: -1
@@ -166,6 +167,7 @@ STAGE PLANS:
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
+                        null sort order: 
                         sort order: 
                         Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE
                         tag: -1
@@ -316,6 +318,7 @@ STAGE PLANS:
                       Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                         tag: -1
@@ -463,6 +466,7 @@ STAGE PLANS:
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
+                        null sort order: 
                         sort order: 
                         Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                         tag: -1
@@ -646,6 +650,7 @@ STAGE PLANS:
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: string)
+                      null sort order: a
                       sort order: +
                       Map-reduce partition columns: _col0 (type: string)
                       Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -753,6 +758,7 @@ STAGE PLANS:
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
+                        null sort order: 
                         sort order: 
                         Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE
                         tag: -1
@@ -861,6 +867,7 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
+                    null sort order: 
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                     tag: -1
@@ -910,6 +917,7 @@ STAGE PLANS:
                   Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col0 (type: string)
+                    null sort order: a
                     sort order: +
                     Map-reduce partition columns: _col0 (type: string)
                     Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE
@@ -1038,6 +1046,7 @@ STAGE PLANS:
                       Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string), _col1 (type: string)
+                        null sort order: aa
                         sort order: ++
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -1284,6 +1293,7 @@ STAGE PLANS:
                       Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -1523,6 +1533,7 @@ STAGE PLANS:
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
+                        null sort order: 
                         sort order: 
                         Statistics: Num rows: 1 Data size: 84 Basic stats: COMPLETE Column stats: NONE
                         tag: -1
@@ -1773,6 +1784,7 @@ STAGE PLANS:
                       Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string), _col1 (type: string)
+                        null sort order: aa
                         sort order: ++
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/tez/optimize_nullscan.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/optimize_nullscan.q.out b/ql/src/test/results/clientpositive/tez/optimize_nullscan.q.out
index 4b04177..336a5e9 100644
--- a/ql/src/test/results/clientpositive/tez/optimize_nullscan.q.out
+++ b/ql/src/test/results/clientpositive/tez/optimize_nullscan.q.out
@@ -122,6 +122,7 @@ STAGE PLANS:
                       Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -270,6 +271,7 @@ STAGE PLANS:
                       Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
@@ -337,6 +339,7 @@ STAGE PLANS:
                       Number of rows: 0
                       Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
                       Reduce Output Operator
+                        null sort order: 
                         sort order: 
                         Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
                         tag: -1
@@ -579,6 +582,7 @@ STAGE PLANS:
                   Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col0 (type: string)
+                    null sort order: a
                     sort order: +
                     Map-reduce partition columns: _col0 (type: string)
                     Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
@@ -692,6 +696,7 @@ STAGE PLANS:
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
+                        null sort order: 
                         sort order: 
                         Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                         tag: -1
@@ -761,6 +766,7 @@ STAGE PLANS:
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
+                        null sort order: 
                         sort order: 
                         Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                         tag: -1
@@ -1130,6 +1136,7 @@ STAGE PLANS:
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
+                        null sort order: 
                         sort order: 
                         Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
                         tag: 0
@@ -1197,6 +1204,7 @@ STAGE PLANS:
                       Number of rows: 0
                       Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
                       Reduce Output Operator
+                        null sort order: 
                         sort order: 
                         Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
                         tag: -1
@@ -1438,6 +1446,7 @@ STAGE PLANS:
                   Number of rows: 0
                   Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
                   Reduce Output Operator
+                    null sort order: 
                     sort order: 
                     Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
                     tag: 1
@@ -1644,6 +1653,7 @@ STAGE PLANS:
                     Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: key (type: string)
+                      null sort order: a
                       sort order: +
                       Map-reduce partition columns: key (type: string)
                       Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
@@ -1709,6 +1719,7 @@ STAGE PLANS:
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Reduce Output Operator
                       key expressions: key (type: string)
+                      null sort order: a
                       sort order: +
                       Map-reduce partition columns: key (type: string)
                       Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -1831,6 +1842,7 @@ STAGE PLANS:
                     Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: value (type: string)
+                      null sort order: a
                       sort order: +
                       Map-reduce partition columns: value (type: string)
                       Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
@@ -1897,6 +1909,7 @@ STAGE PLANS:
                     Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: value (type: string)
+                      null sort order: a
                       sort order: +
                       Map-reduce partition columns: value (type: string)
                       Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
@@ -2060,6 +2073,7 @@ STAGE PLANS:
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
+                        null sort order: 
                         sort order: 
                         Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                         tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/tez/orc_analyze.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/orc_analyze.q.out b/ql/src/test/results/clientpositive/tez/orc_analyze.q.out
index 87855fa..9b7e7b7 100644
--- a/ql/src/test/results/clientpositive/tez/orc_analyze.q.out
+++ b/ql/src/test/results/clientpositive/tez/orc_analyze.q.out
@@ -917,7 +917,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	4                   	 
 Bucket Columns:     	[first_name]        	 
-Sort Columns:       	[Order(col:last_name, order:1)]	 
+Sort Columns:       	[Order(col:last_name, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: desc formatted orc_create_people partition(state="Or")
@@ -960,7 +960,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	4                   	 
 Bucket Columns:     	[first_name]        	 
-Sort Columns:       	[Order(col:last_name, order:1)]	 
+Sort Columns:       	[Order(col:last_name, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: analyze table orc_create_people partition(state) compute statistics partialscan
@@ -1015,7 +1015,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	4                   	 
 Bucket Columns:     	[first_name]        	 
-Sort Columns:       	[Order(col:last_name, order:1)]	 
+Sort Columns:       	[Order(col:last_name, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: desc formatted orc_create_people partition(state="Or")
@@ -1058,7 +1058,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	4                   	 
 Bucket Columns:     	[first_name]        	 
-Sort Columns:       	[Order(col:last_name, order:1)]	 
+Sort Columns:       	[Order(col:last_name, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: analyze table orc_create_people partition(state) compute statistics noscan
@@ -1113,7 +1113,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	4                   	 
 Bucket Columns:     	[first_name]        	 
-Sort Columns:       	[Order(col:last_name, order:1)]	 
+Sort Columns:       	[Order(col:last_name, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: desc formatted orc_create_people partition(state="Or")
@@ -1156,7 +1156,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	4                   	 
 Bucket Columns:     	[first_name]        	 
-Sort Columns:       	[Order(col:last_name, order:1)]	 
+Sort Columns:       	[Order(col:last_name, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: drop table orc_create_people
@@ -1262,7 +1262,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	4                   	 
 Bucket Columns:     	[first_name]        	 
-Sort Columns:       	[Order(col:last_name, order:1)]	 
+Sort Columns:       	[Order(col:last_name, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: desc formatted orc_create_people partition(state="Or")
@@ -1305,7 +1305,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	4                   	 
 Bucket Columns:     	[first_name]        	 
-Sort Columns:       	[Order(col:last_name, order:1)]	 
+Sort Columns:       	[Order(col:last_name, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: drop table orc_create_people

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/tez/order_null.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/order_null.q.out b/ql/src/test/results/clientpositive/tez/order_null.q.out
new file mode 100644
index 0000000..252e348
--- /dev/null
+++ b/ql/src/test/results/clientpositive/tez/order_null.q.out
@@ -0,0 +1,222 @@
+PREHOOK: query: create table src_null (a int, b string)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@src_null
+POSTHOOK: query: create table src_null (a int, b string)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@src_null
+PREHOOK: query: insert into src_null values (1, 'A')
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__1
+PREHOOK: Output: default@src_null
+POSTHOOK: query: insert into src_null values (1, 'A')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__1
+POSTHOOK: Output: default@src_null
+POSTHOOK: Lineage: src_null.a EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+POSTHOOK: Lineage: src_null.b SIMPLE [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+PREHOOK: query: insert into src_null values (null, null)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__2
+PREHOOK: Output: default@src_null
+POSTHOOK: query: insert into src_null values (null, null)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__2
+POSTHOOK: Output: default@src_null
+POSTHOOK: Lineage: src_null.a EXPRESSION [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+POSTHOOK: Lineage: src_null.b SIMPLE [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+PREHOOK: query: insert into src_null values (3, null)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__3
+PREHOOK: Output: default@src_null
+POSTHOOK: query: insert into src_null values (3, null)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__3
+POSTHOOK: Output: default@src_null
+POSTHOOK: Lineage: src_null.a EXPRESSION [(values__tmp__table__3)values__tmp__table__3.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+POSTHOOK: Lineage: src_null.b SIMPLE [(values__tmp__table__3)values__tmp__table__3.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+PREHOOK: query: insert into src_null values (2, null)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__4
+PREHOOK: Output: default@src_null
+POSTHOOK: query: insert into src_null values (2, null)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__4
+POSTHOOK: Output: default@src_null
+POSTHOOK: Lineage: src_null.a EXPRESSION [(values__tmp__table__4)values__tmp__table__4.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+POSTHOOK: Lineage: src_null.b SIMPLE [(values__tmp__table__4)values__tmp__table__4.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+PREHOOK: query: insert into src_null values (2, 'A')
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__5
+PREHOOK: Output: default@src_null
+POSTHOOK: query: insert into src_null values (2, 'A')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__5
+POSTHOOK: Output: default@src_null
+POSTHOOK: Lineage: src_null.a EXPRESSION [(values__tmp__table__5)values__tmp__table__5.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+POSTHOOK: Lineage: src_null.b SIMPLE [(values__tmp__table__5)values__tmp__table__5.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+PREHOOK: query: insert into src_null values (2, 'B')
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__6
+PREHOOK: Output: default@src_null
+POSTHOOK: query: insert into src_null values (2, 'B')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__6
+POSTHOOK: Output: default@src_null
+POSTHOOK: Lineage: src_null.a EXPRESSION [(values__tmp__table__6)values__tmp__table__6.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+POSTHOOK: Lineage: src_null.b SIMPLE [(values__tmp__table__6)values__tmp__table__6.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+PREHOOK: query: SELECT x.* FROM src_null x ORDER BY a asc
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src_null
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT x.* FROM src_null x ORDER BY a asc
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src_null
+#### A masked pattern was here ####
+NULL	NULL
+1	A
+2	NULL
+2	A
+2	B
+3	NULL
+PREHOOK: query: SELECT x.* FROM src_null x ORDER BY a desc
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src_null
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT x.* FROM src_null x ORDER BY a desc
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src_null
+#### A masked pattern was here ####
+3	NULL
+2	NULL
+2	A
+2	B
+1	A
+NULL	NULL
+PREHOOK: query: SELECT x.* FROM src_null x ORDER BY b asc, a asc nulls last
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src_null
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT x.* FROM src_null x ORDER BY b asc, a asc nulls last
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src_null
+#### A masked pattern was here ####
+2	NULL
+3	NULL
+NULL	NULL
+1	A
+2	A
+2	B
+PREHOOK: query: SELECT x.* FROM src_null x ORDER BY b desc, a asc
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src_null
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT x.* FROM src_null x ORDER BY b desc, a asc
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src_null
+#### A masked pattern was here ####
+2	B
+1	A
+2	A
+NULL	NULL
+2	NULL
+3	NULL
+PREHOOK: query: SELECT x.* FROM src_null x ORDER BY a asc nulls first
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src_null
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT x.* FROM src_null x ORDER BY a asc nulls first
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src_null
+#### A masked pattern was here ####
+NULL	NULL
+1	A
+2	NULL
+2	A
+2	B
+3	NULL
+PREHOOK: query: SELECT x.* FROM src_null x ORDER BY a desc nulls first
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src_null
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT x.* FROM src_null x ORDER BY a desc nulls first
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src_null
+#### A masked pattern was here ####
+NULL	NULL
+3	NULL
+2	NULL
+2	A
+2	B
+1	A
+PREHOOK: query: SELECT x.* FROM src_null x ORDER BY b asc nulls last, a
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src_null
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT x.* FROM src_null x ORDER BY b asc nulls last, a
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src_null
+#### A masked pattern was here ####
+1	A
+2	A
+2	B
+NULL	NULL
+2	NULL
+3	NULL
+PREHOOK: query: SELECT x.* FROM src_null x ORDER BY b desc nulls last, a
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src_null
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT x.* FROM src_null x ORDER BY b desc nulls last, a
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src_null
+#### A masked pattern was here ####
+2	B
+1	A
+2	A
+NULL	NULL
+2	NULL
+3	NULL
+PREHOOK: query: SELECT x.* FROM src_null x ORDER BY a asc nulls last, b desc
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src_null
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT x.* FROM src_null x ORDER BY a asc nulls last, b desc
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src_null
+#### A masked pattern was here ####
+1	A
+2	B
+2	A
+2	NULL
+3	NULL
+NULL	NULL
+PREHOOK: query: SELECT x.* FROM src_null x ORDER BY b desc nulls last, a desc nulls last
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src_null
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT x.* FROM src_null x ORDER BY b desc nulls last, a desc nulls last
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src_null
+#### A masked pattern was here ####
+2	B
+2	A
+1	A
+3	NULL
+2	NULL
+NULL	NULL
+PREHOOK: query: SELECT x.* FROM src_null x ORDER BY b asc nulls first, a asc nulls last
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src_null
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT x.* FROM src_null x ORDER BY b asc nulls first, a asc nulls last
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src_null
+#### A masked pattern was here ####
+2	NULL
+3	NULL
+NULL	NULL
+1	A
+2	A
+2	B

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/tez/ptf.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/ptf.q.out b/ql/src/test/results/clientpositive/tez/ptf.q.out
index 8685bdb..b587f8b 100644
--- a/ql/src/test/results/clientpositive/tez/ptf.q.out
+++ b/ql/src/test/results/clientpositive/tez/ptf.q.out
@@ -63,7 +63,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -89,7 +89,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -268,7 +268,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
@@ -294,7 +294,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -420,7 +420,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
@@ -544,7 +544,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: abc
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -570,7 +570,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -722,7 +722,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
@@ -748,7 +748,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -903,7 +903,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
@@ -938,7 +938,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col0
                         raw input shape:
                         window functions:
@@ -1099,7 +1099,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: abc
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col0: int, _col1: string, _col2: string, _col3: string, _col4: string, _col5: int, _col6: string, _col7: double, _col8: string
                         partition by: _col2
                         raw input shape:
@@ -1270,7 +1270,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: abc
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col0: int, _col1: string, _col2: string, _col3: string, _col4: string, _col5: int, _col6: string, _col7: double, _col8: string
                         partition by: _col2
                         raw input shape:
@@ -1378,7 +1378,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_1
                           name: noopwithmap
-                          order by: p_name, p_size(DESC)
+                          order by: p_name ASC NULLS FIRST, p_size DESC NULLS LAST
                           output shape: p_name: string, p_mfgr: string, p_size: int
                           partition by: p_mfgr
                           raw input shape:
@@ -1405,7 +1405,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopwithmap
-                        order by: _col1, _col5(DESC)
+                        order by: _col1 ASC NULLS FIRST, _col5 DESC NULLS LAST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
@@ -1431,7 +1431,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1, _col5(DESC)
+                        order by: _col1 ASC NULLS FIRST, _col5 DESC NULLS LAST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -1550,7 +1550,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_1
                           name: noopwithmap
-                          order by: p_name
+                          order by: p_name ASC NULLS FIRST
                           output shape: p_name: string, p_mfgr: string, p_size: int, p_retailprice: double
                           partition by: p_mfgr
                           raw input shape:
@@ -1578,7 +1578,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopwithmap
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -1605,7 +1605,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -1753,7 +1753,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -1779,7 +1779,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -1930,7 +1930,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col2(DESC), _col1
+                        order by: _col2 DESC NULLS LAST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -1944,7 +1944,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_1
                           name: noopwithmap
-                          order by: _col2(DESC), _col1
+                          order by: _col2 DESC NULLS LAST, _col1 ASC NULLS FIRST
                           output shape: _col1: string, _col2: string, _col5: int, _col7: double
                           partition by: _col2
                           raw input shape:
@@ -1952,7 +1952,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_2
                           name: noop
-                          order by: _col2(DESC), _col1
+                          order by: _col2 DESC NULLS LAST, _col1 ASC NULLS FIRST
                           output shape: _col1: string, _col2: string, _col5: int, _col7: double
                           partition by: _col2
                           raw input shape:
@@ -1979,7 +1979,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopwithmap
-                        order by: _col2(DESC), _col1
+                        order by: _col2 DESC NULLS LAST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -1987,7 +1987,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_2
                         name: noop
-                        order by: _col2(DESC), _col1
+                        order by: _col2 DESC NULLS LAST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -2013,7 +2013,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -2171,7 +2171,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -2197,7 +2197,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -2365,7 +2365,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: abc
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col0: int, _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -2410,7 +2410,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -2573,7 +2573,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
@@ -2739,7 +2739,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col0
+                        order by: _col0 ASC NULLS FIRST
                         output shape: _col0: string, _col1: string, _col2: double
                         partition by: _col0
                         raw input shape:
@@ -2765,7 +2765,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col0
                         raw input shape:
                         window functions:
@@ -2953,7 +2953,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -2985,7 +2985,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -3037,7 +3037,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col5
+                        order by: _col5 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -3073,7 +3073,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col3, _col2
+                        order by: _col3 ASC NULLS FIRST, _col2 ASC NULLS FIRST
                         partition by: _col3
                         raw input shape:
                         window functions:
@@ -3341,14 +3341,14 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col2
+                        order by: _col2 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
                       Partition table definition
                         input alias: ptf_2
                         name: noop
-                        order by: _col2
+                        order by: _col2 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
@@ -3362,7 +3362,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_1
                           name: noopwithmap
-                          order by: _col2, _col1
+                          order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                           output shape: _col1: string, _col2: string, _col5: int
                           partition by: _col2, _col1
                           raw input shape:
@@ -3370,7 +3370,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_2
                           name: noop
-                          order by: _col2, _col1
+                          order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                           output shape: _col1: string, _col2: string, _col5: int
                           partition by: _col2, _col1
                           raw input shape:
@@ -3397,7 +3397,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopwithmap
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
@@ -3405,7 +3405,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_2
                         name: noop
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
@@ -3431,7 +3431,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         partition by: _col2, _col1
                         raw input shape:
                         window functions:
@@ -3613,14 +3613,14 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col2
+                        order by: _col2 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
                       Partition table definition
                         input alias: ptf_2
                         name: noop
-                        order by: _col2
+                        order by: _col2 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
@@ -3646,7 +3646,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
@@ -3672,7 +3672,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col2
+                        order by: _col2 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
@@ -3698,7 +3698,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -3875,14 +3875,14 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
                       Partition table definition
                         input alias: ptf_2
                         name: noop
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
@@ -3908,14 +3908,14 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col2
+                        order by: _col2 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
                       Partition table definition
                         input alias: ptf_2
                         name: noop
-                        order by: _col2
+                        order by: _col2 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
@@ -3941,7 +3941,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -4119,14 +4119,14 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
                       Partition table definition
                         input alias: ptf_2
                         name: noop
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
@@ -4152,7 +4152,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col2
+                        order by: _col2 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
@@ -4166,7 +4166,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_1
                           name: noopwithmap
-                          order by: _col2, _col1
+                          order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                           output shape: _col1: string, _col2: string, _col5: int
                           partition by: _col2, _col1
                           raw input shape:
@@ -4194,7 +4194,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopwithmap
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
@@ -4221,7 +4221,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         partition by: _col2, _col1
                         raw input shape:
                         window functions:
@@ -4400,7 +4400,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
@@ -4414,7 +4414,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_1
                           name: noopwithmap
-                          order by: _col2
+                          order by: _col2 ASC NULLS FIRST
                           output shape: _col1: string, _col2: string, _col5: int
                           partition by: _col2
                           raw input shape:
@@ -4422,7 +4422,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_2
                           name: noop
-                          order by: _col2
+                          order by: _col2 ASC NULLS FIRST
                           output shape: _col1: string, _col2: string, _col5: int
                           partition by: _col2
                           raw input shape:
@@ -4449,7 +4449,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopwithmap
-                        order by: _col2
+                        order by: _col2 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
@@ -4457,7 +4457,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_2
                         name: noop
-                        order by: _col2
+                        order by: _col2 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
@@ -4483,7 +4483,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         partition by: _col2, _col1
                         raw input shape:
                         window functions:
@@ -4656,14 +4656,14 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
                       Partition table definition
                         input alias: ptf_2
                         name: noop
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
@@ -4677,7 +4677,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_1
                           name: noopwithmap
-                          order by: _col2, _col1
+                          order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                           output shape: _col1: string, _col2: string, _col5: int
                           partition by: _col2, _col1
                           raw input shape:
@@ -4705,7 +4705,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopwithmap
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
@@ -4732,7 +4732,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/tez/ptf_matchpath.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/ptf_matchpath.q.out b/ql/src/test/results/clientpositive/tez/ptf_matchpath.q.out
index af88634..8ca3196 100644
--- a/ql/src/test/results/clientpositive/tez/ptf_matchpath.q.out
+++ b/ql/src/test/results/clientpositive/tez/ptf_matchpath.q.out
@@ -101,7 +101,7 @@ STAGE PLANS:
                         input alias: ptf_1
                         arguments: 'LATE.LATE+', 'LATE', (_col5 > 15.0), 'origin_city_name, fl_num, year, month, day_of_month, size(tpath) as sz, tpath[0].day_of_month as tpath'
                         name: matchpath
-                        order by: _col2, _col3, _col4
+                        order by: _col2 ASC NULLS FIRST, _col3 ASC NULLS FIRST, _col4 ASC NULLS FIRST
                         output shape: tpath: int
                         partition by: _col6
                         raw input shape:
@@ -228,7 +228,7 @@ STAGE PLANS:
                         input alias: ptf_1
                         arguments: 'LATE.LATE+', 'LATE', (_col5 > 15.0), 'origin_city_name, fl_num, year, month, day_of_month, size(tpath) as sz, tpath[0].day_of_month as tpath'
                         name: matchpath
-                        order by: _col6, _col2, _col3, _col4
+                        order by: _col6 ASC NULLS FIRST, _col2 ASC NULLS FIRST, _col3 ASC NULLS FIRST, _col4 ASC NULLS FIRST
                         output shape: tpath: int
                         partition by: 0
                         raw input shape:
@@ -352,7 +352,7 @@ STAGE PLANS:
                         input alias: ptf_1
                         arguments: 'LATE.LATE+', 'LATE', (_col5 > 15.0), 'origin_city_name, fl_num, year, month, day_of_month, size(tpath) as sz, tpath[0].day_of_month as tpath'
                         name: matchpath
-                        order by: _col6, _col2, _col3, _col4
+                        order by: _col6 ASC NULLS FIRST, _col2 ASC NULLS FIRST, _col3 ASC NULLS FIRST, _col4 ASC NULLS FIRST
                         output shape: tpath: int
                         partition by: 0
                         raw input shape:

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/tez/ptf_streaming.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/ptf_streaming.q.out b/ql/src/test/results/clientpositive/tez/ptf_streaming.q.out
index 57f3334..d8aa492 100644
--- a/ql/src/test/results/clientpositive/tez/ptf_streaming.q.out
+++ b/ql/src/test/results/clientpositive/tez/ptf_streaming.q.out
@@ -63,7 +63,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopstreaming
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -89,7 +89,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -268,7 +268,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopstreaming
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
@@ -294,7 +294,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -436,7 +436,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: abc
                         name: noopstreaming
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col0: int, _col1: string, _col2: string, _col3: string, _col4: string, _col5: int, _col6: string, _col7: double, _col8: string
                         partition by: _col2
                         raw input shape:
@@ -561,7 +561,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_1
                           name: noopwithmapstreaming
-                          order by: p_name, p_size(DESC)
+                          order by: p_name ASC NULLS FIRST, p_size DESC NULLS LAST
                           output shape: p_name: string, p_mfgr: string, p_size: int
                           partition by: p_mfgr
                           raw input shape:
@@ -588,7 +588,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopwithmapstreaming
-                        order by: _col1, _col5(DESC)
+                        order by: _col1 ASC NULLS FIRST, _col5 DESC NULLS LAST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
@@ -614,7 +614,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1, _col5(DESC)
+                        order by: _col1 ASC NULLS FIRST, _col5 DESC NULLS LAST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -733,7 +733,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_1
                           name: noopwithmapstreaming
-                          order by: p_name
+                          order by: p_name ASC NULLS FIRST
                           output shape: p_name: string, p_mfgr: string, p_size: int, p_retailprice: double
                           partition by: p_mfgr
                           raw input shape:
@@ -761,7 +761,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopwithmapstreaming
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -788,7 +788,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -939,7 +939,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopstreaming
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -953,7 +953,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_1
                           name: noopwithmapstreaming
-                          order by: _col2, _col1
+                          order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                           output shape: _col1: string, _col2: string, _col5: int, _col7: double
                           partition by: _col2
                           raw input shape:
@@ -961,7 +961,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_2
                           name: noopstreaming
-                          order by: _col2, _col1
+                          order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                           output shape: _col1: string, _col2: string, _col5: int, _col7: double
                           partition by: _col2
                           raw input shape:
@@ -988,7 +988,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopwithmapstreaming
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -996,7 +996,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_2
                         name: noopstreaming
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -1022,7 +1022,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -1175,7 +1175,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopstreaming
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -1189,7 +1189,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_1
                           name: noopwithmap
-                          order by: _col2, _col1
+                          order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                           output shape: _col1: string, _col2: string, _col5: int, _col7: double
                           partition by: _col2
                           raw input shape:
@@ -1197,7 +1197,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_2
                           name: noopstreaming
-                          order by: _col2, _col1
+                          order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                           output shape: _col1: string, _col2: string, _col5: int, _col7: double
                           partition by: _col2
                           raw input shape:
@@ -1224,7 +1224,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopwithmap
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -1232,7 +1232,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_2
                         name: noopstreaming
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -1258,7 +1258,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -1411,7 +1411,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopstreaming
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -1425,7 +1425,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_1
                           name: noopwithmapstreaming
-                          order by: _col2, _col1
+                          order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                           output shape: _col1: string, _col2: string, _col5: int, _col7: double
                           partition by: _col2
                           raw input shape:
@@ -1433,7 +1433,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_2
                           name: noop
-                          order by: _col2, _col1
+                          order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                           output shape: _col1: string, _col2: string, _col5: int, _col7: double
                           partition by: _col2
                           raw input shape:
@@ -1460,7 +1460,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopwithmapstreaming
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -1468,7 +1468,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_2
                         name: noop
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -1494,7 +1494,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -1664,7 +1664,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: abc
                         name: noopstreaming
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col0: int, _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -1709,7 +1709,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -1895,14 +1895,14 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopstreaming
-                        order by: _col2
+                        order by: _col2 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
                       Partition table definition
                         input alias: ptf_2
                         name: noop
-                        order by: _col2
+                        order by: _col2 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
@@ -1916,7 +1916,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_1
                           name: noopwithmap
-                          order by: _col2, _col1
+                          order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                           output shape: _col1: string, _col2: string, _col5: int
                           partition by: _col2, _col1
                           raw input shape:
@@ -1924,7 +1924,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_2
                           name: noopstreaming
-                          order by: _col2, _col1
+                          order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                           output shape: _col1: string, _col2: string, _col5: int
                           partition by: _col2, _col1
                           raw input shape:
@@ -1951,7 +1951,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopwithmap
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
@@ -1959,7 +1959,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_2
                         name: noopstreaming
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
@@ -1985,7 +1985,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         partition by: _col2, _col1
                         raw input shape:
                         window functions:
@@ -2167,14 +2167,14 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopstreaming
-                        order by: _col2
+                        order by: _col2 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
                       Partition table definition
                         input alias: ptf_2
                         name: noop
-                        order by: _col2
+                        order by: _col2 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
@@ -2200,7 +2200,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopstreaming
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
@@ -2226,7 +2226,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col2
+                        order by: _col2 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
@@ -2252,7 +2252,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -2427,14 +2427,14 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopstreaming
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
                       Partition table definition
                         input alias: ptf_2
                         name: noop
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
@@ -2448,7 +2448,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_1
                           name: noopwithmapstreaming
-                          order by: _col2, _col1
+                          order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                           output shape: _col1: string, _col2: string, _col5: int
                           partition by: _col2, _col1
                           raw input shape:
@@ -2476,7 +2476,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopwithmapstreaming
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
@@ -2503,7 +2503,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/tez/subquery_in.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/subquery_in.q.out b/ql/src/test/results/clientpositive/tez/subquery_in.q.out
index 517e322..cc60c53 100644
--- a/ql/src/test/results/clientpositive/tez/subquery_in.q.out
+++ b/ql/src/test/results/clientpositive/tez/subquery_in.q.out
@@ -334,7 +334,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col5
+                        order by: _col5 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -511,7 +511,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col5
+                        order by: _col5 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/tez/tez_join_result_complex.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/tez_join_result_complex.q.out b/ql/src/test/results/clientpositive/tez/tez_join_result_complex.q.out
index 3d64a9f..58c4c86 100644
--- a/ql/src/test/results/clientpositive/tez/tez_join_result_complex.q.out
+++ b/ql/src/test/results/clientpositive/tez/tez_join_result_complex.q.out
@@ -444,6 +444,7 @@ STAGE PLANS:
                       Statistics: Num rows: 2 Data size: 7501 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 2 Data size: 7501 Basic stats: COMPLETE Column stats: NONE
@@ -1664,6 +1665,7 @@ STAGE PLANS:
                       Statistics: Num rows: 2 Data size: 7501 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 2 Data size: 7501 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/tez/transform_ppr1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/transform_ppr1.q.out b/ql/src/test/results/clientpositive/tez/transform_ppr1.q.out
index 3ca5cf2..92d1f4c 100644
--- a/ql/src/test/results/clientpositive/tez/transform_ppr1.q.out
+++ b/ql/src/test/results/clientpositive/tez/transform_ppr1.q.out
@@ -136,6 +136,7 @@ STAGE PLANS:
                         Statistics: Num rows: 333 Data size: 3537 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
                           key expressions: _col1 (type: string)
+                          null sort order: a
                           sort order: +
                           Map-reduce partition columns: _col1 (type: string)
                           Statistics: Num rows: 333 Data size: 3537 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/tez/transform_ppr2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/transform_ppr2.q.out b/ql/src/test/results/clientpositive/tez/transform_ppr2.q.out
index ebb8a18..c131bbe 100644
--- a/ql/src/test/results/clientpositive/tez/transform_ppr2.q.out
+++ b/ql/src/test/results/clientpositive/tez/transform_ppr2.q.out
@@ -138,6 +138,7 @@ STAGE PLANS:
                         Statistics: Num rows: 333 Data size: 3537 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
                           key expressions: _col1 (type: string)
+                          null sort order: a
                           sort order: +
                           Map-reduce partition columns: _col1 (type: string)
                           Statistics: Num rows: 333 Data size: 3537 Basic stats: COMPLETE Column stats: NONE


[08/18] hive git commit: HIVE-12994: Implement support for NULLS FIRST/NULLS LAST (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/vectorized_ptf.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/vectorized_ptf.q.out b/ql/src/test/results/clientpositive/spark/vectorized_ptf.q.out
index 426527f..f0a4444 100644
--- a/ql/src/test/results/clientpositive/spark/vectorized_ptf.q.out
+++ b/ql/src/test/results/clientpositive/spark/vectorized_ptf.q.out
@@ -157,8 +157,9 @@ TOK_QUERY
                   p_mfgr
             TOK_ORDERBY
                TOK_TABSORTCOLNAMEASC
-                  TOK_TABLE_OR_COL
-                     p_name
+                  TOK_NULLS_FIRST
+                     TOK_TABLE_OR_COL
+                        p_name
    TOK_INSERT
       TOK_DESTINATION
          TOK_DIR
@@ -183,8 +184,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             r
          TOK_SELEXPR
             TOK_FUNCTION
@@ -196,8 +198,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             dr
          TOK_SELEXPR
             TOK_FUNCTION
@@ -211,8 +214,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
                   TOK_WINDOWRANGE
                      preceding
                         unbounded
@@ -240,6 +244,7 @@ STAGE PLANS:
                   GatherStats: false
                   Reduce Output Operator
                     key expressions: p_mfgr (type: string), p_name (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: p_mfgr (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -312,13 +317,14 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
                   Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col2 (type: string), _col1 (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: _col2 (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -341,7 +347,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -510,10 +516,11 @@ TOK_QUERY
                   p_mfgr
             TOK_SORTBY
                TOK_TABSORTCOLNAMEASC
-                  .
-                     TOK_TABLE_OR_COL
-                        j
-                     p_name
+                  TOK_NULLS_FIRST
+                     .
+                        TOK_TABLE_OR_COL
+                           j
+                        p_name
    TOK_INSERT
       TOK_DESTINATION
          TOK_DIR
@@ -546,8 +553,9 @@ TOK_QUERY
                               p_mfgr
                         TOK_ORDERBY
                            TOK_TABSORTCOLNAMEASC
-                              TOK_TABLE_OR_COL
-                                 p_name
+                              TOK_NULLS_FIRST
+                                 TOK_TABLE_OR_COL
+                                    p_name
             deltaSz
 
 
@@ -576,6 +584,7 @@ STAGE PLANS:
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: p_partkey (type: int)
+                      null sort order: a
                       sort order: +
                       Map-reduce partition columns: p_partkey (type: int)
                       Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -644,6 +653,7 @@ STAGE PLANS:
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: p_partkey (type: int)
+                      null sort order: a
                       sort order: +
                       Map-reduce partition columns: p_partkey (type: int)
                       Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -712,6 +722,7 @@ STAGE PLANS:
                 Statistics: Num rows: 28 Data size: 17646 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col2 (type: string), _col1 (type: string)
+                  null sort order: aa
                   sort order: ++
                   Map-reduce partition columns: _col2 (type: string)
                   Statistics: Num rows: 28 Data size: 17646 Basic stats: COMPLETE Column stats: NONE
@@ -734,13 +745,14 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
                   Statistics: Num rows: 28 Data size: 17646 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col2 (type: string), _col1 (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: _col2 (type: string)
                     Statistics: Num rows: 28 Data size: 17646 Basic stats: COMPLETE Column stats: NONE
@@ -763,7 +775,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -883,8 +895,9 @@ TOK_QUERY
                   p_mfgr
             TOK_ORDERBY
                TOK_TABSORTCOLNAMEASC
-                  TOK_TABLE_OR_COL
-                     p_name
+                  TOK_NULLS_FIRST
+                     TOK_TABLE_OR_COL
+                        p_name
    TOK_INSERT
       TOK_DESTINATION
          TOK_DIR
@@ -920,6 +933,7 @@ STAGE PLANS:
                   GatherStats: false
                   Reduce Output Operator
                     key expressions: p_mfgr (type: string), p_name (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: p_mfgr (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -992,7 +1006,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
@@ -1110,8 +1124,9 @@ TOK_QUERY
                   p_mfgr
             TOK_ORDERBY
                TOK_TABSORTCOLNAMEASC
-                  TOK_TABLE_OR_COL
-                     p_name
+                  TOK_NULLS_FIRST
+                     TOK_TABLE_OR_COL
+                        p_name
    TOK_INSERT
       TOK_DESTINATION
          TOK_DIR
@@ -1136,8 +1151,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             r
          TOK_SELEXPR
             TOK_FUNCTION
@@ -1149,8 +1165,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             dr
          TOK_SELEXPR
             TOK_FUNCTION
@@ -1164,8 +1181,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
                   TOK_WINDOWRANGE
                      preceding
                         unbounded
@@ -1193,6 +1211,7 @@ STAGE PLANS:
                   GatherStats: false
                   Reduce Output Operator
                     key expressions: p_mfgr (type: string), p_name (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: p_mfgr (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -1265,13 +1284,14 @@ STAGE PLANS:
                       Partition table definition
                         input alias: abc
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
                   Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col2 (type: string), _col1 (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: _col2 (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -1294,7 +1314,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -1439,8 +1459,9 @@ TOK_QUERY
                   p_mfgr
             TOK_ORDERBY
                TOK_TABSORTCOLNAMEASC
-                  TOK_TABLE_OR_COL
-                     p_name
+                  TOK_NULLS_FIRST
+                     TOK_TABLE_OR_COL
+                        p_name
    TOK_INSERT
       TOK_DESTINATION
          TOK_DIR
@@ -1465,8 +1486,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             r
          TOK_SELEXPR
             TOK_FUNCTION
@@ -1478,8 +1500,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             dr
          TOK_SELEXPR
             TOK_TABLE_OR_COL
@@ -1502,8 +1525,9 @@ TOK_QUERY
                               p_mfgr
                         TOK_ORDERBY
                            TOK_TABSORTCOLNAMEASC
-                              TOK_TABLE_OR_COL
-                                 p_name
+                              TOK_NULLS_FIRST
+                                 TOK_TABLE_OR_COL
+                                    p_name
             deltaSz
 
 
@@ -1527,6 +1551,7 @@ STAGE PLANS:
                   GatherStats: false
                   Reduce Output Operator
                     key expressions: p_mfgr (type: string), p_name (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: p_mfgr (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -1599,13 +1624,14 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
                   Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col2 (type: string), _col1 (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: _col2 (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -1628,7 +1654,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -1776,8 +1802,9 @@ TOK_QUERY
                   p_mfgr
             TOK_ORDERBY
                TOK_TABSORTCOLNAMEASC
-                  TOK_TABLE_OR_COL
-                     p_name
+                  TOK_NULLS_FIRST
+                     TOK_TABLE_OR_COL
+                        p_name
    TOK_INSERT
       TOK_DESTINATION
          TOK_DIR
@@ -1802,8 +1829,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             r
          TOK_SELEXPR
             TOK_FUNCTION
@@ -1815,8 +1843,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             dr
          TOK_SELEXPR
             TOK_TABLE_OR_COL
@@ -1839,8 +1868,9 @@ TOK_QUERY
                               p_mfgr
                         TOK_ORDERBY
                            TOK_TABSORTCOLNAMEASC
-                              TOK_TABLE_OR_COL
-                                 p_name
+                              TOK_NULLS_FIRST
+                                 TOK_TABLE_OR_COL
+                                    p_name
             deltaSz
       TOK_GROUPBY
          TOK_TABLE_OR_COL
@@ -1871,6 +1901,7 @@ STAGE PLANS:
                   GatherStats: false
                   Reduce Output Operator
                     key expressions: p_mfgr (type: string), p_name (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: p_mfgr (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -1943,7 +1974,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
@@ -1959,6 +1990,7 @@ STAGE PLANS:
                       Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: int)
+                        null sort order: aaa
                         sort order: +++
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -1981,7 +2013,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col0
                         raw input shape:
                         window functions:
@@ -2124,8 +2156,9 @@ TOK_QUERY
                      p_mfgr
                TOK_ORDERBY
                   TOK_TABSORTCOLNAMEASC
-                     TOK_TABLE_OR_COL
-                        p_name
+                     TOK_NULLS_FIRST
+                        TOK_TABLE_OR_COL
+                           p_name
          TOK_TABREF
             TOK_TABNAME
                part_orc
@@ -2170,6 +2203,7 @@ STAGE PLANS:
                   GatherStats: false
                   Reduce Output Operator
                     key expressions: p_mfgr (type: string), p_name (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: p_mfgr (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -2238,6 +2272,7 @@ STAGE PLANS:
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: p_partkey (type: int)
+                      null sort order: a
                       sort order: +
                       Map-reduce partition columns: p_partkey (type: int)
                       Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -2309,7 +2344,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: abc
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col0: int, _col1: string, _col2: string, _col3: string, _col4: string, _col5: int, _col6: string, _col7: double, _col8: string
                         partition by: _col2
                         raw input shape:
@@ -2320,6 +2355,7 @@ STAGE PLANS:
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: int)
+                      null sort order: a
                       sort order: +
                       Map-reduce partition columns: _col0 (type: int)
                       Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -2449,8 +2485,9 @@ TOK_QUERY
                      p_mfgr
                TOK_ORDERBY
                   TOK_TABSORTCOLNAMEASC
-                     TOK_TABLE_OR_COL
-                        p_name
+                     TOK_NULLS_FIRST
+                        TOK_TABLE_OR_COL
+                           p_name
          =
             .
                TOK_TABLE_OR_COL
@@ -2495,6 +2532,7 @@ STAGE PLANS:
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: p_partkey (type: int)
+                      null sort order: a
                       sort order: +
                       Map-reduce partition columns: p_partkey (type: int)
                       Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -2558,6 +2596,7 @@ STAGE PLANS:
                   GatherStats: false
                   Reduce Output Operator
                     key expressions: p_mfgr (type: string), p_name (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: p_mfgr (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -2667,7 +2706,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: abc
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col0: int, _col1: string, _col2: string, _col3: string, _col4: string, _col5: int, _col6: string, _col7: double, _col8: string
                         partition by: _col2
                         raw input shape:
@@ -2678,6 +2717,7 @@ STAGE PLANS:
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: int)
+                      null sort order: a
                       sort order: +
                       Map-reduce partition columns: _col0 (type: int)
                       Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -2768,11 +2808,13 @@ TOK_QUERY
                   p_mfgr
             TOK_ORDERBY
                TOK_TABSORTCOLNAMEASC
-                  TOK_TABLE_OR_COL
-                     p_name
+                  TOK_NULLS_FIRST
+                     TOK_TABLE_OR_COL
+                        p_name
                TOK_TABSORTCOLNAMEDESC
-                  TOK_TABLE_OR_COL
-                     p_size
+                  TOK_NULLS_LAST
+                     TOK_TABLE_OR_COL
+                        p_size
    TOK_INSERT
       TOK_DESTINATION
          TOK_DIR
@@ -2797,11 +2839,13 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
                         TOK_TABSORTCOLNAMEDESC
-                           TOK_TABLE_OR_COL
-                              p_size
+                           TOK_NULLS_LAST
+                              TOK_TABLE_OR_COL
+                                 p_size
             r
 
 
@@ -2832,7 +2876,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_1
                           name: noopwithmap
-                          order by: p_name, p_size(DESC)
+                          order by: p_name ASC NULLS FIRST, p_size DESC NULLS LAST
                           output shape: p_name: string, p_mfgr: string, p_size: int
                           partition by: p_mfgr
                           raw input shape:
@@ -2841,6 +2885,7 @@ STAGE PLANS:
                     Map-side function: true
                     Reduce Output Operator
                       key expressions: p_mfgr (type: string), p_name (type: string), p_size (type: int)
+                      null sort order: aaz
                       sort order: ++-
                       Map-reduce partition columns: p_mfgr (type: string)
                       Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -2911,7 +2956,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopwithmap
-                        order by: _col1, _col5(DESC)
+                        order by: _col1 ASC NULLS FIRST, _col5 DESC NULLS LAST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
@@ -2919,6 +2964,7 @@ STAGE PLANS:
                   Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col2 (type: string), _col1 (type: string), _col5 (type: int)
+                    null sort order: aaz
                     sort order: ++-
                     Map-reduce partition columns: _col2 (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -2940,7 +2986,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1, _col5(DESC)
+                        order by: _col1 ASC NULLS FIRST, _col5 DESC NULLS LAST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -3064,8 +3110,9 @@ TOK_QUERY
                   p_mfgr
             TOK_ORDERBY
                TOK_TABSORTCOLNAMEASC
-                  TOK_TABLE_OR_COL
-                     p_name
+                  TOK_NULLS_FIRST
+                     TOK_TABLE_OR_COL
+                        p_name
    TOK_INSERT
       TOK_DESTINATION
          TOK_DIR
@@ -3090,8 +3137,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             r
          TOK_SELEXPR
             TOK_FUNCTION
@@ -3103,8 +3151,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             dr
          TOK_SELEXPR
             TOK_FUNCTION
@@ -3118,8 +3167,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
                   TOK_WINDOWRANGE
                      preceding
                         unbounded
@@ -3154,7 +3204,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_1
                           name: noopwithmap
-                          order by: p_name
+                          order by: p_name ASC NULLS FIRST
                           output shape: p_name: string, p_mfgr: string, p_size: int, p_retailprice: double
                           partition by: p_mfgr
                           raw input shape:
@@ -3163,6 +3213,7 @@ STAGE PLANS:
                     Map-side function: true
                     Reduce Output Operator
                       key expressions: p_mfgr (type: string), p_name (type: string)
+                      null sort order: aa
                       sort order: ++
                       Map-reduce partition columns: p_mfgr (type: string)
                       Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -3234,7 +3285,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopwithmap
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -3242,6 +3293,7 @@ STAGE PLANS:
                   Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col2 (type: string), _col1 (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: _col2 (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -3264,7 +3316,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -3405,8 +3457,9 @@ TOK_QUERY
                   p_mfgr
             TOK_ORDERBY
                TOK_TABSORTCOLNAMEASC
-                  TOK_TABLE_OR_COL
-                     p_name
+                  TOK_NULLS_FIRST
+                     TOK_TABLE_OR_COL
+                        p_name
    TOK_INSERT
       TOK_DESTINATION
          TOK_DIR
@@ -3431,8 +3484,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             r
          TOK_SELEXPR
             TOK_FUNCTION
@@ -3444,8 +3498,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             dr
          TOK_SELEXPR
             TOK_FUNCTION
@@ -3459,8 +3514,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
                   TOK_WINDOWRANGE
                      preceding
                         unbounded
@@ -3488,6 +3544,7 @@ STAGE PLANS:
                   GatherStats: false
                   Reduce Output Operator
                     key expressions: p_mfgr (type: string), p_name (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: p_mfgr (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -3560,13 +3617,14 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
                   Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col2 (type: string), _col1 (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: _col2 (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -3589,7 +3647,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -3736,11 +3794,13 @@ TOK_QUERY
                         p_mfgr
                   TOK_ORDERBY
                      TOK_TABSORTCOLNAMEASC
-                        TOK_TABLE_OR_COL
-                           p_mfgr
+                        TOK_NULLS_FIRST
+                           TOK_TABLE_OR_COL
+                              p_mfgr
                      TOK_TABSORTCOLNAMEASC
-                        TOK_TABLE_OR_COL
-                           p_name
+                        TOK_NULLS_FIRST
+                           TOK_TABLE_OR_COL
+                              p_name
    TOK_INSERT
       TOK_DESTINATION
          TOK_DIR
@@ -3765,8 +3825,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             r
          TOK_SELEXPR
             TOK_FUNCTION
@@ -3778,8 +3839,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             dr
          TOK_SELEXPR
             TOK_FUNCTION
@@ -3793,8 +3855,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
                   TOK_WINDOWRANGE
                      preceding
                         unbounded
@@ -3823,6 +3886,7 @@ STAGE PLANS:
                   GatherStats: false
                   Reduce Output Operator
                     key expressions: p_mfgr (type: string), p_name (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: p_mfgr (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -3895,7 +3959,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -3909,7 +3973,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_1
                           name: noopwithmap
-                          order by: _col2, _col1
+                          order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                           output shape: _col1: string, _col2: string, _col5: int, _col7: double
                           partition by: _col2
                           raw input shape:
@@ -3917,7 +3981,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_2
                           name: noop
-                          order by: _col2, _col1
+                          order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                           output shape: _col1: string, _col2: string, _col5: int, _col7: double
                           partition by: _col2
                           raw input shape:
@@ -3925,6 +3989,7 @@ STAGE PLANS:
                     Map-side function: true
                     Reduce Output Operator
                       key expressions: _col2 (type: string), _col1 (type: string)
+                      null sort order: aa
                       sort order: ++
                       Map-reduce partition columns: _col2 (type: string)
                       Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -3947,7 +4012,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopwithmap
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -3955,13 +4020,14 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_2
                         name: noop
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
                   Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col2 (type: string), _col1 (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: _col2 (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -3984,7 +4050,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -4138,8 +4204,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             TOK_INSERT
                TOK_DESTINATION
                   TOK_DIR
@@ -4163,8 +4230,9 @@ TOK_QUERY
                                     p_mfgr
                               TOK_ORDERBY
                                  TOK_TABSORTCOLNAMEASC
-                                    TOK_TABLE_OR_COL
-                                       p_name
+                                    TOK_NULLS_FIRST
+                                       TOK_TABLE_OR_COL
+                                          p_name
                      cd
                   TOK_SELEXPR
                      TOK_TABLE_OR_COL
@@ -4187,8 +4255,9 @@ TOK_QUERY
                                  p_mfgr
                            TOK_ORDERBY
                               TOK_TABSORTCOLNAMEASC
-                                 TOK_TABLE_OR_COL
-                                    p_name
+                                 TOK_NULLS_FIRST
+                                    TOK_TABLE_OR_COL
+                                       p_name
                         TOK_WINDOWRANGE
                            preceding
                               2
@@ -4238,6 +4307,7 @@ STAGE PLANS:
                   GatherStats: false
                   Reduce Output Operator
                     key expressions: p_mfgr (type: string), p_name (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: p_mfgr (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -4310,13 +4380,14 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
                   Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col2 (type: string), _col1 (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: _col2 (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -4339,7 +4410,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -4488,8 +4559,9 @@ TOK_QUERY
                      p_mfgr
                TOK_ORDERBY
                   TOK_TABSORTCOLNAMEASC
-                     TOK_TABLE_OR_COL
-                        p_name
+                     TOK_NULLS_FIRST
+                        TOK_TABLE_OR_COL
+                           p_name
          TOK_TABREF
             TOK_TABNAME
                part_orc
@@ -4530,10 +4602,11 @@ TOK_QUERY
                            p_mfgr
                      TOK_SORTBY
                         TOK_TABSORTCOLNAMEASC
-                           .
-                              TOK_TABLE_OR_COL
-                                 abc
-                              p_name
+                           TOK_NULLS_FIRST
+                              .
+                                 TOK_TABLE_OR_COL
+                                    abc
+                                 p_name
             r
          TOK_SELEXPR
             TOK_FUNCTION
@@ -4547,10 +4620,11 @@ TOK_QUERY
                            p_mfgr
                      TOK_SORTBY
                         TOK_TABSORTCOLNAMEASC
-                           .
-                              TOK_TABLE_OR_COL
-                                 abc
-                              p_name
+                           TOK_NULLS_FIRST
+                              .
+                                 TOK_TABLE_OR_COL
+                                    abc
+                                 p_name
             dr
          TOK_SELEXPR
             TOK_FUNCTION
@@ -4568,10 +4642,11 @@ TOK_QUERY
                            p_mfgr
                      TOK_SORTBY
                         TOK_TABSORTCOLNAMEASC
-                           .
-                              TOK_TABLE_OR_COL
-                                 abc
-                              p_name
+                           TOK_NULLS_FIRST
+                              .
+                                 TOK_TABLE_OR_COL
+                                    abc
+                                 p_name
             cd
          TOK_SELEXPR
             .
@@ -4594,10 +4669,11 @@ TOK_QUERY
                            p_mfgr
                      TOK_SORTBY
                         TOK_TABSORTCOLNAMEASC
-                           .
-                              TOK_TABLE_OR_COL
-                                 abc
-                              p_name
+                           TOK_NULLS_FIRST
+                              .
+                                 TOK_TABLE_OR_COL
+                                    abc
+                                 p_name
                   TOK_WINDOWRANGE
                      preceding
                         unbounded
@@ -4634,10 +4710,11 @@ TOK_QUERY
                               p_mfgr
                         TOK_SORTBY
                            TOK_TABSORTCOLNAMEASC
-                              .
-                                 TOK_TABLE_OR_COL
-                                    abc
-                                 p_name
+                              TOK_NULLS_FIRST
+                                 .
+                                    TOK_TABLE_OR_COL
+                                       abc
+                                    p_name
             deltaSz
 
 
@@ -4662,6 +4739,7 @@ STAGE PLANS:
                   GatherStats: false
                   Reduce Output Operator
                     key expressions: p_mfgr (type: string), p_name (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: p_mfgr (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -4730,6 +4808,7 @@ STAGE PLANS:
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: p_partkey (type: int)
+                      null sort order: a
                       sort order: +
                       Map-reduce partition columns: p_partkey (type: int)
                       Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -4801,7 +4880,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: abc
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col0: int, _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -4812,6 +4891,7 @@ STAGE PLANS:
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: int)
+                      null sort order: a
                       sort order: +
                       Map-reduce partition columns: _col0 (type: int)
                       Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -4831,6 +4911,7 @@ STAGE PLANS:
                 Statistics: Num rows: 28 Data size: 17646 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col2 (type: string), _col1 (type: string)
+                  null sort order: aa
                   sort order: ++
                   Map-reduce partition columns: _col2 (type: string)
                   Statistics: Num rows: 28 Data size: 17646 Basic stats: COMPLETE Column stats: NONE
@@ -4853,7 +4934,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -5009,8 +5090,9 @@ TOK_QUERY
                   p_mfgr
             TOK_ORDERBY
                TOK_TABSORTCOLNAMEASC
-                  TOK_TABLE_OR_COL
-                     p_name
+                  TOK_NULLS_FIRST
+                     TOK_TABLE_OR_COL
+                        p_name
    TOK_INSERT
       TOK_DESTINATION
          TOK_DIR
@@ -5047,6 +5129,7 @@ STAGE PLANS:
                   GatherStats: false
                   Reduce Output Operator
                     key expressions: p_mfgr (type: string), p_name (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: p_mfgr (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -5119,7 +5202,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
@@ -5135,6 +5218,7 @@ STAGE PLANS:
                       Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: int)
+                        null sort order: aaa
                         sort order: +++
                         Map-reduce partition columns: _col0 (type: string), _col1 (type: string), _col2 (type: int)
                         Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -5268,8 +5352,9 @@ TOK_QUERY
                   p_mfgr
             TOK_ORDERBY
                TOK_TABSORTCOLNAMEASC
-                  TOK_TABLE_OR_COL
-                     p_mfgr
+                  TOK_NULLS_FIRST
+                     TOK_TABLE_OR_COL
+                        p_mfgr
    TOK_INSERT
       TOK_DESTINATION
          TOK_DIR
@@ -5302,8 +5387,9 @@ TOK_QUERY
                         p_mfgr
                   TOK_ORDERBY
                      TOK_TABSORTCOLNAMEASC
-                        TOK_TABLE_OR_COL
-                           p_brand
+                        TOK_NULLS_FIRST
+                           TOK_TABLE_OR_COL
+                              p_brand
                TOK_WINDOWRANGE
                   preceding
                      2
@@ -5340,6 +5426,7 @@ STAGE PLANS:
                       Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string), _col1 (type: string)
+                        null sort order: aa
                         sort order: ++
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -5414,13 +5501,14 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col0
+                        order by: _col0 ASC NULLS FIRST
                         output shape: _col0: string, _col1: string, _col2: double
                         partition by: _col0
                         raw input shape:
                   Statistics: Num rows: 13 Data size: 8021 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col0 (type: string), _col1 (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: _col0 (type: string)
                     Statistics: Num rows: 13 Data size: 8021 Basic stats: COMPLETE Column stats: NONE
@@ -5443,7 +5531,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col0
                         raw input shape:
                         window functions:
@@ -5616,8 +5704,9 @@ TOK_QUERY
                   p_mfgr
             TOK_ORDERBY
                TOK_TABSORTCOLNAMEASC
-                  TOK_TABLE_OR_COL
-                     p_name
+                  TOK_NULLS_FIRST
+                     TOK_TABLE_OR_COL
+                        p_name
    TOK_INSERT
       TOK_DESTINATION
          TOK_TAB
@@ -5643,8 +5732,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_SORTBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             r
          TOK_SELEXPR
             TOK_FUNCTION
@@ -5656,8 +5746,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_SORTBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             dr
          TOK_SELEXPR
             TOK_FUNCTION
@@ -5671,8 +5762,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_SORTBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
                   TOK_WINDOWRANGE
                      preceding
                         unbounded
@@ -5707,8 +5799,9 @@ TOK_QUERY
                               p_mfgr
                         TOK_SORTBY
                            TOK_TABSORTCOLNAMEASC
-                              TOK_TABLE_OR_COL
-                                 p_size
+                              TOK_NULLS_FIRST
+                                 TOK_TABLE_OR_COL
+                                    p_size
                      TOK_WINDOWVALUES
                         preceding
                            5
@@ -5725,11 +5818,13 @@ TOK_QUERY
                            p_mfgr
                      TOK_SORTBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_mfgr
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_mfgr
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             r
          TOK_SELEXPR
             TOK_FUNCTION
@@ -5741,11 +5836,13 @@ TOK_QUERY
                            p_mfgr
                      TOK_SORTBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_mfgr
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_mfgr
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             dr
          TOK_SELEXPR
             TOK_FUNCTION
@@ -5757,11 +5854,13 @@ TOK_QUERY
                            p_mfgr
                      TOK_SORTBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_mfgr
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_mfgr
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             cud
          TOK_SELEXPR
             TOK_FUNCTION
@@ -5782,11 +5881,13 @@ TOK_QUERY
                         p_mfgr
                   TOK_SORTBY
                      TOK_TABSORTCOLNAMEASC
-                        TOK_TABLE_OR_COL
-                           p_mfgr
+                        TOK_NULLS_FIRST
+                           TOK_TABLE_OR_COL
+                              p_mfgr
                      TOK_TABSORTCOLNAMEASC
-                        TOK_TABLE_OR_COL
-                           p_name
+                        TOK_NULLS_FIRST
+                           TOK_TABLE_OR_COL
+                              p_name
                TOK_WINDOWRANGE
                   preceding
                      2
@@ -5820,6 +5921,7 @@ STAGE PLANS:
                   GatherStats: false
                   Reduce Output Operator
                     key expressions: p_mfgr (type: string), p_name (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: p_mfgr (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -5892,7 +5994,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -5963,7 +6065,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col5
+                        order by: _col5 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -5980,6 +6082,7 @@ STAGE PLANS:
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col2 (type: string), _col1 (type: string)
+                      null sort order: aa
                       sort order: ++
                       Map-reduce partition columns: _col2 (type: string)
                       Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -6002,7 +6105,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col3, _col2
+                        order by: _col3 ASC NULLS FIRST, _col2 ASC NULLS FIRST
                         partition by: _col3
                         raw input shape:
                         window functions:
@@ -6079,12 +6182,13 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                   Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col2 (type: string), _col1 (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: _col2 (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -6106,12 +6210,13 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                   Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col2 (type: string), _col5 (type: int)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: _col2 (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -6348,8 +6453,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_mfgr
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_mfgr
             TOK_PARTITIONINGSPEC
                TOK_DISTRIBUTEBY
                   TOK_TABLE_OR_COL
@@ -6358,11 +6464,13 @@ TOK_QUERY
                      p_name
                TOK_ORDERBY
                   TOK_TABSORTCOLNAMEASC
-                     TOK_TABLE_OR_COL
-                        p_mfgr
+                     TOK_NULLS_FIRST
+                        TOK_TABLE_OR_COL
+                           p_mfgr
                   TOK_TABSORTCOLNAMEASC
-                     TOK_TABLE_OR_COL
-                        p_name
+                     TOK_NULLS_FIRST
+                        TOK_TABLE_OR_COL
+                           p_name
          TOK_PARTITIONINGSPEC
             TOK_DISTRIBUTEBY
                TOK_TABLE_OR_COL
@@ -6371,11 +6479,13 @@ TOK_QUERY
                   p_name
             TOK_ORDERBY
                TOK_TABSORTCOLNAMEASC
-                  TOK_TABLE_OR_COL
-                     p_mfgr
+                  TOK_NULLS_FIRST
+                     TOK_TABLE_OR_COL
+                        p_mfgr
                TOK_TABSORTCOLNAMEASC
-                  TOK_TABLE_OR_COL
-                     p_name
+                  TOK_NULLS_FIRST
+                     TOK_TABLE_OR_COL
+                        p_name
    TOK_INSERT
       TOK_DESTINATION
          TOK_DIR
@@ -6452,6 +6562,7 @@ STAGE PLANS:
                   GatherStats: false
                   Reduce Output Operator
                     key expressions: p_mfgr (type: string)
+                    null sort order: a
                     sort order: +
                     Map-reduce partition columns: p_mfgr (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -6524,14 +6635,14 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col2
+                        order by: _col2 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
                       Partition table definition
                         input alias: ptf_2
                         name: noop
-                        order by: _col2
+                        order by: _col2 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
@@ -6545,7 +6656,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_1
                           name: noopwithmap
-                          order by: _col2, _col1
+                          order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                           output shape: _col1: string, _col2: string, _col5: int
                           partition by: _col2, _col1
                           raw input shape:
@@ -6553,7 +6664,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_2
                           name: noop
-                          order by: _col2, _col1
+                          order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                           output shape: _col1: string, _col2: string, _col5: int
                           partition by: _col2, _col1
                           raw input shape:
@@ -6561,6 +6672,7 @@ STAGE PLANS:
                     Map-side function: true
                     Reduce Output Operator
                       key expressions: _col2 (type: string), _col1 (type: string)
+                      null sort order: aa
                       sort order: ++
                       Map-reduce partition columns: _col2 (type: string), _col1 (type: string)
                       Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -6583,7 +6695,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopwithmap
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
@@ -6591,13 +6703,14 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_2
                         name: noop
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
                   Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col2 (type: string), _col1 (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: _col2 (type: string), _col1 (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -6620,7 +6733,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         partition by: _col2, _col1
                         raw input shape:
                         window functions:
@@ -6799,8 +6912,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_mfgr
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_mfgr
             TOK_PARTITIONINGSPEC
                TOK_DISTRIBUTEBY
                   TOK_TABLE_OR_COL
@@ -6809,19 +6923,22 @@ TOK_QUERY
                      p_name
                TOK_ORDERBY
                   TOK_TABSORTCOLNAMEASC
-                     TOK_TABLE_OR_COL
-                        p_mfgr
+                     TOK_NULLS_FIRST
+                        TOK_TABLE_OR_COL
+                           p_mfgr
                   TOK_TABSORTCOLNAMEASC
-                     TOK_TABLE_OR_COL
-                        p_name
+                     TOK_NULLS_FIRST
+                        TOK_TABLE_OR_COL
+                           p_name
          TOK_PARTITIONINGSPEC
             TOK_DISTRIBUTEBY
                TOK_TABLE_OR_COL
                   p_mfgr
             TOK_ORDERBY
                TOK_TABSORTCOLNAMEASC
-                  TOK_TABLE_OR_COL
-                     p_mfgr
+                  TOK_NULLS_FIRST
+                     TOK_TABLE_OR_COL
+                        p_mfgr
    TOK_INSERT
       TOK_DESTINATION
          TOK_DIR
@@ -6843,8 +6960,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             r
          TOK_SELEXPR
             TOK_FUNCTION
@@ -6856,8 +6974,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             dr
          TOK_SELEXPR
             TOK_TABLE_OR_COL
@@ -6874,8 +6993,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
                   TOK_WINDOWRANGE
                      preceding
                         unbounded
@@ -6905,6 +7025,7 @@ STAGE PLANS:
                   GatherStats: false
                   Reduce Output Operator
                     key expressions: p_mfgr (type: string)
+                    null sort order: a
                     sort order: +
                     Map-reduce partition columns: p_mfgr (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -6977,20 +7098,21 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col2
+                        order by: _col2 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
                       Partition table definition
                         input alias: ptf_2
                         name: noop
-                        order by: _col2
+                        order by: _col2 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
                   Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col2 (type: string), _col1 (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: _col2 (type: string), _col1 (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -7013,13 +7135,14 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
                   Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col2 (type: string)
+                    null sort order: a
                     sort order: +
                     Map-reduce partition columns: _col2 (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -7042,13 +7165,14 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col2
+                        order by: _col2 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
                   Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col2 (type: string), _col1 (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: _col2 (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -7071,7 +7195,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -7248,19 +7372,22 @@ TOK_QUERY
                            p_name
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_mfgr
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_mfgr
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             TOK_PARTITIONINGSPEC
                TOK_DISTRIBUTEBY
                   TOK_TABLE_OR_COL
                      p_mfgr
                TOK_ORDERBY
                   TOK_TABSORTCOLNAMEASC
-                     TOK_TABLE_OR_COL
-                        p_mfgr
+                     TOK_NULLS_FIRST
+                        TOK_TABLE_OR_COL
+                           p_mfgr
    TOK_INSERT
       TOK_DESTINATION
          TOK_DIR
@@ -7282,8 +7409,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             r
          TOK_SELEXPR
             TOK_FUNCTION
@@ -7295,8 +7423,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             dr
          TOK_SELEXPR
             TOK_TABLE_OR_COL
@@ -7313,8 +7442,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             s1
 
 
@@ -7339,6 +7469,7 @@ STAGE PLANS:
                   GatherStats: false
                   Reduce Output Operator
                     key expressions: p_mfgr (type: string), p_name (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: p_mfgr (type: string), p_name (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -7411,20 +7542,21 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
                       Partition table definition
                         input alias: ptf_2
                         name: noop
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
                   Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col2 (type: string)
+                    null sort order: a
                     sort order: +
                     Map-reduce partition columns: _col2 (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -7447,20 +7579,21 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col2
+                        order by: _col2 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
                       Partition table definition
                         input alias: ptf_2
                         name: noop
-                        order by: _col2
+                        order by: _col2 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
                   Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col2 (type: string), _col1 (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: _col2 (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -7483,7 +7616,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -7660,19 +7793,22 @@ TOK_QUERY
                            p_name
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_mfgr
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_mfgr
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             TOK_PARTITIONINGSPEC
                TOK_DISTRIBUTEBY
                   TOK_TABLE_OR_COL
                      p_mfgr
                TOK_ORDERBY
                   TOK_TABSORTCOLNAMEASC
-                     TOK_TABLE_OR_COL
-                        p_mfgr
+                     TOK_NULLS_FIRST
+                        TOK_TABLE_OR_COL
+                           p_mfgr
          TOK_PARTITIONINGSPEC
             TOK_DISTRIBUTEBY
                TOK_TABLE_OR_COL
@@ -7681,11 +7817,13 @@ TOK_QUERY
                   p_name
             TOK_ORDERBY
                TOK_TABSORTCOLNAMEASC
-                  TOK_TABLE_OR_COL
-                     p_mfgr
+                  TOK_NULLS_FIRST
+                     TOK_TABLE_OR_COL
+                        p_mfgr
                TOK_TABSORTCOLNAMEASC
-                  TOK_TABLE_OR_COL
-                     p_name
+                  TOK_NULLS_FIRST
+                     TOK_TABLE_OR_COL
+                        p_name
    TOK_INSERT
       TOK_DESTINATION
          TOK_DIR
@@ -7763,6 +7901,7 @@ STAGE PLANS:
                   GatherStats: false
                   Reduce Output Operator
                     key expressions: p_mfgr (type: string), p_name (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: p_mfgr (type: string), p_name (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -7835,20 +7974,21 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
                       Partition table definition
                         input alias: ptf_2
                         name: noop
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
                   Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col2 (type: string)
+                    null sort order: a
                     sort order: +
                     Map-reduce partition columns: _col2 (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -7871,7 +8011,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col2
+                        order by: _col2 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
@@ -7885,7 +8025,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_1
                           name: noopwithmap
-                          order by: _col2, _col1
+                          order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                           output shape: _col1: string, _col2: string, _col5: int
                           partition by: _col2, _col1
                           raw input shape:
@@ -7894,6 +8034,7 @@ STAGE PLANS:
                     Map-side function: true
                     Reduce Output Operator
                       key expressions: _col2 (type: string), _col1 (type: string)
+                      null sort order: aa
                       sort order: ++
                       Map-reduce partition columns: _col2 (type: string), _col1 (type: string)
                       Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -7916,7 +8057,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopwithmap
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
@@ -7924,6 +8065,7 @@ STAGE PLANS:
                   Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col2 (type: string), _col1 (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: _col2 (type: string), _col1 (type: string)
                     Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -7946,7 +8088,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         partition by: _col2, _col1
                         raw input shape:
                         window functions:
@@ -8123,19 +8265,22 @@ TOK_QUERY
                         p_name
                   TOK_ORDERBY
                      TOK_TABSORTCOLNAMEASC
-                        TOK_TABLE_OR_COL
-                           p_mfgr
+                        TOK_NULLS_FIRST
+                           TOK_TABLE_OR_COL
+                              p_mfgr
     

<TRUNCATED>

[13/18] hive git commit: HIVE-12994: Implement support for NULLS FIRST/NULLS LAST (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/partition_coltype_literals.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/partition_coltype_literals.q.out b/ql/src/test/results/clientpositive/partition_coltype_literals.q.out
index bc159eb..1a93b7a 100644
--- a/ql/src/test/results/clientpositive/partition_coltype_literals.q.out
+++ b/ql/src/test/results/clientpositive/partition_coltype_literals.q.out
@@ -154,7 +154,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.RCFileOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	4                   	 
 Bucket Columns:     	[key]               	 
-Sort Columns:       	[Order(col:key, order:0)]	 
+Sort Columns:       	[Order(col:key, order:0, nullOrder:1)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: -- rename partition
@@ -205,7 +205,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.RCFileOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	4                   	 
 Bucket Columns:     	[key]               	 
-Sort Columns:       	[Order(col:key, order:0)]	 
+Sort Columns:       	[Order(col:key, order:0, nullOrder:1)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: -- insert partition

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/pcr.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/pcr.q.out b/ql/src/test/results/clientpositive/pcr.q.out
index 89c5c22..056b092 100644
--- a/ql/src/test/results/clientpositive/pcr.q.out
+++ b/ql/src/test/results/clientpositive/pcr.q.out
@@ -85,11 +85,13 @@ TOK_QUERY
                5
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               ds
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  ds
 
 
 STAGE DEPENDENCIES:
@@ -114,6 +116,7 @@ STAGE PLANS:
                 Statistics: Num rows: 13 Data size: 104 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int), _col2 (type: string)
+                  null sort order: aa
                   sort order: ++
                   Statistics: Num rows: 13 Data size: 104 Basic stats: COMPLETE Column stats: NONE
                   tag: -1
@@ -306,8 +309,9 @@ TOK_QUERY
                5
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
 
 
 STAGE DEPENDENCIES:
@@ -332,6 +336,7 @@ STAGE PLANS:
                 Statistics: Num rows: 40 Data size: 320 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
+                  null sort order: a
                   sort order: +
                   Statistics: Num rows: 40 Data size: 320 Basic stats: COMPLETE Column stats: NONE
                   tag: -1
@@ -615,11 +620,13 @@ TOK_QUERY
                'val_2'
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               ds
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  ds
 
 
 STAGE DEPENDENCIES:
@@ -644,6 +651,7 @@ STAGE PLANS:
                 Statistics: Num rows: 13 Data size: 104 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int), _col2 (type: string)
+                  null sort order: aa
                   sort order: ++
                   Statistics: Num rows: 13 Data size: 104 Basic stats: COMPLETE Column stats: NONE
                   tag: -1
@@ -851,11 +859,13 @@ TOK_QUERY
                   'val_5'
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               ds
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  ds
 
 
 STAGE DEPENDENCIES:
@@ -880,6 +890,7 @@ STAGE PLANS:
                 Statistics: Num rows: 10 Data size: 80 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int), _col2 (type: string)
+                  null sort order: aa
                   sort order: ++
                   Statistics: Num rows: 10 Data size: 80 Basic stats: COMPLETE Column stats: NONE
                   tag: -1
@@ -1089,11 +1100,13 @@ TOK_QUERY
                   'val_5'
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               ds
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  ds
 
 
 STAGE DEPENDENCIES:
@@ -1118,6 +1131,7 @@ STAGE PLANS:
                 Statistics: Num rows: 16 Data size: 128 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int), _col2 (type: string)
+                  null sort order: aa
                   sort order: ++
                   Statistics: Num rows: 16 Data size: 128 Basic stats: COMPLETE Column stats: NONE
                   tag: -1
@@ -1383,11 +1397,13 @@ TOK_QUERY
                   'val_5'
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               ds
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  ds
 
 
 STAGE DEPENDENCIES:
@@ -1412,6 +1428,7 @@ STAGE PLANS:
                 Statistics: Num rows: 33 Data size: 264 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int), _col2 (type: string)
+                  null sort order: aa
                   sort order: ++
                   Statistics: Num rows: 33 Data size: 264 Basic stats: COMPLETE Column stats: NONE
                   tag: -1
@@ -1677,11 +1694,13 @@ TOK_QUERY
                14
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               value
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  value
 
 
 STAGE DEPENDENCIES:
@@ -1706,6 +1725,7 @@ STAGE PLANS:
                 Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: 14 (type: int), _col1 (type: string)
+                  null sort order: aa
                   sort order: ++
                   Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
                   tag: -1
@@ -1887,11 +1907,13 @@ TOK_QUERY
                '2000-04-09'
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               value
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  value
 
 
 STAGE DEPENDENCIES:
@@ -1912,6 +1934,7 @@ STAGE PLANS:
               Statistics: Num rows: 40 Data size: 320 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
                 key expressions: _col0 (type: int), _col1 (type: string)
+                null sort order: aa
                 sort order: ++
                 Statistics: Num rows: 40 Data size: 320 Basic stats: COMPLETE Column stats: NONE
                 tag: -1
@@ -2133,11 +2156,13 @@ TOK_QUERY
                '2000-04-10'
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               value
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  value
 
 
 STAGE DEPENDENCIES:
@@ -2158,6 +2183,7 @@ STAGE PLANS:
               Statistics: Num rows: 60 Data size: 480 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
                 key expressions: _col0 (type: int), _col1 (type: string)
+                null sort order: aa
                 sort order: ++
                 Statistics: Num rows: 60 Data size: 480 Basic stats: COMPLETE Column stats: NONE
                 tag: -1
@@ -2460,14 +2486,17 @@ TOK_QUERY
                   2
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               value
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  value
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               ds
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  ds
 
 
 STAGE DEPENDENCIES:
@@ -2492,6 +2521,7 @@ STAGE PLANS:
                 Statistics: Num rows: 40 Data size: 320 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string)
+                  null sort order: aaa
                   sort order: +++
                   Statistics: Num rows: 40 Data size: 320 Basic stats: COMPLETE Column stats: NONE
                   tag: -1
@@ -2689,10 +2719,11 @@ TOK_QUERY
             TOK_ALLCOLREF
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            .
-               TOK_TABLE_OR_COL
-                  t1
-               key
+            TOK_NULLS_FIRST
+               .
+                  TOK_TABLE_OR_COL
+                     t1
+                  key
 
 
 STAGE DEPENDENCIES:
@@ -2718,6 +2749,7 @@ STAGE PLANS:
                 Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
@@ -2738,6 +2770,7 @@ STAGE PLANS:
                 Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
@@ -2829,6 +2862,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col0 (type: int)
+              null sort order: a
               sort order: +
               Statistics: Num rows: 22 Data size: 176 Basic stats: COMPLETE Column stats: NONE
               tag: -1
@@ -2991,10 +3025,11 @@ TOK_QUERY
             TOK_ALLCOLREF
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            .
-               TOK_TABLE_OR_COL
-                  t1
-               key
+            TOK_NULLS_FIRST
+               .
+                  TOK_TABLE_OR_COL
+                     t1
+                  key
 
 
 STAGE DEPENDENCIES:
@@ -3020,6 +3055,7 @@ STAGE PLANS:
                 Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
@@ -3040,6 +3076,7 @@ STAGE PLANS:
                 Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
@@ -3177,6 +3214,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col0 (type: int)
+              null sort order: a
               sort order: +
               Statistics: Num rows: 22 Data size: 176 Basic stats: COMPLETE Column stats: NONE
               tag: -1
@@ -3354,14 +3392,17 @@ TOK_QUERY
                   2
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               value
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  value
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               ds
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  ds
 
 
 STAGE DEPENDENCIES:
@@ -3386,6 +3427,7 @@ STAGE PLANS:
                 Statistics: Num rows: 48 Data size: 384 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string)
+                  null sort order: aaa
                   sort order: +++
                   Statistics: Num rows: 48 Data size: 384 Basic stats: COMPLETE Column stats: NONE
                   tag: -1
@@ -3718,14 +3760,17 @@ TOK_QUERY
                   2
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               value
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  value
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               ds
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  ds
 
 
 STAGE DEPENDENCIES:
@@ -3750,6 +3795,7 @@ STAGE PLANS:
                 Statistics: Num rows: 30 Data size: 240 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string)
+                  null sort order: aaa
                   sort order: +++
                   Statistics: Num rows: 30 Data size: 240 Basic stats: COMPLETE Column stats: NONE
                   tag: -1
@@ -5268,8 +5314,9 @@ TOK_QUERY
                11
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
       TOK_LIMIT
          10
 
@@ -5292,6 +5339,7 @@ STAGE PLANS:
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
                 key expressions: _col0 (type: string)
+                null sort order: a
                 sort order: +
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                 tag: -1
@@ -5446,14 +5494,17 @@ TOK_QUERY
                11
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               ds
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  ds
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               hr
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  hr
 
 
 STAGE DEPENDENCIES:
@@ -5478,6 +5529,7 @@ STAGE PLANS:
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: '11' (type: string), '2008-04-08' (type: string), _col3 (type: string)
+                  null sort order: aaa
                   sort order: +++
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   tag: -1
@@ -5670,14 +5722,17 @@ TOK_QUERY
                11
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               ds
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  ds
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               hr
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  hr
 
 
 STAGE DEPENDENCIES:
@@ -5702,6 +5757,7 @@ STAGE PLANS:
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: '11' (type: string), _col2 (type: string), '11' (type: string)
+                  null sort order: aaa
                   sort order: +++
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/pcs.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/pcs.q.out b/ql/src/test/results/clientpositive/pcs.q.out
index 51a21dd..a1382f1 100644
--- a/ql/src/test/results/clientpositive/pcs.q.out
+++ b/ql/src/test/results/clientpositive/pcs.q.out
@@ -125,14 +125,17 @@ TOK_QUERY
                   2
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               value
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  value
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               ds
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  ds
 
 
 STAGE DEPENDENCIES:
@@ -157,6 +160,7 @@ STAGE PLANS:
                 Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string)
+                  null sort order: aaa
                   sort order: +++
                   Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
                   tag: -1
@@ -788,6 +792,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: ds (type: string)
+              null sort order: a
               sort order: +
               Map-reduce partition columns: ds (type: string)
               Statistics: Num rows: 40 Data size: 320 Basic stats: COMPLETE Column stats: NONE
@@ -800,6 +805,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: ds (type: string)
+              null sort order: a
               sort order: +
               Map-reduce partition columns: ds (type: string)
               Statistics: Num rows: 40 Data size: 320 Basic stats: COMPLETE Column stats: NONE
@@ -1198,7 +1204,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col0
+                  order by: _col0 ASC NULLS FIRST
                   partition by: _col0
                   raw input shape:
                   window functions:
@@ -1356,20 +1362,23 @@ TOK_QUERY
             '2008-04-08'
       TOK_SORTBY
          TOK_TABSORTCOLNAMEASC
-            .
-               TOK_TABLE_OR_COL
-                  A
-               key
+            TOK_NULLS_FIRST
+               .
+                  TOK_TABLE_OR_COL
+                     A
+                  key
          TOK_TABSORTCOLNAMEASC
-            .
-               TOK_TABLE_OR_COL
-                  A
-               value
+            TOK_NULLS_FIRST
+               .
+                  TOK_TABLE_OR_COL
+                     A
+                  value
          TOK_TABSORTCOLNAMEASC
-            .
-               TOK_TABLE_OR_COL
-                  A
-               ds
+            TOK_NULLS_FIRST
+               .
+                  TOK_TABLE_OR_COL
+                     A
+                  ds
 
 
 STAGE DEPENDENCIES:
@@ -1400,6 +1409,7 @@ STAGE PLANS:
                     Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: int), _col1 (type: string), '2008-04-08' (type: string)
+                      null sort order: aaa
                       sort order: +++
                       Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                       tag: -1
@@ -1424,6 +1434,7 @@ STAGE PLANS:
                     Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: int), _col1 (type: string), '2008-04-08' (type: string)
+                      null sort order: aaa
                       sort order: +++
                       Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                       tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/perf/query12.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query12.q.out b/ql/src/test/results/clientpositive/perf/query12.q.out
index f326a92..986a307 100644
--- a/ql/src/test/results/clientpositive/perf/query12.q.out
+++ b/ql/src/test/results/clientpositive/perf/query12.q.out
@@ -24,7 +24,7 @@ Stage-0
               Select Operator [SEL_21] (rows=34938 width=1436)
                 Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
                 PTF Operator [PTF_20] (rows=34938 width=1436)
-                  Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col3","partition by:":"_col3"}]
+                  Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col3 ASC NULLS FIRST","partition by:":"_col3"}]
                   Select Operator [SEL_19] (rows=34938 width=1436)
                     Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
                   <-Reducer 4 [SIMPLE_EDGE]

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/perf/query20.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query20.q.out b/ql/src/test/results/clientpositive/perf/query20.q.out
index 74d286a..9af7729 100644
--- a/ql/src/test/results/clientpositive/perf/query20.q.out
+++ b/ql/src/test/results/clientpositive/perf/query20.q.out
@@ -26,7 +26,7 @@ Stage-0
               Select Operator [SEL_23] (rows=139755 width=1436)
                 Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
                 PTF Operator [PTF_22] (rows=139755 width=1436)
-                  Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col3","partition by:":"_col3"}]
+                  Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col3 ASC NULLS FIRST","partition by:":"_col3"}]
                   Select Operator [SEL_21] (rows=139755 width=1436)
                     Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
                   <-Reducer 4 [SIMPLE_EDGE]

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/perf/query51.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query51.q.out b/ql/src/test/results/clientpositive/perf/query51.q.out
index f568ead..08546c5 100644
--- a/ql/src/test/results/clientpositive/perf/query51.q.out
+++ b/ql/src/test/results/clientpositive/perf/query51.q.out
@@ -112,7 +112,7 @@ Stage-0
                 Filter Operator [FIL_58] (rows=7365 width=1119)
                   predicate:(max_window_0 > max_window_1)
                   PTF Operator [PTF_45] (rows=22096 width=1119)
-                    Function definitions:[{},{"name:":"windowingtablefunction","order by:":"CASE WHEN (_col1 is not null) THEN (_col1) ELSE (_col4) END","partition by:":"CASE WHEN (_col0 is not null) THEN (_col0) ELSE (_col3) END"}]
+                    Function definitions:[{},{"name:":"windowingtablefunction","order by:":"CASE WHEN (_col1 is not null) THEN (_col1) ELSE (_col4) END ASC NULLS FIRST","partition by:":"CASE WHEN (_col0 is not null) THEN (_col0) ELSE (_col3) END"}]
                     Select Operator [SEL_44] (rows=22096 width=1119)
                       Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
                     <-Reducer 4 [SIMPLE_EDGE]
@@ -126,7 +126,7 @@ Stage-0
                             Select Operator [SEL_37] (rows=20088 width=1119)
                               Output:["_col0","_col1","_col2"]
                               PTF Operator [PTF_36] (rows=20088 width=1119)
-                                Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col1","partition by:":"_col0"}]
+                                Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col1 ASC NULLS FIRST","partition by:":"_col0"}]
                                 Group By Operator [GBY_32] (rows=20088 width=1119)
                                   Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, KEY._col1
                                 <-Reducer 9 [SIMPLE_EDGE]
@@ -162,7 +162,7 @@ Stage-0
                             Select Operator [SEL_17] (rows=20088 width=1119)
                               Output:["_col0","_col1","_col2"]
                               PTF Operator [PTF_16] (rows=20088 width=1119)
-                                Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col1","partition by:":"_col0"}]
+                                Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col1 ASC NULLS FIRST","partition by:":"_col0"}]
                                 Group By Operator [GBY_12] (rows=20088 width=1119)
                                   Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, KEY._col1
                                 <-Reducer 2 [SIMPLE_EDGE]

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/perf/query67.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query67.q.out b/ql/src/test/results/clientpositive/perf/query67.q.out
index f158829..83dab38 100644
--- a/ql/src/test/results/clientpositive/perf/query67.q.out
+++ b/ql/src/test/results/clientpositive/perf/query67.q.out
@@ -111,7 +111,7 @@ Stage-0
                 Filter Operator [FIL_47] (rows=762300 width=1436)
                   predicate:(rank_window_0 <= 100)
                   PTF Operator [PTF_29] (rows=2286900 width=1436)
-                    Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col16(DESC)","partition by:":"_col0"}]
+                    Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col16 DESC NULLS LAST","partition by:":"_col0"}]
                     Select Operator [SEL_28] (rows=2286900 width=1436)
                       Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col16"]
                     <-Reducer 5 [SIMPLE_EDGE]

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/perf/query70.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query70.q.out b/ql/src/test/results/clientpositive/perf/query70.q.out
index b921346..37cebc8 100644
--- a/ql/src/test/results/clientpositive/perf/query70.q.out
+++ b/ql/src/test/results/clientpositive/perf/query70.q.out
@@ -31,7 +31,7 @@ Stage-0
               Select Operator [SEL_57] (rows=66289 width=1119)
                 Output:["_col0","_col1","_col2","_col3","_col4"]
                 PTF Operator [PTF_56] (rows=66289 width=1119)
-                  Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col4","partition by:":"_col5, CASE WHEN ((_col5 = 2)) THEN (_col0) ELSE (null) END"}]
+                  Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col4 ASC NULLS FIRST","partition by:":"_col5, CASE WHEN ((_col5 = 2)) THEN (_col0) ELSE (null) END"}]
                   Select Operator [SEL_55] (rows=66289 width=1119)
                     Output:["_col0","_col1","_col4","_col5"]
                   <-Reducer 4 [SIMPLE_EDGE]
@@ -97,7 +97,7 @@ Stage-0
                                             Filter Operator [FIL_83] (rows=6696 width=1119)
                                               predicate:(rank_window_0 <= 5)
                                               PTF Operator [PTF_31] (rows=20088 width=1119)
-                                                Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col1(DESC)","partition by:":"_col0"}]
+                                                Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col1 DESC NULLS LAST","partition by:":"_col0"}]
                                                 Select Operator [SEL_30] (rows=20088 width=1119)
                                                   Output:["_col0","_col1"]
                                                 <-Reducer 13 [SIMPLE_EDGE]

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/perf/query89.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query89.q.out b/ql/src/test/results/clientpositive/perf/query89.q.out
index 71db5d4..0cda449 100644
--- a/ql/src/test/results/clientpositive/perf/query89.q.out
+++ b/ql/src/test/results/clientpositive/perf/query89.q.out
@@ -81,7 +81,7 @@ Stage-0
                   Select Operator [SEL_29] (rows=153730 width=1436)
                     Output:["avg_window_0","_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
                     PTF Operator [PTF_28] (rows=153730 width=1436)
-                      Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col0, _col2, _col3, _col4","partition by:":"_col0, _col2, _col3, _col4"}]
+                      Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col0 ASC NULLS FIRST, _col2 ASC NULLS FIRST, _col3 ASC NULLS FIRST, _col4 ASC NULLS FIRST","partition by:":"_col0, _col2, _col3, _col4"}]
                       Select Operator [SEL_27] (rows=153730 width=1436)
                         Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
                       <-Reducer 5 [SIMPLE_EDGE]

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/perf/query98.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/perf/query98.q.out b/ql/src/test/results/clientpositive/perf/query98.q.out
index f51744d..fc4b7a3 100644
--- a/ql/src/test/results/clientpositive/perf/query98.q.out
+++ b/ql/src/test/results/clientpositive/perf/query98.q.out
@@ -24,7 +24,7 @@ Stage-0
             Select Operator [SEL_23] (rows=139755 width=1436)
               Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"]
               PTF Operator [PTF_22] (rows=139755 width=1436)
-                Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col3","partition by:":"_col3"}]
+                Function definitions:[{},{"name:":"windowingtablefunction","order by:":"_col3 ASC NULLS FIRST","partition by:":"_col3"}]
                 Select Operator [SEL_21] (rows=139755 width=1436)
                   Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
                 <-Reducer 4 [SIMPLE_EDGE]

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/pointlookup2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/pointlookup2.q.out b/ql/src/test/results/clientpositive/pointlookup2.q.out
index 5192e1a..fb17e72 100644
--- a/ql/src/test/results/clientpositive/pointlookup2.q.out
+++ b/ql/src/test/results/clientpositive/pointlookup2.q.out
@@ -141,14 +141,17 @@ TOK_QUERY
                   2
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               value
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  value
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               ds
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  ds
 
 
 STAGE DEPENDENCIES:
@@ -173,6 +176,7 @@ STAGE PLANS:
                 Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string)
+                  null sort order: aaa
                   sort order: +++
                   Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
                   tag: -1
@@ -365,10 +369,11 @@ TOK_QUERY
             TOK_ALLCOLREF
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            .
-               TOK_TABLE_OR_COL
-                  t1
-               key
+            TOK_NULLS_FIRST
+               .
+                  TOK_TABLE_OR_COL
+                     t1
+                  key
 
 
 STAGE DEPENDENCIES:
@@ -394,6 +399,7 @@ STAGE PLANS:
                 Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
@@ -414,6 +420,7 @@ STAGE PLANS:
                 Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
@@ -505,6 +512,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col0 (type: int)
+              null sort order: a
               sort order: +
               Statistics: Num rows: 22 Data size: 176 Basic stats: COMPLETE Column stats: NONE
               tag: -1
@@ -627,10 +635,11 @@ TOK_QUERY
             TOK_ALLCOLREF
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            .
-               TOK_TABLE_OR_COL
-                  t1
-               key
+            TOK_NULLS_FIRST
+               .
+                  TOK_TABLE_OR_COL
+                     t1
+                  key
 
 
 STAGE DEPENDENCIES:
@@ -656,6 +665,7 @@ STAGE PLANS:
                 Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
@@ -676,6 +686,7 @@ STAGE PLANS:
                 Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
@@ -813,6 +824,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col0 (type: int)
+              null sort order: a
               sort order: +
               Statistics: Num rows: 22 Data size: 176 Basic stats: COMPLETE Column stats: NONE
               tag: -1
@@ -941,20 +953,23 @@ TOK_QUERY
                   2
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            .
-               TOK_TABLE_OR_COL
-                  t2
-               key
+            TOK_NULLS_FIRST
+               .
+                  TOK_TABLE_OR_COL
+                     t2
+                  key
          TOK_TABSORTCOLNAMEASC
-            .
-               TOK_TABLE_OR_COL
-                  t2
-               value
+            TOK_NULLS_FIRST
+               .
+                  TOK_TABLE_OR_COL
+                     t2
+                  value
          TOK_TABSORTCOLNAMEASC
-            .
-               TOK_TABLE_OR_COL
-                  t1
-               ds
+            TOK_NULLS_FIRST
+               .
+                  TOK_TABLE_OR_COL
+                     t1
+                  ds
 
 
 STAGE DEPENDENCIES:
@@ -979,6 +994,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
+                  null sort order: 
                   sort order: 
                   Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
                   tag: 0
@@ -997,6 +1013,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 1 Data size: 18 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
+                  null sort order: 
                   sort order: 
                   Statistics: Num rows: 1 Data size: 18 Basic stats: COMPLETE Column stats: NONE
                   tag: 1
@@ -1182,6 +1199,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col4 (type: int), _col5 (type: string), _col2 (type: string)
+              null sort order: aaa
               sort order: +++
               Statistics: Num rows: 11 Data size: 88 Basic stats: COMPLETE Column stats: NONE
               tag: -1
@@ -1310,20 +1328,23 @@ TOK_QUERY
                   2
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            .
-               TOK_TABLE_OR_COL
-                  t1
-               key
+            TOK_NULLS_FIRST
+               .
+                  TOK_TABLE_OR_COL
+                     t1
+                  key
          TOK_TABSORTCOLNAMEASC
-            .
-               TOK_TABLE_OR_COL
-                  t1
-               value
+            TOK_NULLS_FIRST
+               .
+                  TOK_TABLE_OR_COL
+                     t1
+                  value
          TOK_TABSORTCOLNAMEASC
-            .
-               TOK_TABLE_OR_COL
-                  t2
-               ds
+            TOK_NULLS_FIRST
+               .
+                  TOK_TABLE_OR_COL
+                     t2
+                  ds
 
 
 STAGE DEPENDENCIES:
@@ -1348,6 +1369,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 30 Data size: 240 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
+                  null sort order: 
                   sort order: 
                   Statistics: Num rows: 30 Data size: 240 Basic stats: COMPLETE Column stats: NONE
                   tag: 0
@@ -1366,6 +1388,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 1 Data size: 18 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
+                  null sort order: 
                   sort order: 
                   Statistics: Num rows: 1 Data size: 18 Basic stats: COMPLETE Column stats: NONE
                   tag: 1
@@ -1597,6 +1620,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string)
+              null sort order: aaa
               sort order: +++
               Statistics: Num rows: 16 Data size: 128 Basic stats: COMPLETE Column stats: NONE
               tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/pointlookup3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/pointlookup3.q.out b/ql/src/test/results/clientpositive/pointlookup3.q.out
index 791be61..d5c4157 100644
--- a/ql/src/test/results/clientpositive/pointlookup3.q.out
+++ b/ql/src/test/results/clientpositive/pointlookup3.q.out
@@ -98,17 +98,21 @@ TOK_QUERY
                   2
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               value
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  value
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               ds1
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  ds1
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               ds2
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  ds2
 
 
 STAGE DEPENDENCIES:
@@ -133,6 +137,7 @@ STAGE PLANS:
                 Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string)
+                  null sort order: aaaa
                   sort order: ++++
                   Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
                   tag: -1
@@ -338,17 +343,21 @@ TOK_QUERY
                   2
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               value
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  value
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               ds1
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  ds1
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               ds2
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  ds2
 
 
 STAGE DEPENDENCIES:
@@ -373,6 +382,7 @@ STAGE PLANS:
                 Statistics: Num rows: 10 Data size: 80 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: 1 (type: int), _col1 (type: string), _col2 (type: string), '2001-04-08' (type: string)
+                  null sort order: aaaa
                   sort order: ++++
                   Statistics: Num rows: 10 Data size: 80 Basic stats: COMPLETE Column stats: NONE
                   tag: -1
@@ -520,10 +530,11 @@ TOK_QUERY
             TOK_ALLCOLREF
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            .
-               TOK_TABLE_OR_COL
-                  t1
-               key
+            TOK_NULLS_FIRST
+               .
+                  TOK_TABLE_OR_COL
+                     t1
+                  key
 
 
 STAGE DEPENDENCIES:
@@ -549,6 +560,7 @@ STAGE PLANS:
                 Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
@@ -569,6 +581,7 @@ STAGE PLANS:
                 Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
@@ -661,6 +674,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col0 (type: int)
+              null sort order: a
               sort order: +
               Statistics: Num rows: 22 Data size: 176 Basic stats: COMPLETE Column stats: NONE
               tag: -1
@@ -783,10 +797,11 @@ TOK_QUERY
             TOK_ALLCOLREF
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            .
-               TOK_TABLE_OR_COL
-                  t1
-               key
+            TOK_NULLS_FIRST
+               .
+                  TOK_TABLE_OR_COL
+                     t1
+                  key
 
 
 STAGE DEPENDENCIES:
@@ -812,6 +827,7 @@ STAGE PLANS:
                 Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
@@ -832,6 +848,7 @@ STAGE PLANS:
                 Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
@@ -971,6 +988,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col0 (type: int)
+              null sort order: a
               sort order: +
               Statistics: Num rows: 22 Data size: 176 Basic stats: COMPLETE Column stats: NONE
               tag: -1
@@ -1099,20 +1117,23 @@ TOK_QUERY
                   2
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            .
-               TOK_TABLE_OR_COL
-                  t2
-               key
+            TOK_NULLS_FIRST
+               .
+                  TOK_TABLE_OR_COL
+                     t2
+                  key
          TOK_TABSORTCOLNAMEASC
-            .
-               TOK_TABLE_OR_COL
-                  t2
-               value
+            TOK_NULLS_FIRST
+               .
+                  TOK_TABLE_OR_COL
+                     t2
+                  value
          TOK_TABSORTCOLNAMEASC
-            .
-               TOK_TABLE_OR_COL
-                  t1
-               ds1
+            TOK_NULLS_FIRST
+               .
+                  TOK_TABLE_OR_COL
+                     t1
+                  ds1
 
 
 STAGE DEPENDENCIES:
@@ -1137,6 +1158,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2, _col3
                 Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
+                  null sort order: 
                   sort order: 
                   Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
                   tag: 0
@@ -1155,6 +1177,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2, _col3
                 Statistics: Num rows: 30 Data size: 240 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
+                  null sort order: 
                   sort order: 
                   Statistics: Num rows: 30 Data size: 240 Basic stats: COMPLETE Column stats: NONE
                   tag: 1
@@ -1344,6 +1367,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col4 (type: int), _col5 (type: string), _col2 (type: string)
+              null sort order: aaa
               sort order: +++
               Statistics: Num rows: 16 Data size: 128 Basic stats: COMPLETE Column stats: NONE
               tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/pointlookup4.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/pointlookup4.q.out b/ql/src/test/results/clientpositive/pointlookup4.q.out
index 948ea7a..0a9bd3e 100644
--- a/ql/src/test/results/clientpositive/pointlookup4.q.out
+++ b/ql/src/test/results/clientpositive/pointlookup4.q.out
@@ -108,17 +108,21 @@ TOK_QUERY
                   2
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               value
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  value
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               ds1
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  ds1
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               ds2
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  ds2
 
 
 STAGE DEPENDENCIES:
@@ -143,6 +147,7 @@ STAGE PLANS:
                 Statistics: Num rows: 40 Data size: 320 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string)
+                  null sort order: aaaa
                   sort order: ++++
                   Statistics: Num rows: 40 Data size: 320 Basic stats: COMPLETE Column stats: NONE
                   tag: -1
@@ -348,17 +353,21 @@ TOK_QUERY
                   2
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               value
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  value
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               ds1
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  ds1
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               ds2
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  ds2
 
 
 STAGE DEPENDENCIES:
@@ -383,6 +392,7 @@ STAGE PLANS:
                 Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string)
+                  null sort order: aaaa
                   sort order: ++++
                   Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
                   tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/ppd_join_filter.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/ppd_join_filter.q.out b/ql/src/test/results/clientpositive/ppd_join_filter.q.out
index 8b640e1..74d2de1 100644
--- a/ql/src/test/results/clientpositive/ppd_join_filter.q.out
+++ b/ql/src/test/results/clientpositive/ppd_join_filter.q.out
@@ -144,6 +144,7 @@ STAGE PLANS:
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -250,6 +251,7 @@ STAGE PLANS:
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -259,6 +261,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col0 (type: string)
+              null sort order: a
               sort order: +
               Map-reduce partition columns: _col0 (type: string)
               Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
@@ -558,6 +561,7 @@ STAGE PLANS:
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -664,6 +668,7 @@ STAGE PLANS:
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -673,6 +678,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col0 (type: string)
+              null sort order: a
               sort order: +
               Map-reduce partition columns: _col0 (type: string)
               Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
@@ -972,6 +978,7 @@ STAGE PLANS:
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -1078,6 +1085,7 @@ STAGE PLANS:
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -1087,6 +1095,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col0 (type: string)
+              null sort order: a
               sort order: +
               Map-reduce partition columns: _col0 (type: string)
               Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
@@ -1386,6 +1395,7 @@ STAGE PLANS:
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -1492,6 +1502,7 @@ STAGE PLANS:
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -1501,6 +1512,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col0 (type: string)
+              null sort order: a
               sort order: +
               Map-reduce partition columns: _col0 (type: string)
               Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/ppd_union_view.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/ppd_union_view.q.out b/ql/src/test/results/clientpositive/ppd_union_view.q.out
index 210cece..36bb214 100644
--- a/ql/src/test/results/clientpositive/ppd_union_view.q.out
+++ b/ql/src/test/results/clientpositive/ppd_union_view.q.out
@@ -182,6 +182,7 @@ STAGE PLANS:
                 Statistics: Num rows: 1 Data size: 14 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string), _col2 (type: string)
+                  null sort order: aa
                   sort order: ++
                   Map-reduce partition columns: _col0 (type: string), _col2 (type: string)
                   Statistics: Num rows: 1 Data size: 14 Basic stats: COMPLETE Column stats: NONE
@@ -202,6 +203,7 @@ STAGE PLANS:
                 Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col1 (type: string), _col2 (type: string)
+                  null sort order: aa
                   sort order: ++
                   Map-reduce partition columns: _col1 (type: string), _col2 (type: string)
                   Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: NONE
@@ -529,6 +531,7 @@ STAGE PLANS:
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string), '2011-10-15' (type: string)
+                  null sort order: aa
                   sort order: ++
                   Map-reduce partition columns: _col0 (type: string), '2011-10-15' (type: string)
                   Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -549,6 +552,7 @@ STAGE PLANS:
                 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col1 (type: string), '2011-10-15' (type: string)
+                  null sort order: aa
                   sort order: ++
                   Map-reduce partition columns: _col1 (type: string), '2011-10-15' (type: string)
                   Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/ppd_vc.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/ppd_vc.q.out b/ql/src/test/results/clientpositive/ppd_vc.q.out
index 69a9bc6..4575172 100644
--- a/ql/src/test/results/clientpositive/ppd_vc.q.out
+++ b/ql/src/test/results/clientpositive/ppd_vc.q.out
@@ -390,14 +390,17 @@ TOK_QUERY
                   b
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               ds
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  ds
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               hr
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  hr
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               BLOCK__OFFSET__INSIDE__FILE
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  BLOCK__OFFSET__INSIDE__FILE
 
 
 STAGE DEPENDENCIES:
@@ -423,6 +426,7 @@ STAGE PLANS:
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -442,6 +446,7 @@ STAGE PLANS:
                 Statistics: Num rows: 222 Data size: 2358 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 222 Data size: 2358 Basic stats: COMPLETE Column stats: NONE
@@ -724,6 +729,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col2 (type: string), _col3 (type: string), _col4 (type: bigint)
+              null sort order: aaa
               sort order: +++
               Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
               tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/ppd_windowing1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/ppd_windowing1.q.out b/ql/src/test/results/clientpositive/ppd_windowing1.q.out
index 071b730..5f73138 100644
--- a/ql/src/test/results/clientpositive/ppd_windowing1.q.out
+++ b/ql/src/test/results/clientpositive/ppd_windowing1.q.out
@@ -37,7 +37,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col0
+                  order by: _col0 ASC NULLS FIRST
                   partition by: _col0
                   raw input shape:
                   window functions:
@@ -103,7 +103,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col0
+                  order by: _col0 ASC NULLS FIRST
                   partition by: _col0
                   raw input shape:
                   window functions:
@@ -169,7 +169,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col0
+                  order by: _col0 ASC NULLS FIRST
                   partition by: _col0
                   raw input shape:
                   window functions:
@@ -235,7 +235,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col0
+                  order by: _col0 ASC NULLS FIRST
                   partition by: _col0
                   raw input shape:
                   window functions:
@@ -301,7 +301,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col0
+                  order by: _col0 ASC NULLS FIRST
                   partition by: _col0
                   raw input shape:
                   window functions:
@@ -369,7 +369,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col0, _col1
+                  order by: _col0 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   partition by: _col0, _col1
                   raw input shape:
                   window functions:
@@ -435,7 +435,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col0, _col1
+                  order by: _col0 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   partition by: _col0, _col1
                   raw input shape:
                   window functions:
@@ -501,7 +501,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col0, _col1
+                  order by: _col0 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   partition by: _col0, _col1
                   raw input shape:
                   window functions:
@@ -567,7 +567,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col0, _col1
+                  order by: _col0 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   partition by: _col0, _col1
                   raw input shape:
                   window functions:
@@ -633,7 +633,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col0, _col1
+                  order by: _col0 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   partition by: _col0, _col1
                   raw input shape:
                   window functions:
@@ -702,7 +702,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col0, _col1
+                  order by: _col0 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   partition by: _col0, _col1
                   raw input shape:
                   window functions:
@@ -748,7 +748,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col1
                   raw input shape:
                   window functions:
@@ -815,7 +815,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col0, _col1
+                  order by: _col0 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   partition by: _col0, _col1
                   raw input shape:
                   window functions:
@@ -861,7 +861,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col1
                   raw input shape:
                   window functions:
@@ -928,7 +928,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col0, _col1
+                  order by: _col0 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   partition by: _col0, _col1
                   raw input shape:
                   window functions:
@@ -974,7 +974,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col1
                   raw input shape:
                   window functions:
@@ -1041,7 +1041,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col0, _col1
+                  order by: _col0 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   partition by: _col0, _col1
                   raw input shape:
                   window functions:
@@ -1087,7 +1087,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col1
                   raw input shape:
                   window functions:
@@ -1156,7 +1156,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col0, _col1
+                  order by: _col0 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   partition by: _col0, _col1
                   raw input shape:
                   window functions:
@@ -1202,7 +1202,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col1
                   raw input shape:
                   window functions:
@@ -1269,7 +1269,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col0, _col1
+                  order by: _col0 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   partition by: _col0, _col1
                   raw input shape:
                   window functions:
@@ -1315,7 +1315,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col1
                   raw input shape:
                   window functions:
@@ -1382,7 +1382,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col0, _col1
+                  order by: _col0 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   partition by: _col0, _col1
                   raw input shape:
                   window functions:
@@ -1428,7 +1428,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col1
                   raw input shape:
                   window functions:
@@ -1495,7 +1495,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col0, _col1
+                  order by: _col0 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   partition by: _col0, _col1
                   raw input shape:
                   window functions:
@@ -1541,7 +1541,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col1
                   raw input shape:
                   window functions:
@@ -1607,7 +1607,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col0, _col1
+                  order by: _col0 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   partition by: _col0, _col1
                   raw input shape:
                   window functions:
@@ -1653,7 +1653,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col2
+                  order by: _col2 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:
@@ -1722,7 +1722,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: (UDFToDouble(_col0) + UDFToDouble(2))
+                  order by: (UDFToDouble(_col0) + UDFToDouble(2)) ASC NULLS FIRST
                   partition by: (UDFToDouble(_col0) + UDFToDouble(2))
                   raw input shape:
                   window functions:
@@ -1789,7 +1789,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: (UDFToDouble(_col0) + UDFToDouble(_col1))
+                  order by: (UDFToDouble(_col0) + UDFToDouble(_col1)) ASC NULLS FIRST
                   partition by: (UDFToDouble(_col0) + UDFToDouble(_col1))
                   raw input shape:
                   window functions:
@@ -1857,7 +1857,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col0, _col1
+                  order by: _col0 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   partition by: _col0, _col1
                   raw input shape:
                   window functions:
@@ -1924,7 +1924,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: (UDFToDouble(_col0) + UDFToDouble(_col1))
+                  order by: (UDFToDouble(_col0) + UDFToDouble(_col1)) ASC NULLS FIRST
                   partition by: (UDFToDouble(_col0) + UDFToDouble(_col1))
                   raw input shape:
                   window functions:
@@ -1991,7 +1991,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col0
+                  order by: _col0 ASC NULLS FIRST
                   partition by: _col0
                   raw input shape:
                   window functions:


[15/18] hive git commit: HIVE-12994: Implement support for NULLS FIRST/NULLS LAST (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/bucketcontext_4.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/bucketcontext_4.q.out b/ql/src/test/results/clientpositive/bucketcontext_4.q.out
index f95e0ae..652b377 100644
--- a/ql/src/test/results/clientpositive/bucketcontext_4.q.out
+++ b/ql/src/test/results/clientpositive/bucketcontext_4.q.out
@@ -288,6 +288,7 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
+                    null sort order: 
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                     tag: -1
@@ -468,6 +469,7 @@ STAGE PLANS:
                   mode: hash
                   outputColumnNames: _col0
                   Reduce Output Operator
+                    null sort order: 
                     sort order: 
                     tag: -1
                     value expressions: _col0 (type: bigint)

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/bucketcontext_5.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/bucketcontext_5.q.out b/ql/src/test/results/clientpositive/bucketcontext_5.q.out
index 031ce78..ee4532a 100644
--- a/ql/src/test/results/clientpositive/bucketcontext_5.q.out
+++ b/ql/src/test/results/clientpositive/bucketcontext_5.q.out
@@ -166,6 +166,7 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
+                    null sort order: 
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                     tag: -1
@@ -337,6 +338,7 @@ STAGE PLANS:
                   mode: hash
                   outputColumnNames: _col0
                   Reduce Output Operator
+                    null sort order: 
                     sort order: 
                     tag: -1
                     value expressions: _col0 (type: bigint)

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/bucketcontext_6.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/bucketcontext_6.q.out b/ql/src/test/results/clientpositive/bucketcontext_6.q.out
index bada9ce..e280b95 100644
--- a/ql/src/test/results/clientpositive/bucketcontext_6.q.out
+++ b/ql/src/test/results/clientpositive/bucketcontext_6.q.out
@@ -184,6 +184,7 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
+                    null sort order: 
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                     tag: -1
@@ -408,6 +409,7 @@ STAGE PLANS:
                   mode: hash
                   outputColumnNames: _col0
                   Reduce Output Operator
+                    null sort order: 
                     sort order: 
                     tag: -1
                     value expressions: _col0 (type: bigint)

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/bucketcontext_7.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/bucketcontext_7.q.out b/ql/src/test/results/clientpositive/bucketcontext_7.q.out
index f87ce5f..b44ae95 100644
--- a/ql/src/test/results/clientpositive/bucketcontext_7.q.out
+++ b/ql/src/test/results/clientpositive/bucketcontext_7.q.out
@@ -305,6 +305,7 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
+                    null sort order: 
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                     tag: -1
@@ -533,6 +534,7 @@ STAGE PLANS:
                   mode: hash
                   outputColumnNames: _col0
                   Reduce Output Operator
+                    null sort order: 
                     sort order: 
                     tag: -1
                     value expressions: _col0 (type: bigint)

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/bucketcontext_8.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/bucketcontext_8.q.out b/ql/src/test/results/clientpositive/bucketcontext_8.q.out
index daa6767..eee7b57 100644
--- a/ql/src/test/results/clientpositive/bucketcontext_8.q.out
+++ b/ql/src/test/results/clientpositive/bucketcontext_8.q.out
@@ -305,6 +305,7 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
+                    null sort order: 
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                     tag: -1
@@ -533,6 +534,7 @@ STAGE PLANS:
                   mode: hash
                   outputColumnNames: _col0
                   Reduce Output Operator
+                    null sort order: 
                     sort order: 
                     tag: -1
                     value expressions: _col0 (type: bigint)

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/bucketmapjoin10.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/bucketmapjoin10.q.out b/ql/src/test/results/clientpositive/bucketmapjoin10.q.out
index b09ccbe..041f2d0 100644
--- a/ql/src/test/results/clientpositive/bucketmapjoin10.q.out
+++ b/ql/src/test/results/clientpositive/bucketmapjoin10.q.out
@@ -329,6 +329,7 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
+                    null sort order: 
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                     tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/bucketmapjoin11.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/bucketmapjoin11.q.out b/ql/src/test/results/clientpositive/bucketmapjoin11.q.out
index 1387aa5..5e9fbe4 100644
--- a/ql/src/test/results/clientpositive/bucketmapjoin11.q.out
+++ b/ql/src/test/results/clientpositive/bucketmapjoin11.q.out
@@ -347,6 +347,7 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
+                    null sort order: 
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                     tag: -1
@@ -724,6 +725,7 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
+                    null sort order: 
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                     tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/bucketmapjoin12.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/bucketmapjoin12.q.out b/ql/src/test/results/clientpositive/bucketmapjoin12.q.out
index c5627fa..3a38d7b 100644
--- a/ql/src/test/results/clientpositive/bucketmapjoin12.q.out
+++ b/ql/src/test/results/clientpositive/bucketmapjoin12.q.out
@@ -263,6 +263,7 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
+                    null sort order: 
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                     tag: -1
@@ -534,6 +535,7 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
+                    null sort order: 
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                     tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/bucketmapjoin13.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/bucketmapjoin13.q.out b/ql/src/test/results/clientpositive/bucketmapjoin13.q.out
index b07ad4f..c78b7a1 100644
--- a/ql/src/test/results/clientpositive/bucketmapjoin13.q.out
+++ b/ql/src/test/results/clientpositive/bucketmapjoin13.q.out
@@ -218,6 +218,7 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
+                    null sort order: 
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                     tag: -1
@@ -547,6 +548,7 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
+                    null sort order: 
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                     tag: -1
@@ -827,6 +829,7 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
+                    null sort order: 
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                     tag: -1
@@ -1109,6 +1112,7 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
+                    null sort order: 
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                     tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/bucketmapjoin7.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/bucketmapjoin7.q.out b/ql/src/test/results/clientpositive/bucketmapjoin7.q.out
index 1b9eb47..7f96a23 100644
--- a/ql/src/test/results/clientpositive/bucketmapjoin7.q.out
+++ b/ql/src/test/results/clientpositive/bucketmapjoin7.q.out
@@ -126,15 +126,17 @@ TOK_QUERY
                value
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            .
-               TOK_TABLE_OR_COL
-                  a
-               key
+            TOK_NULLS_FIRST
+               .
+                  TOK_TABLE_OR_COL
+                     a
+                  key
          TOK_TABSORTCOLNAMEASC
-            .
-               TOK_TABLE_OR_COL
-                  b
-               value
+            TOK_NULLS_FIRST
+               .
+                  TOK_TABLE_OR_COL
+                     b
+                  value
       TOK_LIMIT
          1
 
@@ -245,6 +247,7 @@ STAGE PLANS:
                   Statistics: Num rows: 755 Data size: 3025 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col0 (type: int), _col1 (type: string)
+                    null sort order: aa
                     sort order: ++
                     Statistics: Num rows: 755 Data size: 3025 Basic stats: COMPLETE Column stats: NONE
                     tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/bucketmapjoin8.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/bucketmapjoin8.q.out b/ql/src/test/results/clientpositive/bucketmapjoin8.q.out
index 48c7e39..804c882 100644
--- a/ql/src/test/results/clientpositive/bucketmapjoin8.q.out
+++ b/ql/src/test/results/clientpositive/bucketmapjoin8.q.out
@@ -229,6 +229,7 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
+                    null sort order: 
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                     tag: -1
@@ -517,6 +518,7 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
+                    null sort order: 
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                     tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/bucketmapjoin9.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/bucketmapjoin9.q.out b/ql/src/test/results/clientpositive/bucketmapjoin9.q.out
index 8518b17..cc0672e 100644
--- a/ql/src/test/results/clientpositive/bucketmapjoin9.q.out
+++ b/ql/src/test/results/clientpositive/bucketmapjoin9.q.out
@@ -229,6 +229,7 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
+                    null sort order: 
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                     tag: -1
@@ -542,6 +543,7 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
+                    null sort order: 
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                     tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/cbo_rp_outer_join_ppr.q.java1.7.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/cbo_rp_outer_join_ppr.q.java1.7.out b/ql/src/test/results/clientpositive/cbo_rp_outer_join_ppr.q.java1.7.out
index 7074ccb..f680a02 100644
--- a/ql/src/test/results/clientpositive/cbo_rp_outer_join_ppr.q.java1.7.out
+++ b/ql/src/test/results/clientpositive/cbo_rp_outer_join_ppr.q.java1.7.out
@@ -124,6 +124,7 @@ STAGE PLANS:
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
                 key expressions: key (type: string)
+                null sort order: a
                 sort order: +
                 Map-reduce partition columns: key (type: string)
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -140,6 +141,7 @@ STAGE PLANS:
               Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
                 key expressions: key (type: string)
+                null sort order: a
                 sort order: +
                 Map-reduce partition columns: key (type: string)
                 Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
@@ -601,6 +603,7 @@ STAGE PLANS:
                 Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: key (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: key (type: string)
                   Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
@@ -621,6 +624,7 @@ STAGE PLANS:
                 Statistics: Num rows: 111 Data size: 1179 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: key (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: key (type: string)
                   Statistics: Num rows: 111 Data size: 1179 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/columnstats_partlvl.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/columnstats_partlvl.q.out b/ql/src/test/results/clientpositive/columnstats_partlvl.q.out
index 655847e..bfdc3b6 100644
--- a/ql/src/test/results/clientpositive/columnstats_partlvl.q.out
+++ b/ql/src/test/results/clientpositive/columnstats_partlvl.q.out
@@ -124,6 +124,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Reduce Output Operator
                   key expressions: _col0 (type: double)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: double)
                   tag: -1
@@ -322,6 +323,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Reduce Output Operator
                   key expressions: _col0 (type: double)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: double)
                   tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/columnstats_tbllvl.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/columnstats_tbllvl.q.out b/ql/src/test/results/clientpositive/columnstats_tbllvl.q.out
index cb51f5b..96cf8fd 100644
--- a/ql/src/test/results/clientpositive/columnstats_tbllvl.q.out
+++ b/ql/src/test/results/clientpositive/columnstats_tbllvl.q.out
@@ -121,6 +121,7 @@ STAGE PLANS:
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
                 Reduce Output Operator
+                  null sort order: 
                   sort order: 
                   tag: -1
                   value expressions: _col0 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:string,numbitvectors:int>), _col1 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:string,numbitvectors:int>), _col2 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,bitvector:string,numbitvectors:int>)
@@ -522,6 +523,7 @@ STAGE PLANS:
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
                 Reduce Output Operator
+                  null sort order: 
                   sort order: 
                   tag: -1
                   value expressions: _col0 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:string,numbitvectors:int>), _col1 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:string,numbitvectors:int>), _col2 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,bitvector:string,numbitvectors:int>)

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/combine2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/combine2.q.out b/ql/src/test/results/clientpositive/combine2.q.out
index 2469f09..8c59816 100644
--- a/ql/src/test/results/clientpositive/combine2.q.out
+++ b/ql/src/test/results/clientpositive/combine2.q.out
@@ -195,6 +195,7 @@ STAGE PLANS:
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
+                  null sort order: 
                   sort order: 
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/constantPropagateForSubQuery.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/constantPropagateForSubQuery.q.out b/ql/src/test/results/clientpositive/constantPropagateForSubQuery.q.out
index 57d8dca..dc2f869 100644
--- a/ql/src/test/results/clientpositive/constantPropagateForSubQuery.q.out
+++ b/ql/src/test/results/clientpositive/constantPropagateForSubQuery.q.out
@@ -92,6 +92,7 @@ STAGE PLANS:
                 outputColumnNames: _col1
                 Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
+                  null sort order: 
                   sort order: 
                   Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
                   tag: 0
@@ -106,6 +107,7 @@ STAGE PLANS:
               outputColumnNames: _col0, _col1
               Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
+                null sort order: 
                 sort order: 
                 Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
                 tag: 1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/correlationoptimizer12.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/correlationoptimizer12.q.out b/ql/src/test/results/clientpositive/correlationoptimizer12.q.out
index 615e47a..3d936e5 100644
--- a/ql/src/test/results/clientpositive/correlationoptimizer12.q.out
+++ b/ql/src/test/results/clientpositive/correlationoptimizer12.q.out
@@ -50,7 +50,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col0
+                  order by: _col0 ASC NULLS FIRST
                   partition by: _col0
                   raw input shape:
                   window functions:
@@ -135,7 +135,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col0
+                  order by: _col0 ASC NULLS FIRST
                   partition by: _col0
                   raw input shape:
                   window functions:

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/ctas.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/ctas.q.out b/ql/src/test/results/clientpositive/ctas.q.out
index bf6db75..06003ee 100644
--- a/ql/src/test/results/clientpositive/ctas.q.out
+++ b/ql/src/test/results/clientpositive/ctas.q.out
@@ -718,11 +718,13 @@ TOK_CREATETABLE
                   value
          TOK_SORTBY
             TOK_TABSORTCOLNAMEASC
-               TOK_TABLE_OR_COL
-                  key
+               TOK_NULLS_FIRST
+                  TOK_TABLE_OR_COL
+                     key
             TOK_TABSORTCOLNAMEASC
-               TOK_TABLE_OR_COL
-                  value
+               TOK_NULLS_FIRST
+                  TOK_TABLE_OR_COL
+                     value
          TOK_LIMIT
             10
 
@@ -748,6 +750,7 @@ STAGE PLANS:
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
                 key expressions: _col0 (type: string), _col1 (type: string)
+                null sort order: aa
                 sort order: ++
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                 tag: -1
@@ -837,6 +840,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col0 (type: string), _col1 (type: string)
+              null sort order: aa
               sort order: ++
               Statistics: Num rows: 10 Data size: 100 Basic stats: COMPLETE Column stats: NONE
               tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/ctas_colname.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/ctas_colname.q.out b/ql/src/test/results/clientpositive/ctas_colname.q.out
index 435b9e0..7c86c66 100644
--- a/ql/src/test/results/clientpositive/ctas_colname.q.out
+++ b/ql/src/test/results/clientpositive/ctas_colname.q.out
@@ -193,7 +193,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col0
                   raw input shape:
                   window functions:
@@ -354,7 +354,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col0
                   raw input shape:
                   window functions:

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/disable_merge_for_bucketing.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/disable_merge_for_bucketing.q.out b/ql/src/test/results/clientpositive/disable_merge_for_bucketing.q.out
index fe55531..366bd7a 100644
--- a/ql/src/test/results/clientpositive/disable_merge_for_bucketing.q.out
+++ b/ql/src/test/results/clientpositive/disable_merge_for_bucketing.q.out
@@ -49,6 +49,7 @@ STAGE PLANS:
               outputColumnNames: _col0, _col1
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
+                null sort order: 
                 sort order: 
                 Map-reduce partition columns: UDFToInteger(_col0) (type: int)
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/display_colstats_tbllvl.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/display_colstats_tbllvl.q.out b/ql/src/test/results/clientpositive/display_colstats_tbllvl.q.out
index 7154ac7..b79095c 100644
--- a/ql/src/test/results/clientpositive/display_colstats_tbllvl.q.out
+++ b/ql/src/test/results/clientpositive/display_colstats_tbllvl.q.out
@@ -137,6 +137,7 @@ STAGE PLANS:
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
                 Reduce Output Operator
+                  null sort order: 
                   sort order: 
                   tag: -1
                   value expressions: _col0 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:string,numbitvectors:int>), _col1 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:string,numbitvectors:int>), _col2 (type: struct<columntype:string,min:double,max:double,countnulls:bigint,bitvector:string,numbitvectors:int>)

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/dynpart_sort_opt_vectorization.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/dynpart_sort_opt_vectorization.q.out b/ql/src/test/results/clientpositive/dynpart_sort_opt_vectorization.q.out
index d03bfe4..c5d695e 100644
--- a/ql/src/test/results/clientpositive/dynpart_sort_opt_vectorization.q.out
+++ b/ql/src/test/results/clientpositive/dynpart_sort_opt_vectorization.q.out
@@ -1155,7 +1155,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	4                   	 
 Bucket Columns:     	[si]                	 
-Sort Columns:       	[Order(col:f, order:1)]	 
+Sort Columns:       	[Order(col:f, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: desc formatted over1k_part_buck_sort_orc partition(t="__HIVE_DEFAULT_PARTITION__")
@@ -1196,7 +1196,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	4                   	 
 Bucket Columns:     	[si]                	 
-Sort Columns:       	[Order(col:f, order:1)]	 
+Sort Columns:       	[Order(col:f, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: select count(*) from over1k_part_orc
@@ -2129,7 +2129,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[si]                	 
-Sort Columns:       	[Order(col:f, order:1)]	 
+Sort Columns:       	[Order(col:f, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: desc formatted over1k_part_buck_sort2_orc partition(t="__HIVE_DEFAULT_PARTITION__")
@@ -2170,7 +2170,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[si]                	 
-Sort Columns:       	[Order(col:f, order:1)]	 
+Sort Columns:       	[Order(col:f, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: explain select * from over1k_part_buck_sort2_orc
@@ -2339,7 +2339,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[si]                	 
-Sort Columns:       	[Order(col:f, order:1)]	 
+Sort Columns:       	[Order(col:f, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: desc formatted over1k_part_buck_sort2_orc partition(t="__HIVE_DEFAULT_PARTITION__")
@@ -2380,7 +2380,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[si]                	 
-Sort Columns:       	[Order(col:f, order:1)]	 
+Sort Columns:       	[Order(col:f, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: explain select * from over1k_part_buck_sort2_orc

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/dynpart_sort_optimization.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/dynpart_sort_optimization.q.out b/ql/src/test/results/clientpositive/dynpart_sort_optimization.q.out
index 857d609..9a222b1 100644
--- a/ql/src/test/results/clientpositive/dynpart_sort_optimization.q.out
+++ b/ql/src/test/results/clientpositive/dynpart_sort_optimization.q.out
@@ -1060,7 +1060,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	4                   	 
 Bucket Columns:     	[si]                	 
-Sort Columns:       	[Order(col:f, order:1)]	 
+Sort Columns:       	[Order(col:f, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: desc formatted over1k_part_buck_sort partition(t="__HIVE_DEFAULT_PARTITION__")
@@ -1101,7 +1101,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	4                   	 
 Bucket Columns:     	[si]                	 
-Sort Columns:       	[Order(col:f, order:1)]	 
+Sort Columns:       	[Order(col:f, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: select count(*) from over1k_part
@@ -2027,7 +2027,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[si]                	 
-Sort Columns:       	[Order(col:f, order:1)]	 
+Sort Columns:       	[Order(col:f, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: desc formatted over1k_part_buck_sort2 partition(t="__HIVE_DEFAULT_PARTITION__")
@@ -2068,7 +2068,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[si]                	 
-Sort Columns:       	[Order(col:f, order:1)]	 
+Sort Columns:       	[Order(col:f, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: select * from over1k_part_buck_sort2
@@ -2170,7 +2170,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[si]                	 
-Sort Columns:       	[Order(col:f, order:1)]	 
+Sort Columns:       	[Order(col:f, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: desc formatted over1k_part_buck_sort2 partition(t="__HIVE_DEFAULT_PARTITION__")
@@ -2211,7 +2211,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[si]                	 
-Sort Columns:       	[Order(col:f, order:1)]	 
+Sort Columns:       	[Order(col:f, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: select * from over1k_part_buck_sort2

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/encrypted/encryption_join_unencrypted_tbl.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/encrypted/encryption_join_unencrypted_tbl.q.out b/ql/src/test/results/clientpositive/encrypted/encryption_join_unencrypted_tbl.q.out
index d173e01..ecf9861 100644
--- a/ql/src/test/results/clientpositive/encrypted/encryption_join_unencrypted_tbl.q.out
+++ b/ql/src/test/results/clientpositive/encrypted/encryption_join_unencrypted_tbl.q.out
@@ -594,6 +594,7 @@ STAGE PLANS:
                 Statistics: Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: UDFToDouble(_col0) (type: double)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: UDFToDouble(_col0) (type: double)
                   Statistics: Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE
@@ -614,6 +615,7 @@ STAGE PLANS:
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: UDFToDouble(_col0) (type: double)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: UDFToDouble(_col0) (type: double)
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/encrypted/encryption_join_with_different_encryption_keys.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/encrypted/encryption_join_with_different_encryption_keys.q.out b/ql/src/test/results/clientpositive/encrypted/encryption_join_with_different_encryption_keys.q.out
index c2cc46e..d46aaea 100644
--- a/ql/src/test/results/clientpositive/encrypted/encryption_join_with_different_encryption_keys.q.out
+++ b/ql/src/test/results/clientpositive/encrypted/encryption_join_with_different_encryption_keys.q.out
@@ -114,6 +114,7 @@ STAGE PLANS:
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -134,6 +135,7 @@ STAGE PLANS:
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/explain_logical.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/explain_logical.q.out b/ql/src/test/results/clientpositive/explain_logical.q.out
index 59bd9e9..bf35cd5 100644
--- a/ql/src/test/results/clientpositive/explain_logical.q.out
+++ b/ql/src/test/results/clientpositive/explain_logical.q.out
@@ -793,10 +793,11 @@ TOK_QUERY
                value
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            .
-               TOK_TABLE_OR_COL
-                  s1
-               key
+            TOK_NULLS_FIRST
+               .
+                  TOK_TABLE_OR_COL
+                     s1
+                  key
 
 
 LOGICAL PLAN:

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/filter_join_breaktask.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/filter_join_breaktask.q.out b/ql/src/test/results/clientpositive/filter_join_breaktask.q.out
index 8ccd129..018fd5d 100644
--- a/ql/src/test/results/clientpositive/filter_join_breaktask.q.out
+++ b/ql/src/test/results/clientpositive/filter_join_breaktask.q.out
@@ -157,6 +157,7 @@ STAGE PLANS:
                 Statistics: Num rows: 25 Data size: 211 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 25 Data size: 211 Basic stats: COMPLETE Column stats: NONE
@@ -176,6 +177,7 @@ STAGE PLANS:
                 Statistics: Num rows: 25 Data size: 211 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 25 Data size: 211 Basic stats: COMPLETE Column stats: NONE
@@ -267,6 +269,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col3 (type: string)
+              null sort order: a
               sort order: +
               Map-reduce partition columns: _col3 (type: string)
               Statistics: Num rows: 27 Data size: 232 Basic stats: COMPLETE Column stats: NONE
@@ -287,6 +290,7 @@ STAGE PLANS:
                 Statistics: Num rows: 25 Data size: 211 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 25 Data size: 211 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/fouter_join_ppr.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/fouter_join_ppr.q.out b/ql/src/test/results/clientpositive/fouter_join_ppr.q.out
index d7b83a1..e5ae828 100644
--- a/ql/src/test/results/clientpositive/fouter_join_ppr.q.out
+++ b/ql/src/test/results/clientpositive/fouter_join_ppr.q.out
@@ -122,6 +122,7 @@ STAGE PLANS:
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
                 key expressions: _col0 (type: string)
+                null sort order: a
                 sort order: +
                 Map-reduce partition columns: _col0 (type: string)
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -138,6 +139,7 @@ STAGE PLANS:
               Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
                 key expressions: _col0 (type: string)
+                null sort order: a
                 sort order: +
                 Map-reduce partition columns: _col0 (type: string)
                 Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
@@ -591,6 +593,7 @@ STAGE PLANS:
               Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
                 key expressions: _col0 (type: string)
+                null sort order: a
                 sort order: +
                 Map-reduce partition columns: _col0 (type: string)
                 Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
@@ -607,6 +610,7 @@ STAGE PLANS:
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
                 key expressions: _col0 (type: string)
+                null sort order: a
                 sort order: +
                 Map-reduce partition columns: _col0 (type: string)
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -1068,6 +1072,7 @@ STAGE PLANS:
                 Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
@@ -1088,6 +1093,7 @@ STAGE PLANS:
                 Statistics: Num rows: 111 Data size: 1179 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 111 Data size: 1179 Basic stats: COMPLETE Column stats: NONE
@@ -1442,6 +1448,7 @@ STAGE PLANS:
                 Statistics: Num rows: 111 Data size: 1179 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 111 Data size: 1179 Basic stats: COMPLETE Column stats: NONE
@@ -1462,6 +1469,7 @@ STAGE PLANS:
                 Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/groupby_grouping_window.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/groupby_grouping_window.q.out b/ql/src/test/results/clientpositive/groupby_grouping_window.q.out
index 0324836..5cd9737 100644
--- a/ql/src/test/results/clientpositive/groupby_grouping_window.q.out
+++ b/ql/src/test/results/clientpositive/groupby_grouping_window.q.out
@@ -101,7 +101,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col3
+                  order by: _col3 ASC NULLS FIRST
                   partition by: _col0
                   raw input shape:
                   window functions:

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/groupby_map_ppr.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/groupby_map_ppr.q.out b/ql/src/test/results/clientpositive/groupby_map_ppr.q.out
index 9f779d0..556995c 100644
--- a/ql/src/test/results/clientpositive/groupby_map_ppr.q.out
+++ b/ql/src/test/results/clientpositive/groupby_map_ppr.q.out
@@ -120,6 +120,7 @@ STAGE PLANS:
                 Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string), _col1 (type: string)
+                  null sort order: aa
                   sort order: ++
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/groupby_map_ppr_multi_distinct.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/groupby_map_ppr_multi_distinct.q.out b/ql/src/test/results/clientpositive/groupby_map_ppr_multi_distinct.q.out
index 6d78aca..393e6fa 100644
--- a/ql/src/test/results/clientpositive/groupby_map_ppr_multi_distinct.q.out
+++ b/ql/src/test/results/clientpositive/groupby_map_ppr_multi_distinct.q.out
@@ -137,6 +137,7 @@ STAGE PLANS:
                 Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
+                  null sort order: aaa
                   sort order: +++
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/groupby_ppr.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/groupby_ppr.q.out b/ql/src/test/results/clientpositive/groupby_ppr.q.out
index 035f315..2e160b3 100644
--- a/ql/src/test/results/clientpositive/groupby_ppr.q.out
+++ b/ql/src/test/results/clientpositive/groupby_ppr.q.out
@@ -114,6 +114,7 @@ STAGE PLANS:
               Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
                 key expressions: _col0 (type: string), _col1 (type: string)
+                null sort order: aa
                 sort order: ++
                 Map-reduce partition columns: _col0 (type: string)
                 Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/groupby_ppr_multi_distinct.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/groupby_ppr_multi_distinct.q.out b/ql/src/test/results/clientpositive/groupby_ppr_multi_distinct.q.out
index a7aba12..93fbc0e 100644
--- a/ql/src/test/results/clientpositive/groupby_ppr_multi_distinct.q.out
+++ b/ql/src/test/results/clientpositive/groupby_ppr_multi_distinct.q.out
@@ -131,6 +131,7 @@ STAGE PLANS:
               Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
                 key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
+                null sort order: aaa
                 sort order: +++
                 Map-reduce partition columns: _col0 (type: string)
                 Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/groupby_resolution.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/groupby_resolution.q.out b/ql/src/test/results/clientpositive/groupby_resolution.q.out
index 83fc576..76b5238 100644
--- a/ql/src/test/results/clientpositive/groupby_resolution.q.out
+++ b/ql/src/test/results/clientpositive/groupby_resolution.q.out
@@ -681,7 +681,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: 0
                   raw input shape:
                   window functions:

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/groupby_sort_1_23.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/groupby_sort_1_23.q.out b/ql/src/test/results/clientpositive/groupby_sort_1_23.q.out
index 07c9553..3552a86 100644
--- a/ql/src/test/results/clientpositive/groupby_sort_1_23.q.out
+++ b/ql/src/test/results/clientpositive/groupby_sort_1_23.q.out
@@ -468,6 +468,7 @@ STAGE PLANS:
                 Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string), _col1 (type: string)
+                  null sort order: aa
                   sort order: ++
                   Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
                   Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
@@ -1874,6 +1875,7 @@ STAGE PLANS:
                 Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string), 1 (type: int), _col2 (type: string)
+                  null sort order: aaa
                   sort order: +++
                   Map-reduce partition columns: _col0 (type: string), 1 (type: int), _col2 (type: string)
                   Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
@@ -2095,6 +2097,7 @@ STAGE PLANS:
                 Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string), _col1 (type: double)
+                  null sort order: aa
                   sort order: ++
                   Map-reduce partition columns: _col0 (type: string), _col1 (type: double)
                   Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
@@ -2362,6 +2365,7 @@ STAGE PLANS:
                     Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: double)
+                      null sort order: a
                       sort order: +
                       Map-reduce partition columns: _col0 (type: double)
                       Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
@@ -3135,6 +3139,7 @@ STAGE PLANS:
                 Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: double)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: double)
                   Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
@@ -3774,6 +3779,7 @@ STAGE PLANS:
                   Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col0 (type: string)
+                    null sort order: a
                     sort order: +
                     Map-reduce partition columns: _col0 (type: string)
                     Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
@@ -3800,6 +3806,7 @@ STAGE PLANS:
                   Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col0 (type: string)
+                    null sort order: a
                     sort order: +
                     Map-reduce partition columns: _col0 (type: string)
                     Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
@@ -4086,6 +4093,7 @@ STAGE PLANS:
                   Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col0 (type: string), _col1 (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
                     Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
@@ -4194,6 +4202,7 @@ STAGE PLANS:
                   Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col0 (type: string)
+                    null sort order: a
                     sort order: +
                     Map-reduce partition columns: _col0 (type: string)
                     Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
@@ -4204,6 +4213,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col0 (type: string)
+              null sort order: a
               sort order: +
               Map-reduce partition columns: _col0 (type: string)
               Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
@@ -4406,6 +4416,7 @@ STAGE PLANS:
                 Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/groupby_sort_6.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/groupby_sort_6.q.out b/ql/src/test/results/clientpositive/groupby_sort_6.q.out
index 9738c15..844a2a4 100644
--- a/ql/src/test/results/clientpositive/groupby_sort_6.q.out
+++ b/ql/src/test/results/clientpositive/groupby_sort_6.q.out
@@ -87,6 +87,7 @@ STAGE PLANS:
                   Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col0 (type: string)
+                    null sort order: a
                     sort order: +
                     Map-reduce partition columns: _col0 (type: string)
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -256,6 +257,7 @@ STAGE PLANS:
                   Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col0 (type: string)
+                    null sort order: a
                     sort order: +
                     Map-reduce partition columns: _col0 (type: string)
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
@@ -424,6 +426,7 @@ STAGE PLANS:
                 Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 1 Data size: 30 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/groupby_sort_skew_1_23.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/groupby_sort_skew_1_23.q.out b/ql/src/test/results/clientpositive/groupby_sort_skew_1_23.q.out
index fdc114c..73fca7c 100644
--- a/ql/src/test/results/clientpositive/groupby_sort_skew_1_23.q.out
+++ b/ql/src/test/results/clientpositive/groupby_sort_skew_1_23.q.out
@@ -469,6 +469,7 @@ STAGE PLANS:
                 Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string), _col1 (type: string)
+                  null sort order: aa
                   sort order: ++
                   Map-reduce partition columns: rand() (type: double)
                   Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
@@ -561,6 +562,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col0 (type: string), _col1 (type: string)
+              null sort order: aa
               sort order: ++
               Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
               Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
@@ -1940,6 +1942,7 @@ STAGE PLANS:
                 Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string), 1 (type: int), _col2 (type: string)
+                  null sort order: aaa
                   sort order: +++
                   Map-reduce partition columns: rand() (type: double)
                   Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
@@ -2032,6 +2035,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col0 (type: string), 1 (type: int), _col2 (type: string)
+              null sort order: aaa
               sort order: +++
               Map-reduce partition columns: _col0 (type: string), 1 (type: int), _col2 (type: string)
               Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
@@ -2226,6 +2230,7 @@ STAGE PLANS:
                 Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string), _col1 (type: double)
+                  null sort order: aa
                   sort order: ++
                   Map-reduce partition columns: rand() (type: double)
                   Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
@@ -2318,6 +2323,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col0 (type: string), _col1 (type: double)
+              null sort order: aa
               sort order: ++
               Map-reduce partition columns: _col0 (type: string), _col1 (type: double)
               Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
@@ -2558,6 +2564,7 @@ STAGE PLANS:
                     Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: double)
+                      null sort order: a
                       sort order: +
                       Map-reduce partition columns: rand() (type: double)
                       Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
@@ -2650,6 +2657,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col0 (type: double)
+              null sort order: a
               sort order: +
               Map-reduce partition columns: _col0 (type: double)
               Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
@@ -3396,6 +3404,7 @@ STAGE PLANS:
                 Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: double)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: rand() (type: double)
                   Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
@@ -3488,6 +3497,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col0 (type: double)
+              null sort order: a
               sort order: +
               Map-reduce partition columns: _col0 (type: double)
               Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
@@ -4099,6 +4109,7 @@ STAGE PLANS:
                   Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col0 (type: string)
+                    null sort order: a
                     sort order: +
                     Map-reduce partition columns: _col0 (type: string)
                     Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
@@ -4125,6 +4136,7 @@ STAGE PLANS:
                   Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col0 (type: string)
+                    null sort order: a
                     sort order: +
                     Map-reduce partition columns: _col0 (type: string)
                     Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
@@ -4412,6 +4424,7 @@ STAGE PLANS:
                   Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col0 (type: string), _col1 (type: string)
+                    null sort order: aa
                     sort order: ++
                     Map-reduce partition columns: rand() (type: double)
                     Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
@@ -4504,6 +4517,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col0 (type: string), _col1 (type: string)
+              null sort order: aa
               sort order: ++
               Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
               Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
@@ -4584,6 +4598,7 @@ STAGE PLANS:
                   Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col0 (type: string)
+                    null sort order: a
                     sort order: +
                     Map-reduce partition columns: _col0 (type: string)
                     Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
@@ -4594,6 +4609,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col0 (type: string)
+              null sort order: a
               sort order: +
               Map-reduce partition columns: _col0 (type: string)
               Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
@@ -4797,6 +4813,7 @@ STAGE PLANS:
                 Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: rand() (type: double)
                   Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE
@@ -4889,6 +4906,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col0 (type: string)
+              null sort order: a
               sort order: +
               Map-reduce partition columns: _col0 (type: string)
               Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/index_skewtable.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/index_skewtable.q.out b/ql/src/test/results/clientpositive/index_skewtable.q.out
index 972789d..45e6834 100644
--- a/ql/src/test/results/clientpositive/index_skewtable.q.out
+++ b/ql/src/test/results/clientpositive/index_skewtable.q.out
@@ -57,7 +57,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	-1                  	 
 Bucket Columns:     	[]                  	 
-Sort Columns:       	[Order(col:value, order:1)]	 
+Sort Columns:       	[Order(col:value, order:1, nullOrder:0)]	 
 PREHOOK: query: ALTER INDEX kv_index ON kv REBUILD
 PREHOOK: type: ALTERINDEX_REBUILD
 PREHOOK: Input: default@kv

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/infer_bucket_sort.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/infer_bucket_sort.q.out b/ql/src/test/results/clientpositive/infer_bucket_sort.q.out
index 182dd3a..709ca10 100644
--- a/ql/src/test/results/clientpositive/infer_bucket_sort.q.out
+++ b/ql/src/test/results/clientpositive/infer_bucket_sort.q.out
@@ -62,7 +62,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[key]               	 
-Sort Columns:       	[Order(col:key, order:1)]	 
+Sort Columns:       	[Order(col:key, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: -- Test group by where a key isn't selected, should not be bucketed or sorted
@@ -168,7 +168,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[key]               	 
-Sort Columns:       	[Order(col:key, order:1)]	 
+Sort Columns:       	[Order(col:key, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: -- Test join with two keys, should be bucketed and sorted by join keys
@@ -221,7 +221,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[key, value]        	 
-Sort Columns:       	[Order(col:key, order:1), Order(col:value, order:1)]	 
+Sort Columns:       	[Order(col:key, order:1, nullOrder:0), Order(col:value, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: -- Test join with two keys and only one selected, should not be bucketed or sorted
@@ -327,7 +327,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[key]               	 
-Sort Columns:       	[Order(col:key, order:1)]	 
+Sort Columns:       	[Order(col:key, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: -- Test join on three tables on different keys, should be bucketed and sorted by latter key
@@ -380,7 +380,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[key]               	 
-Sort Columns:       	[Order(col:key, order:1)]	 
+Sort Columns:       	[Order(col:key, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: -- Test distribute by, should only be bucketed by key
@@ -486,7 +486,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	-1                  	 
 Bucket Columns:     	[]                  	 
-Sort Columns:       	[Order(col:key, order:1)]	 
+Sort Columns:       	[Order(col:key, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: -- Test sort by desc, should be sorted by key
@@ -539,7 +539,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	-1                  	 
 Bucket Columns:     	[]                  	 
-Sort Columns:       	[Order(col:key, order:0)]	 
+Sort Columns:       	[Order(col:key, order:0, nullOrder:1)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: -- Test cluster by, should be bucketed and sorted by key
@@ -592,7 +592,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[key]               	 
-Sort Columns:       	[Order(col:key, order:1)]	 
+Sort Columns:       	[Order(col:key, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: -- Test distribute by and sort by different keys, should be bucketed by one key sorted by the other
@@ -645,7 +645,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[key]               	 
-Sort Columns:       	[Order(col:value, order:1)]	 
+Sort Columns:       	[Order(col:value, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: -- Test join in simple subquery, should be bucketed and sorted on key
@@ -698,7 +698,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[key]               	 
-Sort Columns:       	[Order(col:key, order:1)]	 
+Sort Columns:       	[Order(col:key, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: -- Test join in simple subquery renaming key column, should be bucketed and sorted on key
@@ -751,7 +751,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[key]               	 
-Sort Columns:       	[Order(col:key, order:1)]	 
+Sort Columns:       	[Order(col:key, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: -- Test group by in simple subquery, should be bucketed and sorted on key
@@ -804,7 +804,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[key]               	 
-Sort Columns:       	[Order(col:key, order:1)]	 
+Sort Columns:       	[Order(col:key, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: -- Test group by in simple subquery renaming key column, should be bucketed and sorted on key
@@ -857,7 +857,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[key]               	 
-Sort Columns:       	[Order(col:key, order:1)]	 
+Sort Columns:       	[Order(col:key, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: -- Test group by in subquery with where outside, should still be bucketed and sorted on key
@@ -910,7 +910,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[key]               	 
-Sort Columns:       	[Order(col:key, order:1)]	 
+Sort Columns:       	[Order(col:key, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: -- Test group by in subquery with expression on value, should still be bucketed and sorted on key
@@ -963,7 +963,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[key]               	 
-Sort Columns:       	[Order(col:key, order:1)]	 
+Sort Columns:       	[Order(col:key, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: -- Test group by in subquery with lateral view outside, should still be bucketed and sorted
@@ -1016,7 +1016,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[key]               	 
-Sort Columns:       	[Order(col:key, order:1)]	 
+Sort Columns:       	[Order(col:key, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: -- Test group by in subquery with another group by outside, should be bucketed and sorted by the
@@ -1071,7 +1071,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[value]             	 
-Sort Columns:       	[Order(col:value, order:1)]	 
+Sort Columns:       	[Order(col:value, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: -- Test group by in subquery with select on outside reordering the columns, should be bucketed and
@@ -1126,7 +1126,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[value]             	 
-Sort Columns:       	[Order(col:value, order:1)]	 
+Sort Columns:       	[Order(col:value, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: -- Test group by in subquery followed by distribute by, should only be bucketed by the distribute key
@@ -1179,7 +1179,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[key]               	 
-Sort Columns:       	[Order(col:key, order:1)]	 
+Sort Columns:       	[Order(col:key, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: -- Test group by in subquery followed by sort by, should only be sorted by the sort key
@@ -1232,7 +1232,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[key]               	 
-Sort Columns:       	[Order(col:key, order:1)]	 
+Sort Columns:       	[Order(col:key, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: -- Test group by in subquery followed by transform script, should not be bucketed or sorted
@@ -1338,6 +1338,6 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[key, value]        	 
-Sort Columns:       	[Order(col:key, order:1), Order(col:value, order:1)]	 
+Sort Columns:       	[Order(col:key, order:1, nullOrder:0), Order(col:value, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/infer_bucket_sort_bucketed_table.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/infer_bucket_sort_bucketed_table.q.out b/ql/src/test/results/clientpositive/infer_bucket_sort_bucketed_table.q.out
index 33d795b..6db9428 100644
--- a/ql/src/test/results/clientpositive/infer_bucket_sort_bucketed_table.q.out
+++ b/ql/src/test/results/clientpositive/infer_bucket_sort_bucketed_table.q.out
@@ -64,7 +64,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	3                   	 
 Bucket Columns:     	[value]             	 
-Sort Columns:       	[Order(col:value, order:1)]	 
+Sort Columns:       	[Order(col:value, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: -- If the count(*) from sampling the buckets matches the count(*) from each file, the table is

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/infer_bucket_sort_convert_join.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/infer_bucket_sort_convert_join.q.out b/ql/src/test/results/clientpositive/infer_bucket_sort_convert_join.q.out
index 2f7e538..e5626a4 100644
--- a/ql/src/test/results/clientpositive/infer_bucket_sort_convert_join.q.out
+++ b/ql/src/test/results/clientpositive/infer_bucket_sort_convert_join.q.out
@@ -123,6 +123,6 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[key]               	 
-Sort Columns:       	[Order(col:key, order:1)]	 
+Sort Columns:       	[Order(col:key, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/infer_bucket_sort_dyn_part.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/infer_bucket_sort_dyn_part.q.out b/ql/src/test/results/clientpositive/infer_bucket_sort_dyn_part.q.out
index 7e3b48f..5f983c7 100644
--- a/ql/src/test/results/clientpositive/infer_bucket_sort_dyn_part.q.out
+++ b/ql/src/test/results/clientpositive/infer_bucket_sort_dyn_part.q.out
@@ -302,7 +302,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.RCFileOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[key]               	 
-Sort Columns:       	[Order(col:key, order:1)]	 
+Sort Columns:       	[Order(col:key, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: DESCRIBE FORMATTED test_table PARTITION (ds='2008-04-08', hr='12')
@@ -342,7 +342,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.RCFileOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[key]               	 
-Sort Columns:       	[Order(col:key, order:1)]	 
+Sort Columns:       	[Order(col:key, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: CREATE TABLE srcpart_merge_dp LIKE srcpart

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/infer_bucket_sort_grouping_operators.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/infer_bucket_sort_grouping_operators.q.out b/ql/src/test/results/clientpositive/infer_bucket_sort_grouping_operators.q.out
index ebfce60..90a71ae 100644
--- a/ql/src/test/results/clientpositive/infer_bucket_sort_grouping_operators.q.out
+++ b/ql/src/test/results/clientpositive/infer_bucket_sort_grouping_operators.q.out
@@ -144,7 +144,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[key, value]        	 
-Sort Columns:       	[Order(col:key, order:1), Order(col:value, order:1)]	 
+Sort Columns:       	[Order(col:key, order:1, nullOrder:0), Order(col:value, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: -- Test rollup, should be bucketed and sorted on key, value, grouping_key
@@ -203,7 +203,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[key, value, grouping_key]	 
-Sort Columns:       	[Order(col:key, order:1), Order(col:value, order:1), Order(col:grouping_key, order:1)]	 
+Sort Columns:       	[Order(col:key, order:1, nullOrder:0), Order(col:value, order:1, nullOrder:0), Order(col:grouping_key, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: -- Test cube, should not be bucketed or sorted because its missing the grouping ID
@@ -328,7 +328,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[key, value]        	 
-Sort Columns:       	[Order(col:key, order:1), Order(col:value, order:1)]	 
+Sort Columns:       	[Order(col:key, order:1, nullOrder:0), Order(col:value, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: -- Test cube, should be bucketed and sorted on key, value, grouping_key
@@ -387,7 +387,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[key, value, grouping_key]	 
-Sort Columns:       	[Order(col:key, order:1), Order(col:value, order:1), Order(col:grouping_key, order:1)]	 
+Sort Columns:       	[Order(col:key, order:1, nullOrder:0), Order(col:value, order:1, nullOrder:0), Order(col:grouping_key, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: -- Test grouping sets, should not be bucketed or sorted because its missing the grouping ID
@@ -512,7 +512,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[key, value]        	 
-Sort Columns:       	[Order(col:key, order:1), Order(col:value, order:1)]	 
+Sort Columns:       	[Order(col:key, order:1, nullOrder:0), Order(col:value, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: -- Test grouping sets, should be bucketed and sorted on key, value, grouping_key
@@ -571,6 +571,6 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	1                   	 
 Bucket Columns:     	[key, value, grouping_key]	 
-Sort Columns:       	[Order(col:key, order:1), Order(col:value, order:1), Order(col:grouping_key, order:1)]	 
+Sort Columns:       	[Order(col:key, order:1, nullOrder:0), Order(col:value, order:1, nullOrder:0), Order(col:grouping_key, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   


[02/18] hive git commit: HIVE-12994: Implement support for NULLS FIRST/NULLS LAST (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/vectorized_ptf.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vectorized_ptf.q.out b/ql/src/test/results/clientpositive/vectorized_ptf.q.out
index 479953c..3b17591 100644
--- a/ql/src/test/results/clientpositive/vectorized_ptf.q.out
+++ b/ql/src/test/results/clientpositive/vectorized_ptf.q.out
@@ -157,8 +157,9 @@ TOK_QUERY
                   p_mfgr
             TOK_ORDERBY
                TOK_TABSORTCOLNAMEASC
-                  TOK_TABLE_OR_COL
-                     p_name
+                  TOK_NULLS_FIRST
+                     TOK_TABLE_OR_COL
+                        p_name
    TOK_INSERT
       TOK_DESTINATION
          TOK_DIR
@@ -183,8 +184,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             r
          TOK_SELEXPR
             TOK_FUNCTION
@@ -196,8 +198,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             dr
          TOK_SELEXPR
             TOK_FUNCTION
@@ -211,8 +214,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
                   TOK_WINDOWRANGE
                      preceding
                         unbounded
@@ -235,6 +239,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: p_mfgr (type: string), p_name (type: string)
+              null sort order: aa
               sort order: ++
               Map-reduce partition columns: p_mfgr (type: string)
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -306,7 +311,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noop
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int, _col7: double
                   partition by: _col2
                   raw input shape:
@@ -336,6 +341,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col2 (type: string), _col1 (type: string)
+              null sort order: aa
               sort order: ++
               Map-reduce partition columns: _col2 (type: string)
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -382,7 +388,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:
@@ -551,10 +557,11 @@ TOK_QUERY
                   p_mfgr
             TOK_SORTBY
                TOK_TABSORTCOLNAMEASC
-                  .
-                     TOK_TABLE_OR_COL
-                        j
-                     p_name
+                  TOK_NULLS_FIRST
+                     .
+                        TOK_TABLE_OR_COL
+                           j
+                        p_name
    TOK_INSERT
       TOK_DESTINATION
          TOK_DIR
@@ -587,8 +594,9 @@ TOK_QUERY
                               p_mfgr
                         TOK_ORDERBY
                            TOK_TABSORTCOLNAMEASC
-                              TOK_TABLE_OR_COL
-                                 p_name
+                              TOK_NULLS_FIRST
+                                 TOK_TABLE_OR_COL
+                                    p_name
             deltaSz
 
 
@@ -612,6 +620,7 @@ STAGE PLANS:
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
                 key expressions: p_partkey (type: int)
+                null sort order: a
                 sort order: +
                 Map-reduce partition columns: p_partkey (type: int)
                 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -628,6 +637,7 @@ STAGE PLANS:
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
                 key expressions: p_partkey (type: int)
+                null sort order: a
                 sort order: +
                 Map-reduce partition columns: p_partkey (type: int)
                 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -717,6 +727,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col2 (type: string), _col1 (type: string)
+              null sort order: aa
               sort order: ++
               Map-reduce partition columns: _col2 (type: string)
               Statistics: Num rows: 28 Data size: 17646 Basic stats: COMPLETE Column stats: NONE
@@ -763,7 +774,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noop
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2
                   raw input shape:
@@ -793,6 +804,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col2 (type: string), _col1 (type: string)
+              null sort order: aa
               sort order: ++
               Map-reduce partition columns: _col2 (type: string)
               Statistics: Num rows: 28 Data size: 17646 Basic stats: COMPLETE Column stats: NONE
@@ -839,7 +851,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:
@@ -959,8 +971,9 @@ TOK_QUERY
                   p_mfgr
             TOK_ORDERBY
                TOK_TABSORTCOLNAMEASC
-                  TOK_TABLE_OR_COL
-                     p_name
+                  TOK_NULLS_FIRST
+                     TOK_TABLE_OR_COL
+                        p_name
    TOK_INSERT
       TOK_DESTINATION
          TOK_DIR
@@ -991,6 +1004,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: p_mfgr (type: string), p_name (type: string)
+              null sort order: aa
               sort order: ++
               Map-reduce partition columns: p_mfgr (type: string)
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -1062,7 +1076,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noop
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2
                   raw input shape:
@@ -1180,8 +1194,9 @@ TOK_QUERY
                   p_mfgr
             TOK_ORDERBY
                TOK_TABSORTCOLNAMEASC
-                  TOK_TABLE_OR_COL
-                     p_name
+                  TOK_NULLS_FIRST
+                     TOK_TABLE_OR_COL
+                        p_name
    TOK_INSERT
       TOK_DESTINATION
          TOK_DIR
@@ -1206,8 +1221,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             r
          TOK_SELEXPR
             TOK_FUNCTION
@@ -1219,8 +1235,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             dr
          TOK_SELEXPR
             TOK_FUNCTION
@@ -1234,8 +1251,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
                   TOK_WINDOWRANGE
                      preceding
                         unbounded
@@ -1258,6 +1276,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: p_mfgr (type: string), p_name (type: string)
+              null sort order: aa
               sort order: ++
               Map-reduce partition columns: p_mfgr (type: string)
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -1329,7 +1348,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: abc
                   name: noop
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int, _col7: double
                   partition by: _col2
                   raw input shape:
@@ -1359,6 +1378,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col2 (type: string), _col1 (type: string)
+              null sort order: aa
               sort order: ++
               Map-reduce partition columns: _col2 (type: string)
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -1405,7 +1425,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:
@@ -1550,8 +1570,9 @@ TOK_QUERY
                   p_mfgr
             TOK_ORDERBY
                TOK_TABSORTCOLNAMEASC
-                  TOK_TABLE_OR_COL
-                     p_name
+                  TOK_NULLS_FIRST
+                     TOK_TABLE_OR_COL
+                        p_name
    TOK_INSERT
       TOK_DESTINATION
          TOK_DIR
@@ -1576,8 +1597,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             r
          TOK_SELEXPR
             TOK_FUNCTION
@@ -1589,8 +1611,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             dr
          TOK_SELEXPR
             TOK_TABLE_OR_COL
@@ -1613,8 +1636,9 @@ TOK_QUERY
                               p_mfgr
                         TOK_ORDERBY
                            TOK_TABSORTCOLNAMEASC
-                              TOK_TABLE_OR_COL
-                                 p_name
+                              TOK_NULLS_FIRST
+                                 TOK_TABLE_OR_COL
+                                    p_name
             deltaSz
 
 
@@ -1633,6 +1657,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: p_mfgr (type: string), p_name (type: string)
+              null sort order: aa
               sort order: ++
               Map-reduce partition columns: p_mfgr (type: string)
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -1704,7 +1729,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noop
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2
                   raw input shape:
@@ -1734,6 +1759,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col2 (type: string), _col1 (type: string)
+              null sort order: aa
               sort order: ++
               Map-reduce partition columns: _col2 (type: string)
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -1780,7 +1806,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:
@@ -1928,8 +1954,9 @@ TOK_QUERY
                   p_mfgr
             TOK_ORDERBY
                TOK_TABSORTCOLNAMEASC
-                  TOK_TABLE_OR_COL
-                     p_name
+                  TOK_NULLS_FIRST
+                     TOK_TABLE_OR_COL
+                        p_name
    TOK_INSERT
       TOK_DESTINATION
          TOK_DIR
@@ -1954,8 +1981,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             r
          TOK_SELEXPR
             TOK_FUNCTION
@@ -1967,8 +1995,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             dr
          TOK_SELEXPR
             TOK_TABLE_OR_COL
@@ -1991,8 +2020,9 @@ TOK_QUERY
                               p_mfgr
                         TOK_ORDERBY
                            TOK_TABSORTCOLNAMEASC
-                              TOK_TABLE_OR_COL
-                                 p_name
+                              TOK_NULLS_FIRST
+                                 TOK_TABLE_OR_COL
+                                    p_name
             deltaSz
       TOK_GROUPBY
          TOK_TABLE_OR_COL
@@ -2018,6 +2048,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: p_mfgr (type: string), p_name (type: string)
+              null sort order: aa
               sort order: ++
               Map-reduce partition columns: p_mfgr (type: string)
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -2089,7 +2120,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noop
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2
                   raw input shape:
@@ -2128,6 +2159,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: int)
+              null sort order: aaa
               sort order: +++
               Map-reduce partition columns: _col0 (type: string)
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -2174,7 +2206,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col0
                   raw input shape:
                   window functions:
@@ -2317,8 +2349,9 @@ TOK_QUERY
                      p_mfgr
                TOK_ORDERBY
                   TOK_TABSORTCOLNAMEASC
-                     TOK_TABLE_OR_COL
-                        p_name
+                     TOK_NULLS_FIRST
+                        TOK_TABLE_OR_COL
+                           p_name
          TOK_TABREF
             TOK_TABNAME
                part_orc
@@ -2358,6 +2391,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: p_mfgr (type: string), p_name (type: string)
+              null sort order: aa
               sort order: ++
               Map-reduce partition columns: p_mfgr (type: string)
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -2429,7 +2463,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: abc
                   name: noop
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   output shape: _col0: int, _col1: string, _col2: string, _col3: string, _col4: string, _col5: int, _col6: string, _col7: double, _col8: string
                   partition by: _col2
                   raw input shape:
@@ -2463,6 +2497,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col0 (type: int)
+              null sort order: a
               sort order: +
               Map-reduce partition columns: _col0 (type: int)
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -2479,6 +2514,7 @@ STAGE PLANS:
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
                 key expressions: p_partkey (type: int)
+                null sort order: a
                 sort order: +
                 Map-reduce partition columns: p_partkey (type: int)
                 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -2676,8 +2712,9 @@ TOK_QUERY
                      p_mfgr
                TOK_ORDERBY
                   TOK_TABSORTCOLNAMEASC
-                     TOK_TABLE_OR_COL
-                        p_name
+                     TOK_NULLS_FIRST
+                        TOK_TABLE_OR_COL
+                           p_name
          =
             .
                TOK_TABLE_OR_COL
@@ -2713,6 +2750,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: p_mfgr (type: string), p_name (type: string)
+              null sort order: aa
               sort order: ++
               Map-reduce partition columns: p_mfgr (type: string)
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -2784,7 +2822,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: abc
                   name: noop
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   output shape: _col0: int, _col1: string, _col2: string, _col3: string, _col4: string, _col5: int, _col6: string, _col7: double, _col8: string
                   partition by: _col2
                   raw input shape:
@@ -2824,6 +2862,7 @@ STAGE PLANS:
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
                 key expressions: p_partkey (type: int)
+                null sort order: a
                 sort order: +
                 Map-reduce partition columns: p_partkey (type: int)
                 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -2833,6 +2872,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col0 (type: int)
+              null sort order: a
               sort order: +
               Map-reduce partition columns: _col0 (type: int)
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -3029,11 +3069,13 @@ TOK_QUERY
                   p_mfgr
             TOK_ORDERBY
                TOK_TABSORTCOLNAMEASC
-                  TOK_TABLE_OR_COL
-                     p_name
+                  TOK_NULLS_FIRST
+                     TOK_TABLE_OR_COL
+                        p_name
                TOK_TABSORTCOLNAMEDESC
-                  TOK_TABLE_OR_COL
-                     p_size
+                  TOK_NULLS_LAST
+                     TOK_TABLE_OR_COL
+                        p_size
    TOK_INSERT
       TOK_DESTINATION
          TOK_DIR
@@ -3058,11 +3100,13 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
                         TOK_TABSORTCOLNAMEDESC
-                           TOK_TABLE_OR_COL
-                              p_size
+                           TOK_NULLS_LAST
+                              TOK_TABLE_OR_COL
+                                 p_size
             r
 
 
@@ -3088,7 +3132,7 @@ STAGE PLANS:
                   Partition table definition
                     input alias: ptf_1
                     name: noopwithmap
-                    order by: p_name, p_size(DESC)
+                    order by: p_name ASC NULLS FIRST, p_size DESC NULLS LAST
                     output shape: p_name: string, p_mfgr: string, p_size: int
                     partition by: p_mfgr
                     raw input shape:
@@ -3097,6 +3141,7 @@ STAGE PLANS:
               Map-side function: true
               Reduce Output Operator
                 key expressions: p_mfgr (type: string), p_name (type: string), p_size (type: int)
+                null sort order: aaz
                 sort order: ++-
                 Map-reduce partition columns: p_mfgr (type: string)
                 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -3166,7 +3211,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noopwithmap
-                  order by: _col1, _col5(DESC)
+                  order by: _col1 ASC NULLS FIRST, _col5 DESC NULLS LAST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2
                   raw input shape:
@@ -3197,6 +3242,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col2 (type: string), _col1 (type: string), _col5 (type: int)
+              null sort order: aaz
               sort order: ++-
               Map-reduce partition columns: _col2 (type: string)
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -3242,7 +3288,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1, _col5(DESC)
+                  order by: _col1 ASC NULLS FIRST, _col5 DESC NULLS LAST
                   partition by: _col2
                   raw input shape:
                   window functions:
@@ -3366,8 +3412,9 @@ TOK_QUERY
                   p_mfgr
             TOK_ORDERBY
                TOK_TABSORTCOLNAMEASC
-                  TOK_TABLE_OR_COL
-                     p_name
+                  TOK_NULLS_FIRST
+                     TOK_TABLE_OR_COL
+                        p_name
    TOK_INSERT
       TOK_DESTINATION
          TOK_DIR
@@ -3392,8 +3439,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             r
          TOK_SELEXPR
             TOK_FUNCTION
@@ -3405,8 +3453,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             dr
          TOK_SELEXPR
             TOK_FUNCTION
@@ -3420,8 +3469,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
                   TOK_WINDOWRANGE
                      preceding
                         unbounded
@@ -3451,7 +3501,7 @@ STAGE PLANS:
                   Partition table definition
                     input alias: ptf_1
                     name: noopwithmap
-                    order by: p_name
+                    order by: p_name ASC NULLS FIRST
                     output shape: p_name: string, p_mfgr: string, p_size: int, p_retailprice: double
                     partition by: p_mfgr
                     raw input shape:
@@ -3460,6 +3510,7 @@ STAGE PLANS:
               Map-side function: true
               Reduce Output Operator
                 key expressions: p_mfgr (type: string), p_name (type: string)
+                null sort order: aa
                 sort order: ++
                 Map-reduce partition columns: p_mfgr (type: string)
                 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -3530,7 +3581,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noopwithmap
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int, _col7: double
                   partition by: _col2
                   raw input shape:
@@ -3561,6 +3612,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col2 (type: string), _col1 (type: string)
+              null sort order: aa
               sort order: ++
               Map-reduce partition columns: _col2 (type: string)
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -3607,7 +3659,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:
@@ -3748,8 +3800,9 @@ TOK_QUERY
                   p_mfgr
             TOK_ORDERBY
                TOK_TABSORTCOLNAMEASC
-                  TOK_TABLE_OR_COL
-                     p_name
+                  TOK_NULLS_FIRST
+                     TOK_TABLE_OR_COL
+                        p_name
    TOK_INSERT
       TOK_DESTINATION
          TOK_DIR
@@ -3774,8 +3827,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             r
          TOK_SELEXPR
             TOK_FUNCTION
@@ -3787,8 +3841,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             dr
          TOK_SELEXPR
             TOK_FUNCTION
@@ -3802,8 +3857,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
                   TOK_WINDOWRANGE
                      preceding
                         unbounded
@@ -3826,6 +3882,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: p_mfgr (type: string), p_name (type: string)
+              null sort order: aa
               sort order: ++
               Map-reduce partition columns: p_mfgr (type: string)
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -3897,7 +3954,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noop
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int, _col7: double
                   partition by: _col2
                   raw input shape:
@@ -3927,6 +3984,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col2 (type: string), _col1 (type: string)
+              null sort order: aa
               sort order: ++
               Map-reduce partition columns: _col2 (type: string)
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -3973,7 +4031,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:
@@ -4120,11 +4178,13 @@ TOK_QUERY
                         p_mfgr
                   TOK_ORDERBY
                      TOK_TABSORTCOLNAMEASC
-                        TOK_TABLE_OR_COL
-                           p_mfgr
+                        TOK_NULLS_FIRST
+                           TOK_TABLE_OR_COL
+                              p_mfgr
                      TOK_TABSORTCOLNAMEASC
-                        TOK_TABLE_OR_COL
-                           p_name
+                        TOK_NULLS_FIRST
+                           TOK_TABLE_OR_COL
+                              p_name
    TOK_INSERT
       TOK_DESTINATION
          TOK_DIR
@@ -4149,8 +4209,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             r
          TOK_SELEXPR
             TOK_FUNCTION
@@ -4162,8 +4223,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             dr
          TOK_SELEXPR
             TOK_FUNCTION
@@ -4177,8 +4239,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
                   TOK_WINDOWRANGE
                      preceding
                         unbounded
@@ -4202,6 +4265,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: p_mfgr (type: string), p_name (type: string)
+              null sort order: aa
               sort order: ++
               Map-reduce partition columns: p_mfgr (type: string)
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -4273,7 +4337,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noop
-                  order by: _col2, _col1
+                  order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int, _col7: double
                   partition by: _col2
                   raw input shape:
@@ -4287,7 +4351,7 @@ STAGE PLANS:
                   Partition table definition
                     input alias: ptf_1
                     name: noopwithmap
-                    order by: _col2, _col1
+                    order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                     output shape: _col1: string, _col2: string, _col5: int, _col7: double
                     partition by: _col2
                     raw input shape:
@@ -4295,7 +4359,7 @@ STAGE PLANS:
                   Partition table definition
                     input alias: ptf_2
                     name: noop
-                    order by: _col2, _col1
+                    order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                     output shape: _col1: string, _col2: string, _col5: int, _col7: double
                     partition by: _col2
                     raw input shape:
@@ -4326,6 +4390,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col2 (type: string), _col1 (type: string)
+              null sort order: aa
               sort order: ++
               Map-reduce partition columns: _col2 (type: string)
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -4372,7 +4437,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noopwithmap
-                  order by: _col2, _col1
+                  order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int, _col7: double
                   partition by: _col2
                   raw input shape:
@@ -4380,7 +4445,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_2
                   name: noop
-                  order by: _col2, _col1
+                  order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int, _col7: double
                   partition by: _col2
                   raw input shape:
@@ -4410,6 +4475,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col2 (type: string), _col1 (type: string)
+              null sort order: aa
               sort order: ++
               Map-reduce partition columns: _col2 (type: string)
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -4456,7 +4522,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:
@@ -4610,8 +4676,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             TOK_INSERT
                TOK_DESTINATION
                   TOK_DIR
@@ -4635,8 +4702,9 @@ TOK_QUERY
                                     p_mfgr
                               TOK_ORDERBY
                                  TOK_TABSORTCOLNAMEASC
-                                    TOK_TABLE_OR_COL
-                                       p_name
+                                    TOK_NULLS_FIRST
+                                       TOK_TABLE_OR_COL
+                                          p_name
                      cd
                   TOK_SELEXPR
                      TOK_TABLE_OR_COL
@@ -4659,8 +4727,9 @@ TOK_QUERY
                                  p_mfgr
                            TOK_ORDERBY
                               TOK_TABSORTCOLNAMEASC
-                                 TOK_TABLE_OR_COL
-                                    p_name
+                                 TOK_NULLS_FIRST
+                                    TOK_TABLE_OR_COL
+                                       p_name
                         TOK_WINDOWRANGE
                            preceding
                               2
@@ -4705,6 +4774,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: p_mfgr (type: string), p_name (type: string)
+              null sort order: aa
               sort order: ++
               Map-reduce partition columns: p_mfgr (type: string)
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -4776,7 +4846,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noop
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int, _col7: double
                   partition by: _col2
                   raw input shape:
@@ -4806,6 +4876,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col2 (type: string), _col1 (type: string)
+              null sort order: aa
               sort order: ++
               Map-reduce partition columns: _col2 (type: string)
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -4852,7 +4923,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:
@@ -5001,8 +5072,9 @@ TOK_QUERY
                      p_mfgr
                TOK_ORDERBY
                   TOK_TABSORTCOLNAMEASC
-                     TOK_TABLE_OR_COL
-                        p_name
+                     TOK_NULLS_FIRST
+                        TOK_TABLE_OR_COL
+                           p_name
          TOK_TABREF
             TOK_TABNAME
                part_orc
@@ -5043,10 +5115,11 @@ TOK_QUERY
                            p_mfgr
                      TOK_SORTBY
                         TOK_TABSORTCOLNAMEASC
-                           .
-                              TOK_TABLE_OR_COL
-                                 abc
-                              p_name
+                           TOK_NULLS_FIRST
+                              .
+                                 TOK_TABLE_OR_COL
+                                    abc
+                                 p_name
             r
          TOK_SELEXPR
             TOK_FUNCTION
@@ -5060,10 +5133,11 @@ TOK_QUERY
                            p_mfgr
                      TOK_SORTBY
                         TOK_TABSORTCOLNAMEASC
-                           .
-                              TOK_TABLE_OR_COL
-                                 abc
-                              p_name
+                           TOK_NULLS_FIRST
+                              .
+                                 TOK_TABLE_OR_COL
+                                    abc
+                                 p_name
             dr
          TOK_SELEXPR
             TOK_FUNCTION
@@ -5081,10 +5155,11 @@ TOK_QUERY
                            p_mfgr
                      TOK_SORTBY
                         TOK_TABSORTCOLNAMEASC
-                           .
-                              TOK_TABLE_OR_COL
-                                 abc
-                              p_name
+                           TOK_NULLS_FIRST
+                              .
+                                 TOK_TABLE_OR_COL
+                                    abc
+                                 p_name
             cd
          TOK_SELEXPR
             .
@@ -5107,10 +5182,11 @@ TOK_QUERY
                            p_mfgr
                      TOK_SORTBY
                         TOK_TABSORTCOLNAMEASC
-                           .
-                              TOK_TABLE_OR_COL
-                                 abc
-                              p_name
+                           TOK_NULLS_FIRST
+                              .
+                                 TOK_TABLE_OR_COL
+                                    abc
+                                 p_name
                   TOK_WINDOWRANGE
                      preceding
                         unbounded
@@ -5147,10 +5223,11 @@ TOK_QUERY
                               p_mfgr
                         TOK_SORTBY
                            TOK_TABSORTCOLNAMEASC
-                              .
-                                 TOK_TABLE_OR_COL
-                                    abc
-                                 p_name
+                              TOK_NULLS_FIRST
+                                 .
+                                    TOK_TABLE_OR_COL
+                                       abc
+                                    p_name
             deltaSz
 
 
@@ -5170,6 +5247,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: p_mfgr (type: string), p_name (type: string)
+              null sort order: aa
               sort order: ++
               Map-reduce partition columns: p_mfgr (type: string)
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -5241,7 +5319,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: abc
                   name: noop
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   output shape: _col0: int, _col1: string, _col2: string, _col5: int, _col7: double
                   partition by: _col2
                   raw input shape:
@@ -5275,6 +5353,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col0 (type: int)
+              null sort order: a
               sort order: +
               Map-reduce partition columns: _col0 (type: int)
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -5291,6 +5370,7 @@ STAGE PLANS:
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
                 key expressions: p_partkey (type: int)
+                null sort order: a
                 sort order: +
                 Map-reduce partition columns: p_partkey (type: int)
                 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -5401,6 +5481,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col2 (type: string), _col1 (type: string)
+              null sort order: aa
               sort order: ++
               Map-reduce partition columns: _col2 (type: string)
               Statistics: Num rows: 28 Data size: 17646 Basic stats: COMPLETE Column stats: NONE
@@ -5447,7 +5528,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:
@@ -5603,8 +5684,9 @@ TOK_QUERY
                   p_mfgr
             TOK_ORDERBY
                TOK_TABSORTCOLNAMEASC
-                  TOK_TABLE_OR_COL
-                     p_name
+                  TOK_NULLS_FIRST
+                     TOK_TABLE_OR_COL
+                        p_name
    TOK_INSERT
       TOK_DESTINATION
          TOK_DIR
@@ -5636,6 +5718,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: p_mfgr (type: string), p_name (type: string)
+              null sort order: aa
               sort order: ++
               Map-reduce partition columns: p_mfgr (type: string)
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -5707,7 +5790,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noop
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2
                   raw input shape:
@@ -5746,6 +5829,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: int)
+              null sort order: aaa
               sort order: +++
               Map-reduce partition columns: _col0 (type: string), _col1 (type: string), _col2 (type: int)
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -5902,8 +5986,9 @@ TOK_QUERY
                   p_mfgr
             TOK_ORDERBY
                TOK_TABSORTCOLNAMEASC
-                  TOK_TABLE_OR_COL
-                     p_mfgr
+                  TOK_NULLS_FIRST
+                     TOK_TABLE_OR_COL
+                        p_mfgr
    TOK_INSERT
       TOK_DESTINATION
          TOK_DIR
@@ -5936,8 +6021,9 @@ TOK_QUERY
                         p_mfgr
                   TOK_ORDERBY
                      TOK_TABSORTCOLNAMEASC
-                        TOK_TABLE_OR_COL
-                           p_brand
+                        TOK_NULLS_FIRST
+                           TOK_TABLE_OR_COL
+                              p_brand
                TOK_WINDOWRANGE
                   preceding
                      2
@@ -5969,6 +6055,7 @@ STAGE PLANS:
                 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string), _col1 (type: string)
+                  null sort order: aa
                   sort order: ++
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -6042,7 +6129,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noop
-                  order by: _col0
+                  order by: _col0 ASC NULLS FIRST
                   output shape: _col0: string, _col1: string, _col2: double
                   partition by: _col0
                   raw input shape:
@@ -6072,6 +6159,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col0 (type: string), _col1 (type: string)
+              null sort order: aa
               sort order: ++
               Map-reduce partition columns: _col0 (type: string)
               Statistics: Num rows: 13 Data size: 8021 Basic stats: COMPLETE Column stats: NONE
@@ -6118,7 +6206,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col0
                   raw input shape:
                   window functions:
@@ -6291,8 +6379,9 @@ TOK_QUERY
                   p_mfgr
             TOK_ORDERBY
                TOK_TABSORTCOLNAMEASC
-                  TOK_TABLE_OR_COL
-                     p_name
+                  TOK_NULLS_FIRST
+                     TOK_TABLE_OR_COL
+                        p_name
    TOK_INSERT
       TOK_DESTINATION
          TOK_TAB
@@ -6318,8 +6407,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_SORTBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             r
          TOK_SELEXPR
             TOK_FUNCTION
@@ -6331,8 +6421,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_SORTBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             dr
          TOK_SELEXPR
             TOK_FUNCTION
@@ -6346,8 +6437,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_SORTBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
                   TOK_WINDOWRANGE
                      preceding
                         unbounded
@@ -6382,8 +6474,9 @@ TOK_QUERY
                               p_mfgr
                         TOK_SORTBY
                            TOK_TABSORTCOLNAMEASC
-                              TOK_TABLE_OR_COL
-                                 p_size
+                              TOK_NULLS_FIRST
+                                 TOK_TABLE_OR_COL
+                                    p_size
                      TOK_WINDOWVALUES
                         preceding
                            5
@@ -6400,11 +6493,13 @@ TOK_QUERY
                            p_mfgr
                      TOK_SORTBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_mfgr
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_mfgr
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             r
          TOK_SELEXPR
             TOK_FUNCTION
@@ -6416,11 +6511,13 @@ TOK_QUERY
                            p_mfgr
                      TOK_SORTBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_mfgr
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_mfgr
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             dr
          TOK_SELEXPR
             TOK_FUNCTION
@@ -6432,11 +6529,13 @@ TOK_QUERY
                            p_mfgr
                      TOK_SORTBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_mfgr
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_mfgr
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             cud
          TOK_SELEXPR
             TOK_FUNCTION
@@ -6457,11 +6556,13 @@ TOK_QUERY
                         p_mfgr
                   TOK_SORTBY
                      TOK_TABSORTCOLNAMEASC
-                        TOK_TABLE_OR_COL
-                           p_mfgr
+                        TOK_NULLS_FIRST
+                           TOK_TABLE_OR_COL
+                              p_mfgr
                      TOK_TABSORTCOLNAMEASC
-                        TOK_TABLE_OR_COL
-                           p_name
+                        TOK_NULLS_FIRST
+                           TOK_TABLE_OR_COL
+                              p_name
                TOK_WINDOWRANGE
                   preceding
                      2
@@ -6489,6 +6590,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: p_mfgr (type: string), p_name (type: string)
+              null sort order: aa
               sort order: ++
               Map-reduce partition columns: p_mfgr (type: string)
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -6560,7 +6662,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noop
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int, _col7: double
                   partition by: _col2
                   raw input shape:
@@ -6607,6 +6709,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col2 (type: string), _col1 (type: string)
+              null sort order: aa
               sort order: ++
               Map-reduce partition columns: _col2 (type: string)
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -6653,7 +6756,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:
@@ -6742,6 +6845,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col2 (type: string), _col5 (type: int)
+              null sort order: aa
               sort order: ++
               Map-reduce partition columns: _col2 (type: string)
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -6788,7 +6892,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col5
+                  order by: _col5 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:
@@ -6828,6 +6932,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col2 (type: string), _col1 (type: string)
+              null sort order: aa
               sort order: ++
               Map-reduce partition columns: _col2 (type: string)
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -6874,7 +6979,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col3, _col2
+                  order by: _col3 ASC NULLS FIRST, _col2 ASC NULLS FIRST
                   partition by: _col3
                   raw input shape:
                   window functions:
@@ -7140,8 +7245,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_mfgr
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_mfgr
             TOK_PARTITIONINGSPEC
                TOK_DISTRIBUTEBY
                   TOK_TABLE_OR_COL
@@ -7150,11 +7256,13 @@ TOK_QUERY
                      p_name
                TOK_ORDERBY
                   TOK_TABSORTCOLNAMEASC
-                     TOK_TABLE_OR_COL
-                        p_mfgr
+                     TOK_NULLS_FIRST
+                        TOK_TABLE_OR_COL
+                           p_mfgr
                   TOK_TABSORTCOLNAMEASC
-                     TOK_TABLE_OR_COL
-                        p_name
+                     TOK_NULLS_FIRST
+                        TOK_TABLE_OR_COL
+                           p_name
          TOK_PARTITIONINGSPEC
             TOK_DISTRIBUTEBY
                TOK_TABLE_OR_COL
@@ -7163,11 +7271,13 @@ TOK_QUERY
                   p_name
             TOK_ORDERBY
                TOK_TABSORTCOLNAMEASC
-                  TOK_TABLE_OR_COL
-                     p_mfgr
+                  TOK_NULLS_FIRST
+                     TOK_TABLE_OR_COL
+                        p_mfgr
                TOK_TABSORTCOLNAMEASC
-                  TOK_TABLE_OR_COL
-                     p_name
+                  TOK_NULLS_FIRST
+                     TOK_TABLE_OR_COL
+                        p_name
    TOK_INSERT
       TOK_DESTINATION
          TOK_DIR
@@ -7239,6 +7349,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: p_mfgr (type: string)
+              null sort order: a
               sort order: +
               Map-reduce partition columns: p_mfgr (type: string)
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -7310,14 +7421,14 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noop
-                  order by: _col2
+                  order by: _col2 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2
                   raw input shape:
                 Partition table definition
                   input alias: ptf_2
                   name: noop
-                  order by: _col2
+                  order by: _col2 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2
                   raw input shape:
@@ -7331,7 +7442,7 @@ STAGE PLANS:
                   Partition table definition
                     input alias: ptf_1
                     name: noopwithmap
-                    order by: _col2, _col1
+                    order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                     output shape: _col1: string, _col2: string, _col5: int
                     partition by: _col2, _col1
                     raw input shape:
@@ -7339,7 +7450,7 @@ STAGE PLANS:
                   Partition table definition
                     input alias: ptf_2
                     name: noop
-                    order by: _col2, _col1
+                    order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                     output shape: _col1: string, _col2: string, _col5: int
                     partition by: _col2, _col1
                     raw input shape:
@@ -7370,6 +7481,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col2 (type: string), _col1 (type: string)
+              null sort order: aa
               sort order: ++
               Map-reduce partition columns: _col2 (type: string), _col1 (type: string)
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -7416,7 +7528,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noopwithmap
-                  order by: _col2, _col1
+                  order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2, _col1
                   raw input shape:
@@ -7424,7 +7536,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_2
                   name: noop
-                  order by: _col2, _col1
+                  order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2, _col1
                   raw input shape:
@@ -7454,6 +7566,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col2 (type: string), _col1 (type: string)
+              null sort order: aa
               sort order: ++
               Map-reduce partition columns: _col2 (type: string), _col1 (type: string)
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -7500,7 +7613,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col2, _col1
+                  order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   partition by: _col2, _col1
                   raw input shape:
                   window functions:
@@ -7679,8 +7792,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_mfgr
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_mfgr
             TOK_PARTITIONINGSPEC
                TOK_DISTRIBUTEBY
                   TOK_TABLE_OR_COL
@@ -7689,19 +7803,22 @@ TOK_QUERY
                      p_name
                TOK_ORDERBY
                   TOK_TABSORTCOLNAMEASC
-                     TOK_TABLE_OR_COL
-                        p_mfgr
+                     TOK_NULLS_FIRST
+                        TOK_TABLE_OR_COL
+                           p_mfgr
                   TOK_TABSORTCOLNAMEASC
-                     TOK_TABLE_OR_COL
-                        p_name
+                     TOK_NULLS_FIRST
+                        TOK_TABLE_OR_COL
+                           p_name
          TOK_PARTITIONINGSPEC
             TOK_DISTRIBUTEBY
                TOK_TABLE_OR_COL
                   p_mfgr
             TOK_ORDERBY
                TOK_TABSORTCOLNAMEASC
-                  TOK_TABLE_OR_COL
-                     p_mfgr
+                  TOK_NULLS_FIRST
+                     TOK_TABLE_OR_COL
+                        p_mfgr
    TOK_INSERT
       TOK_DESTINATION
          TOK_DIR
@@ -7723,8 +7840,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             r
          TOK_SELEXPR
             TOK_FUNCTION
@@ -7736,8 +7854,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             dr
          TOK_SELEXPR
             TOK_TABLE_OR_COL
@@ -7754,8 +7873,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
                   TOK_WINDOWRANGE
                      preceding
                         unbounded
@@ -7780,6 +7900,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: p_mfgr (type: string)
+              null sort order: a
               sort order: +
               Map-reduce partition columns: p_mfgr (type: string)
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -7851,14 +7972,14 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noop
-                  order by: _col2
+                  order by: _col2 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2
                   raw input shape:
                 Partition table definition
                   input alias: ptf_2
                   name: noop
-                  order by: _col2
+                  order by: _col2 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2
                   raw input shape:
@@ -7888,6 +8009,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col2 (type: string), _col1 (type: string)
+              null sort order: aa
               sort order: ++
               Map-reduce partition columns: _col2 (type: string), _col1 (type: string)
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -7934,7 +8056,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noop
-                  order by: _col2, _col1
+                  order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2, _col1
                   raw input shape:
@@ -7964,6 +8086,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col2 (type: string)
+              null sort order: a
               sort order: +
               Map-reduce partition columns: _col2 (type: string)
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -8010,7 +8133,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noop
-                  order by: _col2
+                  order by: _col2 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2
                   raw input shape:
@@ -8040,6 +8163,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col2 (type: string), _col1 (type: string)
+              null sort order: aa
               sort order: ++
               Map-reduce partition columns: _col2 (type: string)
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -8086,7 +8210,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:
@@ -8263,19 +8387,22 @@ TOK_QUERY
                            p_name
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_mfgr
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_mfgr
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             TOK_PARTITIONINGSPEC
                TOK_DISTRIBUTEBY
                   TOK_TABLE_OR_COL
                      p_mfgr
                TOK_ORDERBY
                   TOK_TABSORTCOLNAMEASC
-                     TOK_TABLE_OR_COL
-                        p_mfgr
+                     TOK_NULLS_FIRST
+                        TOK_TABLE_OR_COL
+                           p_mfgr
    TOK_INSERT
       TOK_DESTINATION
          TOK_DIR
@@ -8297,8 +8424,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             r
          TOK_SELEXPR
             TOK_FUNCTION
@@ -8310,8 +8438,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             dr
          TOK_SELEXPR
             TOK_TABLE_OR_COL
@@ -8328,8 +8457,9 @@ TOK_QUERY
                            p_mfgr
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             s1
 
 
@@ -8349,6 +8479,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: p_mfgr (type: string), p_name (type: string)
+              null sort order: aa
               sort order: ++
               Map-reduce partition columns: p_mfgr (type: string), p_name (type: string)
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -8420,14 +8551,14 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noop
-                  order by: _col2, _col1
+                  order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2, _col1
                   raw input shape:
                 Partition table definition
                   input alias: ptf_2
                   name: noop
-                  order by: _col2, _col1
+                  order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2, _col1
                   raw input shape:
@@ -8457,6 +8588,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col2 (type: string)
+              null sort order: a
               sort order: +
               Map-reduce partition columns: _col2 (type: string)
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -8503,14 +8635,14 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noop
-                  order by: _col2
+                  order by: _col2 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2
                   raw input shape:
                 Partition table definition
                   input alias: ptf_2
                   name: noop
-                  order by: _col2
+                  order by: _col2 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2
                   raw input shape:
@@ -8540,6 +8672,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col2 (type: string), _col1 (type: string)
+              null sort order: aa
               sort order: ++
               Map-reduce partition columns: _col2 (type: string)
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -8586,7 +8719,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col1
+                  order by: _col1 ASC NULLS FIRST
                   partition by: _col2
                   raw input shape:
                   window functions:
@@ -8763,19 +8896,22 @@ TOK_QUERY
                            p_name
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_mfgr
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_mfgr
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             TOK_PARTITIONINGSPEC
                TOK_DISTRIBUTEBY
                   TOK_TABLE_OR_COL
                      p_mfgr
                TOK_ORDERBY
                   TOK_TABSORTCOLNAMEASC
-                     TOK_TABLE_OR_COL
-                        p_mfgr
+                     TOK_NULLS_FIRST
+                        TOK_TABLE_OR_COL
+                           p_mfgr
          TOK_PARTITIONINGSPEC
             TOK_DISTRIBUTEBY
                TOK_TABLE_OR_COL
@@ -8784,11 +8920,13 @@ TOK_QUERY
                   p_name
             TOK_ORDERBY
                TOK_TABSORTCOLNAMEASC
-                  TOK_TABLE_OR_COL
-                     p_mfgr
+                  TOK_NULLS_FIRST
+                     TOK_TABLE_OR_COL
+                        p_mfgr
                TOK_TABSORTCOLNAMEASC
-                  TOK_TABLE_OR_COL
-                     p_name
+                  TOK_NULLS_FIRST
+                     TOK_TABLE_OR_COL
+                        p_name
    TOK_INSERT
       TOK_DESTINATION
          TOK_DIR
@@ -8861,6 +8999,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: p_mfgr (type: string), p_name (type: string)
+              null sort order: aa
               sort order: ++
               Map-reduce partition columns: p_mfgr (type: string), p_name (type: string)
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -8932,14 +9071,14 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noop
-                  order by: _col2, _col1
+                  order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2, _col1
                   raw input shape:
                 Partition table definition
                   input alias: ptf_2
                   name: noop
-                  order by: _col2, _col1
+                  order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2, _col1
                   raw input shape:
@@ -8969,6 +9108,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col2 (type: string)
+              null sort order: a
               sort order: +
               Map-reduce partition columns: _col2 (type: string)
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -9015,7 +9155,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noop
-                  order by: _col2
+                  order by: _col2 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2
                   raw input shape:
@@ -9029,7 +9169,7 @@ STAGE PLANS:
                   Partition table definition
                     input alias: ptf_1
                     name: noopwithmap
-                    order by: _col2, _col1
+                    order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                     output shape: _col1: string, _col2: string, _col5: int
                     partition by: _col2, _col1
                     raw input shape:
@@ -9061,6 +9201,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col2 (type: string), _col1 (type: string)
+              null sort order: aa
               sort order: ++
               Map-reduce partition columns: _col2 (type: string), _col1 (type: string)
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -9107,7 +9248,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noopwithmap
-                  order by: _col2, _col1
+                  order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2, _col1
                   raw input shape:
@@ -9138,6 +9279,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col2 (type: string), _col1 (type: string)
+              null sort order: aa
               sort order: ++
               Map-reduce partition columns: _col2 (type: string), _col1 (type: string)
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -9184,7 +9326,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefunction
-                  order by: _col2, _col1
+                  order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   partition by: _col2, _col1
                   raw input shape:
                   window functions:
@@ -9361,19 +9503,22 @@ TOK_QUERY
                         p_name
                   TOK_ORDERBY
                      TOK_TABSORTCOLNAMEASC
-                        TOK_TABLE_OR_COL
-                           p_mfgr
+                        TOK_NULLS_FIRST
+                           TOK_TABLE_OR_COL
+                              p_mfgr
                      TOK_TABSORTCOLNAMEASC
-                        TOK_TABLE_OR_COL
-                           p_name
+                        TOK_NULLS_FIRST
+                           TOK_TABLE_OR_COL
+                              p_name
             TOK_PARTITIONINGSPEC
                TOK_DISTRIBUTEBY
                   TOK_TABLE_OR_COL
                      p_mfgr
                TOK_ORDERBY
                   TOK_TABSORTCOLNAMEASC
-                     TOK_TABLE_OR_COL
-                        p_mfgr
+                     TOK_NULLS_FIRST
+                        TOK_TABLE_OR_COL
+                           p_mfgr
    TOK_INSERT
       TOK_DESTINATION
          TOK_DIR
@@ -9397,11 +9542,13 @@ TOK_QUERY
                            p_name
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_mfgr
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_mfgr
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             r
          TOK_SELEXPR
             TOK_FUNCTION
@@ -9415,11 +9562,13 @@ TOK_QUERY
                            p_name
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_mfgr
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_mfgr
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
             dr
          TOK_SELEXPR
             TOK_TABLE_OR_COL
@@ -9438,11 +9587,13 @@ TOK_QUERY
                            p_name
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_mfgr
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_mfgr
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
                   TOK_WINDOWRANGE
                      preceding
                         unbounded
@@ -9462,11 +9613,13 @@ TOK_QUERY
                            p_name
                      TOK_ORDERBY
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_mfgr
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_mfgr
                         TOK_TABSORTCOLNAMEASC
-                           TOK_TABLE_OR_COL
-                              p_name
+                           TOK_NULLS_FIRST
+                              TOK_TABLE_OR_COL
+                                 p_name
                   TOK_WINDOWRANGE
                      preceding
                         unbounded
@@ -9490,6 +9643,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: p_mfgr (type: string), p_name (type: string)
+              null sort order: aa
               sort order: ++
               Map-reduce partition columns: p_mfgr (type: string), p_name (type: string)
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -9561,7 +9715,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noop
-                  order by: _col2, _col1
+                  order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2, _col1
                   raw input shape:
@@ -9575,7 +9729,7 @@ STAGE PLANS:
                   Partition table definition
                     input alias: ptf_1
                     name: noopwithmap
-                    order by: _col2
+                    order by: _col2 ASC NULLS FIRST
                     output shape: _col1: string, _col2: string, _col5: int
                     partition by: _col2
                     raw input shape:
@@ -9583,7 +9737,7 @@ STAGE PLANS:
                   Partition table definition
                     input alias: ptf_2
                     name: noop
-                    order by: _col2
+                    order by: _col2 ASC NULLS FIRST
                     output shape: _col1: string, _col2: string, _col5: int
                     partition by: _col2
                     raw input shape:
@@ -9614,6 +9768,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col2 (type: string)
+              null sort order: a
               sort order: +
               Map-reduce partition columns: _col2 (type: string)
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -9660,7 +9815,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_1
                   name: noopwithmap
-                  order by: _col2
+                  order by: _col2 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2
                   raw input shape:
@@ -9668,7 +9823,7 @@ STAGE PLANS:
                 Partition table definition
                   input alias: ptf_2
                   name: noop
-                  order by: _col2
+                  order by: _col2 ASC NULLS FIRST
                   output shape: _col1: string, _col2: string, _col5: int
                   partition by: _col2
                   raw input shape:
@@ -9698,6 +9853,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col2 (type: string), _col1 (type: string)
+              null sort order: aa
               sort order: ++
               Map-reduce partition columns: _col2 (type: string), _col1 (type: string)
               Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE
@@ -9744,7 +9900,7 @@ STAGE PLANS:
                 Windowing table definition
                   input alias: ptf_1
                   name: windowingtablefu

<TRUNCATED>

[16/18] hive git commit: HIVE-12994: Implement support for NULLS FIRST/NULLS LAST (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
index 6373746..f0ea4c5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
@@ -6340,6 +6340,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     ArrayList<ExprNodeDesc> partnCols = new ArrayList<ExprNodeDesc>();
     ArrayList<ExprNodeDesc> sortCols = new ArrayList<ExprNodeDesc>();
     ArrayList<Integer> sortOrders = new ArrayList<Integer>();
+    ArrayList<Integer> nullSortOrders = new ArrayList<Integer>();
     boolean multiFileSpray = false;
     int numFiles = 1;
     int totalFiles = 1;
@@ -6357,6 +6358,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         (dest_tab.getSortCols().size() > 0)) {
       sortCols = getSortCols(dest, qb, dest_tab, table_desc, input, true);
       sortOrders = getSortOrders(dest, qb, dest_tab, input);
+      nullSortOrders = getNullSortOrders(dest, qb, dest_tab, input);
       if (!enforceBucketing && !dest_tab.isIndexTable()) {
         throw new SemanticException(ErrorMsg.TBL_SORTED_NOT_BUCKETED.getErrorCodedMsg(dest_tab.getCompleteName()));
       } else {
@@ -6394,8 +6396,12 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       for (int sortOrder : sortOrders) {
         order.append(sortOrder == BaseSemanticAnalyzer.HIVE_COLUMN_ORDER_ASC ? '+' : '-');
       }
-      input = genReduceSinkPlan(input, partnCols, sortCols, order.toString(),  maxReducers,
-        (AcidUtils.isAcidTable(dest_tab) ? getAcidType() : AcidUtils.Operation.NOT_ACID));
+      StringBuilder nullOrder = new StringBuilder();
+      for (int pos : nullSortOrders) {
+        nullOrder.append(pos == BaseSemanticAnalyzer.HIVE_COLUMN_NULLS_FIRST ? 'a' : 'z');
+      }
+      input = genReduceSinkPlan(input, partnCols, sortCols, order.toString(), nullOrder.toString(),
+              maxReducers, (AcidUtils.isAcidTable(dest_tab) ? getAcidType() : AcidUtils.Operation.NOT_ACID));
       reduceSinkOperatorsAddedByEnforceBucketingSorting.add((ReduceSinkOperator)input.getParentOperators().get(0));
       ctx.setMultiFileSpray(multiFileSpray);
       ctx.setNumFiles(numFiles);
@@ -7356,6 +7362,23 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     return orders;
   }
 
+  private ArrayList<Integer> getNullSortOrders(String dest, QB qb, Table tab, Operator input)
+      throws SemanticException {
+    List<Order> tabSortCols = tab.getSortCols();
+    List<FieldSchema> tabCols = tab.getCols();
+
+    ArrayList<Integer> orders = new ArrayList<Integer>();
+    for (Order sortCol : tabSortCols) {
+      for (FieldSchema tabCol : tabCols) {
+        if (sortCol.getCol().equals(tabCol.getName())) {
+          orders.add(sortCol.getNullOrder());
+          break;
+        }
+      }
+    }
+    return orders;
+  }
+
   private Operator genReduceSinkPlan(String dest, QB qb, Operator<?> input,
       int numReducers, boolean hasOrderBy) throws SemanticException {
 
@@ -7396,6 +7419,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     }
     ArrayList<ExprNodeDesc> sortCols = new ArrayList<ExprNodeDesc>();
     StringBuilder order = new StringBuilder();
+    StringBuilder nullOrder = new StringBuilder();
     if (sortExprs != null) {
       int ccount = sortExprs.getChildCount();
       for (int i = 0; i < ccount; ++i) {
@@ -7405,20 +7429,40 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
           // SortBy ASC
           order.append("+");
           cl = (ASTNode) cl.getChild(0);
+          if (cl.getType() == HiveParser.TOK_NULLS_FIRST) {
+            nullOrder.append("a");
+          } else if (cl.getType() == HiveParser.TOK_NULLS_LAST) {
+            nullOrder.append("z");
+          } else {
+            throw new SemanticException(
+                    "Unexpected null ordering option: " + cl.getType());
+          }
+          cl = (ASTNode) cl.getChild(0);
         } else if (cl.getType() == HiveParser.TOK_TABSORTCOLNAMEDESC) {
           // SortBy DESC
           order.append("-");
           cl = (ASTNode) cl.getChild(0);
+          if (cl.getType() == HiveParser.TOK_NULLS_FIRST) {
+            nullOrder.append("a");
+          } else if (cl.getType() == HiveParser.TOK_NULLS_LAST) {
+            nullOrder.append("z");
+          } else {
+            throw new SemanticException(
+                    "Unexpected null ordering option: " + cl.getType());
+          }
+          cl = (ASTNode) cl.getChild(0);
         } else {
           // ClusterBy
           order.append("+");
+          nullOrder.append("a");
         }
         ExprNodeDesc exprNode = genExprNodeDesc(cl, inputRR);
         sortCols.add(exprNode);
       }
     }
     Operator result = genReduceSinkPlan(
-        input, partCols, sortCols, order.toString(), numReducers, Operation.NOT_ACID);
+        input, partCols, sortCols, order.toString(), nullOrder.toString(),
+        numReducers, Operation.NOT_ACID);
     if (result.getParentOperators().size() == 1 &&
         result.getParentOperators().get(0) instanceof ReduceSinkOperator) {
       ((ReduceSinkOperator) result.getParentOperators().get(0))
@@ -7430,7 +7474,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
   @SuppressWarnings("nls")
   private Operator genReduceSinkPlan(Operator<?> input,
       ArrayList<ExprNodeDesc> partitionCols, ArrayList<ExprNodeDesc> sortCols,
-      String sortOrder, int numReducers, AcidUtils.Operation acidOp) throws SemanticException {
+      String sortOrder, String nullOrder, int numReducers, AcidUtils.Operation acidOp)
+              throws SemanticException {
 
     RowResolver inputRR = opParseCtx.get(input).getRowResolver();
 
@@ -7497,7 +7542,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     dummy.setParentOperators(null);
 
     ReduceSinkDesc rsdesc = PlanUtils.getReduceSinkDesc(sortCols, valueCols, outputColumns,
-        false, -1, partitionCols, sortOrder, numReducers, acidOp);
+        false, -1, partitionCols, sortOrder, nullOrder, numReducers, acidOp);
     Operator interim = putOpInsertMap(OperatorFactory.getAndMakeChild(rsdesc,
         new RowSchema(rsRR.getColumnInfos()), input), rsRR);
 
@@ -11552,7 +11597,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
             }
           }
           for (int child_pos = 0; child_pos < orderbyNode.getChildCount(); ++child_pos) {
-            ASTNode colNode = (ASTNode) orderbyNode.getChild(child_pos);
+            ASTNode colNode = (ASTNode) orderbyNode.getChild(child_pos).getChild(0);
             ASTNode node = (ASTNode) colNode.getChild(0);
             if (node.getToken().getType() == HiveParser.Number) {
               if( isByPos ) {
@@ -11884,13 +11929,20 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     int exprCnt = sortNode.getChildCount();
     for(int i=0; i < exprCnt; i++) {
       OrderExpression exprSpec = new OrderExpression();
-      exprSpec.setExpression((ASTNode) sortNode.getChild(i).getChild(0));
-      if ( sortNode.getChild(i).getType() == HiveParser.TOK_TABSORTCOLNAMEASC ) {
+      ASTNode orderSpec = (ASTNode) sortNode.getChild(i);
+      ASTNode nullOrderSpec = (ASTNode) orderSpec.getChild(0);
+      exprSpec.setExpression((ASTNode) nullOrderSpec.getChild(0));
+      if ( orderSpec.getType() == HiveParser.TOK_TABSORTCOLNAMEASC ) {
         exprSpec.setOrder(org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.Order.ASC);
       }
       else {
         exprSpec.setOrder(org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.Order.DESC);
       }
+      if ( nullOrderSpec.getType() == HiveParser.TOK_NULLS_FIRST ) {
+        exprSpec.setNullOrder(org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.NullOrder.NULLS_FIRST);
+      } else {
+        exprSpec.setNullOrder(org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.NullOrder.NULLS_LAST);
+      }
       oSpec.addExpression(exprSpec);
     }
     return oSpec;
@@ -12228,7 +12280,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       RowResolver inputRR,
       ArrayList<ExprNodeDesc> partCols,
       ArrayList<ExprNodeDesc> orderCols,
-      StringBuilder orderString) throws SemanticException {
+      StringBuilder orderString,
+      StringBuilder nullOrderString) throws SemanticException {
 
     List<PTFExpressionDef> partColList = tabDef.getPartition().getExpressions();
 
@@ -12238,6 +12291,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         partCols.add(exprNode);
         orderCols.add(exprNode);
         orderString.append('+');
+        nullOrderString.append('a');
       }
     }
 
@@ -12252,13 +12306,16 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     for (int i = 0; i < orderColList.size(); i++) {
       OrderExpressionDef colDef = orderColList.get(i);
       char orderChar = colDef.getOrder() == PTFInvocationSpec.Order.ASC ? '+' : '-';
+      char nullOrderChar = colDef.getNullOrder() == PTFInvocationSpec.NullOrder.NULLS_FIRST ? 'a' : 'z';
       int index = ExprNodeDescUtils.indexOf(colDef.getExprNode(), orderCols);
       if (index >= 0) {
         orderString.setCharAt(index, orderChar);
+        nullOrderString.setCharAt(index, nullOrderChar);
         continue;
       }
       orderCols.add(colDef.getExprNode());
       orderString.append(orderChar);
+      nullOrderString.append(nullOrderChar);
     }
   }
 
@@ -12301,6 +12358,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       ArrayList<ExprNodeDesc> partCols = new ArrayList<ExprNodeDesc>();
       ArrayList<ExprNodeDesc> orderCols = new ArrayList<ExprNodeDesc>();
       StringBuilder orderString = new StringBuilder();
+      StringBuilder nullOrderString = new StringBuilder();
 
       /*
        * Use the input RR of TableScanOperator in case there is no map-side
@@ -12308,8 +12366,9 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
        * If the parent of ReduceSinkOperator is PTFOperator, use it's
        * output RR.
        */
-      buildPTFReduceSinkDetails(tabDef, rr, partCols, orderCols, orderString);
-      input = genReduceSinkPlan(input, partCols, orderCols, orderString.toString(), -1, Operation.NOT_ACID);
+      buildPTFReduceSinkDetails(tabDef, rr, partCols, orderCols, orderString, nullOrderString);
+      input = genReduceSinkPlan(input, partCols, orderCols, orderString.toString(),
+              nullOrderString.toString(), -1, Operation.NOT_ACID);
     }
 
     /*
@@ -12367,6 +12426,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     ArrayList<ExprNodeDesc> partCols = new ArrayList<ExprNodeDesc>();
     ArrayList<ExprNodeDesc> orderCols = new ArrayList<ExprNodeDesc>();
     StringBuilder order = new StringBuilder();
+    StringBuilder nullOrder = new StringBuilder();
 
     for (PartitionExpression partCol : spec.getQueryPartitionSpec().getExpressions()) {
       ExprNodeDesc partExpr = genExprNodeDesc(partCol.getExpression(), inputRR);
@@ -12374,6 +12434,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         partCols.add(partExpr);
         orderCols.add(partExpr);
         order.append('+');
+        nullOrder.append('a');
       }
     }
 
@@ -12381,17 +12442,21 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       for (OrderExpression orderCol : spec.getQueryOrderSpec().getExpressions()) {
         ExprNodeDesc orderExpr = genExprNodeDesc(orderCol.getExpression(), inputRR);
         char orderChar = orderCol.getOrder() == PTFInvocationSpec.Order.ASC ? '+' : '-';
+        char nullOrderChar = orderCol.getNullOrder() == PTFInvocationSpec.NullOrder.NULLS_FIRST ? 'a' : 'z';
         int index = ExprNodeDescUtils.indexOf(orderExpr, orderCols);
         if (index >= 0) {
           order.setCharAt(index, orderChar);
+          nullOrder.setCharAt(index, nullOrderChar);
           continue;
         }
         orderCols.add(genExprNodeDesc(orderCol.getExpression(), inputRR));
         order.append(orderChar);
+        nullOrder.append(nullOrderChar);
       }
     }
 
-    return genReduceSinkPlan(input, partCols, orderCols, order.toString(), -1, Operation.NOT_ACID);
+    return genReduceSinkPlan(input, partCols, orderCols, order.toString(), nullOrder.toString(),
+            -1, Operation.NOT_ACID);
   }
 
   public static ArrayList<WindowExpressionSpec> parseSelect(String selectExprStr)

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
index 598520c..45dfd27 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java
@@ -735,6 +735,8 @@ public class TypeCheckProcFactory {
       windowingTokens.add(HiveParser.KW_CURRENT);
       windowingTokens.add(HiveParser.TOK_TABSORTCOLNAMEASC);
       windowingTokens.add(HiveParser.TOK_TABSORTCOLNAMEDESC);
+      windowingTokens.add(HiveParser.TOK_NULLS_FIRST);
+      windowingTokens.add(HiveParser.TOK_NULLS_LAST);
     }
 
     protected static boolean isRedundantConversionFunction(ASTNode expr,

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java
index 990a10c..ae8c77f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java
@@ -23,15 +23,12 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
-import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 import java.util.Set;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.common.JavaUtils;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -39,7 +36,6 @@ import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.ql.exec.ColumnInfo;
-import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.RowSchema;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.exec.Utilities;
@@ -66,8 +62,8 @@ import org.apache.hadoop.hive.serde2.Deserializer;
 import org.apache.hadoop.hive.serde2.MetadataTypedColumnsetSerDe;
 import org.apache.hadoop.hive.serde2.binarysortable.BinarySortableSerDe;
 import org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe;
-import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe;
 import org.apache.hadoop.hive.serde2.lazy.LazySerDeParameters;
+import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe;
 import org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
@@ -76,6 +72,8 @@ import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.SequenceFileInputFormat;
 import org.apache.hadoop.mapred.SequenceFileOutputFormat;
 import org.apache.hadoop.mapred.TextInputFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * PlanUtils.
@@ -392,7 +390,7 @@ public final class PlanUtils {
    * Generate the table descriptor for reduce key.
    */
   public static TableDesc getReduceKeyTableDesc(List<FieldSchema> fieldSchemas,
-      String order) {
+      String order, String nullOrder) {
     return new TableDesc(
         SequenceFileInputFormat.class, SequenceFileOutputFormat.class,
         Utilities.makeProperties(serdeConstants.LIST_COLUMNS, MetaStoreUtils
@@ -400,6 +398,7 @@ public final class PlanUtils {
         serdeConstants.LIST_COLUMN_TYPES, MetaStoreUtils
         .getColumnTypesFromFieldSchema(fieldSchemas),
         serdeConstants.SERIALIZATION_SORT_ORDER, order,
+        serdeConstants.SERIALIZATION_NULL_SORT_ORDER, nullOrder,
         serdeConstants.SERIALIZATION_LIB, BinarySortableSerDe.class.getName()));
   }
 
@@ -414,8 +413,10 @@ public final class PlanUtils {
       // be broadcast (instead of partitioned). As a consequence we use
       // a different SerDe than in the MR mapjoin case.
       StringBuilder order = new StringBuilder();
+      StringBuilder nullOrder = new StringBuilder();
       for (FieldSchema f: fieldSchemas) {
         order.append("+");
+        nullOrder.append("a");
       }
       return new TableDesc(
           SequenceFileInputFormat.class, SequenceFileOutputFormat.class,
@@ -424,6 +425,7 @@ public final class PlanUtils {
               serdeConstants.LIST_COLUMN_TYPES, MetaStoreUtils
               .getColumnTypesFromFieldSchema(fieldSchemas),
               serdeConstants.SERIALIZATION_SORT_ORDER, order.toString(),
+              serdeConstants.SERIALIZATION_NULL_SORT_ORDER, nullOrder.toString(),
               serdeConstants.SERIALIZATION_LIB, BinarySortableSerDe.class.getName()));
     } else {
       return new TableDesc(SequenceFileInputFormat.class,
@@ -609,15 +611,15 @@ public final class PlanUtils {
   public static ReduceSinkDesc getReduceSinkDesc(
       ArrayList<ExprNodeDesc> keyCols, ArrayList<ExprNodeDesc> valueCols,
       List<String> outputColumnNames, boolean includeKeyCols, int tag,
-      ArrayList<ExprNodeDesc> partitionCols, String order, int numReducers,
-      AcidUtils.Operation writeType) {
+      ArrayList<ExprNodeDesc> partitionCols, String order, String nullOrder,
+      int numReducers, AcidUtils.Operation writeType) {
     return getReduceSinkDesc(keyCols, keyCols.size(), valueCols,
         new ArrayList<List<Integer>>(),
         includeKeyCols ? outputColumnNames.subList(0, keyCols.size()) :
           new ArrayList<String>(),
         includeKeyCols ? outputColumnNames.subList(keyCols.size(),
             outputColumnNames.size()) : outputColumnNames,
-        includeKeyCols, tag, partitionCols, order, numReducers, writeType);
+        includeKeyCols, tag, partitionCols, order, nullOrder, numReducers, writeType);
   }
 
   /**
@@ -654,8 +656,8 @@ public final class PlanUtils {
       List<String> outputKeyColumnNames,
       List<String> outputValueColumnNames,
       boolean includeKeyCols, int tag,
-      ArrayList<ExprNodeDesc> partitionCols, String order, int numReducers,
-      AcidUtils.Operation writeType) {
+      ArrayList<ExprNodeDesc> partitionCols, String order, String nullOrder,
+      int numReducers, AcidUtils.Operation writeType) {
     TableDesc keyTable = null;
     TableDesc valueTable = null;
     ArrayList<String> outputKeyCols = new ArrayList<String>();
@@ -666,11 +668,14 @@ public final class PlanUtils {
       if (order.length() < outputKeyColumnNames.size()) {
         order = order + "+";
       }
-      keyTable = getReduceKeyTableDesc(keySchema, order);
+      if (nullOrder.length() < outputKeyColumnNames.size()) {
+        nullOrder = nullOrder + "a";
+      }
+      keyTable = getReduceKeyTableDesc(keySchema, order, nullOrder);
       outputKeyCols.addAll(outputKeyColumnNames);
     } else {
       keyTable = getReduceKeyTableDesc(getFieldSchemasFromColumnList(
-          keyCols, "reducesinkkey"),order);
+          keyCols, "reducesinkkey"), order, nullOrder);
      for (int i = 0; i < keyCols.size(); i++) {
         outputKeyCols.add("reducesinkkey" + i);
       }
@@ -767,12 +772,14 @@ public final class PlanUtils {
     }
 
     StringBuilder order = new StringBuilder();
+    StringBuilder nullOrder = new StringBuilder();
     for (int i = 0; i < keyCols.size(); i++) {
       order.append("+");
+      nullOrder.append("a");
     }
     return getReduceSinkDesc(keyCols, numKeys, valueCols, distinctColIndices,
         outputKeyColumnNames, outputValueColumnNames, includeKey, tag,
-        partitionCols, order.toString(), numReducers, writeType);
+        partitionCols, order.toString(), nullOrder.toString(), numReducers, writeType);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java
index 41d9ffe..d7e404c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java
@@ -22,10 +22,10 @@ import java.util.ArrayList;
 import java.util.EnumSet;
 import java.util.List;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.apache.hadoop.hive.ql.plan.Explain.Level;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
 
@@ -380,6 +380,25 @@ public class ReduceSinkDesc extends AbstractOperatorDesc {
     return false;
   }
 
+  /**
+   * Returns the null order in the key columns.
+   *
+   * @return null, which means default for all key columns, or a String
+   *         of the same length as key columns, that consists of only "a"
+   *         (null first) and "z" (null last).
+   */
+  @Explain(displayName = "null sort order", explainLevels = { Level.EXTENDED })
+  public String getNullOrder() {
+    return keySerializeInfo.getProperties().getProperty(
+        org.apache.hadoop.hive.serde.serdeConstants.SERIALIZATION_NULL_SORT_ORDER);
+  }
+
+  public void setNullOrder(String nullOrderStr) {
+    keySerializeInfo.getProperties().setProperty(
+        org.apache.hadoop.hive.serde.serdeConstants.SERIALIZATION_NULL_SORT_ORDER,
+        nullOrderStr);
+  }
+
   public List<List<Integer>> getDistinctColumnIndices() {
     return distinctColumnIndices;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/java/org/apache/hadoop/hive/ql/plan/ptf/OrderExpressionDef.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/ptf/OrderExpressionDef.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/ptf/OrderExpressionDef.java
index e367d13..827911a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/ptf/OrderExpressionDef.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/ptf/OrderExpressionDef.java
@@ -18,15 +18,18 @@
 
 package org.apache.hadoop.hive.ql.plan.ptf;
 
+import org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.NullOrder;
 import org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.Order;
 
 public class OrderExpressionDef extends PTFExpressionDef {
   private Order order;
+  private NullOrder nullOrder;
 
   public OrderExpressionDef() {}
   public OrderExpressionDef(PTFExpressionDef e) {
     super(e);
     order = Order.ASC;
+    nullOrder = NullOrder.NULLS_FIRST;
   }
 
   public Order getOrder() {
@@ -36,5 +39,13 @@ public class OrderExpressionDef extends PTFExpressionDef {
   public void setOrder(Order order) {
     this.order = order;
   }
-}
 
+  public NullOrder getNullOrder() {
+    return nullOrder;
+  }
+
+  public void setNullOrder(NullOrder nullOrder) {
+    this.nullOrder = nullOrder;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/java/org/apache/hadoop/hive/ql/plan/ptf/PartitionedTableFunctionDef.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/ptf/PartitionedTableFunctionDef.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/ptf/PartitionedTableFunctionDef.java
index 2a8b1c0..54b0fb8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/ptf/PartitionedTableFunctionDef.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/ptf/PartitionedTableFunctionDef.java
@@ -122,8 +122,16 @@ public class PartitionedTableFunctionDef extends PTFInputDef {
         builder.append(", ");
       }
       builder.append(expression.getExprNode().getExprString());
-      if (expression.getOrder() == PTFInvocationSpec.Order.DESC) {
-        builder.append("(DESC)");
+      builder.append(" ");
+      if (expression.getOrder() == PTFInvocationSpec.Order.ASC) {
+        builder.append("ASC ");
+      } else {
+        builder.append("DESC ");
+      }
+      if (expression.getNullOrder() == PTFInvocationSpec.NullOrder.NULLS_FIRST) {
+        builder.append("NULLS FIRST");
+      } else {
+        builder.append("NULLS LAST");
       }
     }
     return builder.toString();

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/WindowingTableFunction.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/WindowingTableFunction.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/WindowingTableFunction.java
index 2c076f50..2ac4039 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/WindowingTableFunction.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/ptf/WindowingTableFunction.java
@@ -27,19 +27,18 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.commons.lang.ArrayUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
 import org.apache.hadoop.hive.ql.exec.PTFOperator;
 import org.apache.hadoop.hive.ql.exec.PTFPartition;
-import org.apache.hadoop.hive.ql.exec.WindowFunctionInfo;
 import org.apache.hadoop.hive.ql.exec.PTFPartition.PTFPartitionIterator;
 import org.apache.hadoop.hive.ql.exec.PTFRollingPartition;
+import org.apache.hadoop.hive.ql.exec.WindowFunctionInfo;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.NullOrder;
 import org.apache.hadoop.hive.ql.parse.PTFInvocationSpec.Order;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.parse.WindowingSpec.BoundarySpec;
@@ -62,6 +61,8 @@ import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.StructField;
 import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 @SuppressWarnings("deprecation")
 public class WindowingTableFunction extends TableFunctionEvaluator {
@@ -110,6 +111,7 @@ public class WindowingTableFunction extends TableFunctionEvaluator {
 
     WindowTableFunctionDef wTFnDef = (WindowTableFunctionDef) getTableDef();
     Order order = wTFnDef.getOrder().getExpressions().get(0).getOrder();
+    NullOrder nullOrder = wTFnDef.getOrder().getExpressions().get(0).getNullOrder();
 
     for(WindowFunctionDef wFn : wTFnDef.getWindowFunctions()) {
       boolean processWindow = processWindow(wFn);
@@ -121,7 +123,7 @@ public class WindowingTableFunction extends TableFunctionEvaluator {
         }
         oColumns.add((List<?>)out);
       } else {
-        oColumns.add(executeFnwithWindow(getQueryDef(), wFn, iPart, order));
+        oColumns.add(executeFnwithWindow(getQueryDef(), wFn, iPart, order, nullOrder));
       }
     }
 
@@ -421,7 +423,7 @@ public class WindowingTableFunction extends TableFunctionEvaluator {
         int rowToProcess = streamingState.rollingPart.rowToProcess(wFn.getWindowFrame());
         if (rowToProcess >= 0) {
           Range rng = getRange(wFn, rowToProcess, streamingState.rollingPart,
-              streamingState.order);
+              streamingState.order, streamingState.nullOrder);
           PTFPartitionIterator<Object> rItr = rng.iterator();
           PTFOperator.connectLeadLagFunctionsToPartition(ptfDesc, rItr);
           Object out = evaluateWindowFunction(wFn, rItr);
@@ -499,7 +501,7 @@ public class WindowingTableFunction extends TableFunctionEvaluator {
           int rowToProcess = streamingState.rollingPart.size() - numRowsRemaining;
           if (rowToProcess >= 0) {
             Range rng = getRange(wFn, rowToProcess, streamingState.rollingPart,
-                streamingState.order);
+                streamingState.order, streamingState.nullOrder);
             PTFPartitionIterator<Object> rItr = rng.iterator();
             PTFOperator.connectLeadLagFunctionsToPartition(ptfDesc, rItr);
             Object out = evaluateWindowFunction(wFn, rItr);
@@ -659,11 +661,12 @@ public class WindowingTableFunction extends TableFunctionEvaluator {
   ArrayList<Object> executeFnwithWindow(PTFDesc ptfDesc,
       WindowFunctionDef wFnDef,
       PTFPartition iPart,
-      Order order)
+      Order order,
+      NullOrder nullOrder)
     throws HiveException {
     ArrayList<Object> vals = new ArrayList<Object>();
     for(int i=0; i < iPart.size(); i++) {
-      Range rng = getRange(wFnDef, i, iPart, order);
+      Range rng = getRange(wFnDef, i, iPart, order, nullOrder);
       PTFPartitionIterator<Object> rItr = rng.iterator();
       PTFOperator.connectLeadLagFunctionsToPartition(ptfDesc, rItr);
       Object out = evaluateWindowFunction(wFnDef, rItr);
@@ -672,7 +675,7 @@ public class WindowingTableFunction extends TableFunctionEvaluator {
     return vals;
   }
 
-  private Range getRange(WindowFunctionDef wFnDef, int currRow, PTFPartition p, Order order) throws HiveException
+  private Range getRange(WindowFunctionDef wFnDef, int currRow, PTFPartition p, Order order, NullOrder nullOrder) throws HiveException
   {
     BoundaryDef startB = wFnDef.getWindowFrame().getStart();
     BoundaryDef endB = wFnDef.getWindowFrame().getEnd();
@@ -691,10 +694,10 @@ public class WindowingTableFunction extends TableFunctionEvaluator {
     else {
       ValueBoundaryScanner vbs;
       if ( startB instanceof ValueBoundaryDef ) {
-        vbs = ValueBoundaryScanner.getScanner((ValueBoundaryDef)startB, order);
+        vbs = ValueBoundaryScanner.getScanner((ValueBoundaryDef)startB, order, nullOrder);
       }
       else {
-        vbs = ValueBoundaryScanner.getScanner((ValueBoundaryDef)endB, order);
+        vbs = ValueBoundaryScanner.getScanner((ValueBoundaryDef)endB, order, nullOrder);
       }
       vbs.reset(startB);
       start =  vbs.computeStart(currRow, p);
@@ -775,12 +778,14 @@ public class WindowingTableFunction extends TableFunctionEvaluator {
   {
     BoundaryDef bndDef;
     Order order;
+    NullOrder nullOrder;
     PTFExpressionDef expressionDef;
 
-    public ValueBoundaryScanner(BoundaryDef bndDef, Order order, PTFExpressionDef expressionDef)
+    public ValueBoundaryScanner(BoundaryDef bndDef, Order order, NullOrder nullOrder, PTFExpressionDef expressionDef)
     {
       this.bndDef = bndDef;
       this.order = order;
+      this.nullOrder = nullOrder;
       this.expressionDef = expressionDef;
     }
 
@@ -1135,7 +1140,7 @@ public class WindowingTableFunction extends TableFunctionEvaluator {
 
 
     @SuppressWarnings("incomplete-switch")
-    public static ValueBoundaryScanner getScanner(ValueBoundaryDef vbDef, Order order)
+    public static ValueBoundaryScanner getScanner(ValueBoundaryDef vbDef, Order order, NullOrder nullOrder)
         throws HiveException {
       PrimitiveObjectInspector pOI = (PrimitiveObjectInspector) vbDef.getOI();
       switch(pOI.getPrimitiveCategory()) {
@@ -1144,16 +1149,16 @@ public class WindowingTableFunction extends TableFunctionEvaluator {
       case LONG:
       case SHORT:
       case TIMESTAMP:
-        return new LongValueBoundaryScanner(vbDef, order, vbDef.getExpressionDef());
+        return new LongValueBoundaryScanner(vbDef, order, nullOrder, vbDef.getExpressionDef());
       case DOUBLE:
       case FLOAT:
-        return new DoubleValueBoundaryScanner(vbDef, order, vbDef.getExpressionDef());
+        return new DoubleValueBoundaryScanner(vbDef, order, nullOrder, vbDef.getExpressionDef());
       case DECIMAL:
-        return new HiveDecimalValueBoundaryScanner(vbDef, order, vbDef.getExpressionDef());
+        return new HiveDecimalValueBoundaryScanner(vbDef, order, nullOrder, vbDef.getExpressionDef());
       case DATE:
-        return new DateValueBoundaryScanner(vbDef, order, vbDef.getExpressionDef());
+        return new DateValueBoundaryScanner(vbDef, order, nullOrder, vbDef.getExpressionDef());
       case STRING:
-        return new StringValueBoundaryScanner(vbDef, order, vbDef.getExpressionDef());
+        return new StringValueBoundaryScanner(vbDef, order, nullOrder, vbDef.getExpressionDef());
       }
       throw new HiveException(
           String.format("Internal Error: attempt to setup a Window for datatype %s",
@@ -1162,9 +1167,9 @@ public class WindowingTableFunction extends TableFunctionEvaluator {
   }
 
   public static class LongValueBoundaryScanner extends ValueBoundaryScanner {
-    public LongValueBoundaryScanner(BoundaryDef bndDef, Order order,
+    public LongValueBoundaryScanner(BoundaryDef bndDef, Order order, NullOrder nullOrder,
         PTFExpressionDef expressionDef) {
-      super(bndDef,order,expressionDef);
+      super(bndDef,order,nullOrder,expressionDef);
     }
 
     @Override
@@ -1196,8 +1201,8 @@ public class WindowingTableFunction extends TableFunctionEvaluator {
 
   public static class DoubleValueBoundaryScanner extends ValueBoundaryScanner {
     public DoubleValueBoundaryScanner(BoundaryDef bndDef, Order order,
-        PTFExpressionDef expressionDef) {
-      super(bndDef,order,expressionDef);
+        NullOrder nullOrder, PTFExpressionDef expressionDef) {
+      super(bndDef,order,nullOrder,expressionDef);
     }
 
     @Override
@@ -1229,8 +1234,8 @@ public class WindowingTableFunction extends TableFunctionEvaluator {
 
   public static class HiveDecimalValueBoundaryScanner extends ValueBoundaryScanner {
     public HiveDecimalValueBoundaryScanner(BoundaryDef bndDef, Order order,
-        PTFExpressionDef expressionDef) {
-      super(bndDef,order,expressionDef);
+        NullOrder nullOrder, PTFExpressionDef expressionDef) {
+      super(bndDef,order,nullOrder,expressionDef);
     }
 
     @Override
@@ -1262,8 +1267,8 @@ public class WindowingTableFunction extends TableFunctionEvaluator {
 
   public static class DateValueBoundaryScanner extends ValueBoundaryScanner {
     public DateValueBoundaryScanner(BoundaryDef bndDef, Order order,
-        PTFExpressionDef expressionDef) {
-      super(bndDef,order,expressionDef);
+        NullOrder nullOrder, PTFExpressionDef expressionDef) {
+      super(bndDef,order,nullOrder,expressionDef);
     }
 
     @Override
@@ -1290,8 +1295,8 @@ public class WindowingTableFunction extends TableFunctionEvaluator {
 
   public static class StringValueBoundaryScanner extends ValueBoundaryScanner {
     public StringValueBoundaryScanner(BoundaryDef bndDef, Order order,
-        PTFExpressionDef expressionDef) {
-      super(bndDef,order,expressionDef);
+        NullOrder nullOrder, PTFExpressionDef expressionDef) {
+      super(bndDef,order,nullOrder,expressionDef);
     }
 
     @Override
@@ -1347,6 +1352,7 @@ public class WindowingTableFunction extends TableFunctionEvaluator {
     int[] wFnsToProcess;
     WindowTableFunctionDef wTFnDef;
     Order order;
+    NullOrder nullOrder;
     PTFDesc ptfDesc;
     StructObjectInspector inputOI;
     AggregationBuffer[] aggBuffers;
@@ -1362,6 +1368,7 @@ public class WindowingTableFunction extends TableFunctionEvaluator {
       this.currIdx = 0;
       wTFnDef = (WindowTableFunctionDef) getTableDef();
       order = wTFnDef.getOrder().getExpressions().get(0).getOrder();
+      nullOrder = wTFnDef.getOrder().getExpressions().get(0).getNullOrder();
       ptfDesc = getQueryDef();
       inputOI = iPart.getOutputOI();
 
@@ -1416,7 +1423,7 @@ public class WindowingTableFunction extends TableFunctionEvaluator {
             out = ObjectInspectorUtils.copyToStandardObject(out, wFn.getOI());
             output.set(j, out);
           } else {
-            Range rng = getRange(wFn, currIdx, iPart, order);
+            Range rng = getRange(wFn, currIdx, iPart, order, nullOrder);
             PTFPartitionIterator<Object> rItr = rng.iterator();
             PTFOperator.connectLeadLagFunctionsToPartition(ptfDesc, rItr);
             output.set(j, evaluateWindowFunction(wFn, rItr));
@@ -1453,6 +1460,7 @@ public class WindowingTableFunction extends TableFunctionEvaluator {
     AggregationBuffer[] aggBuffers;
     Object[][] funcArgs;
     Order order;
+    NullOrder nullOrder;
     RankLimit rnkLimit;
 
     @SuppressWarnings("unchecked")
@@ -1467,6 +1475,7 @@ public class WindowingTableFunction extends TableFunctionEvaluator {
           precedingSpan, followingSpan);
 
       order = tabDef.getOrder().getExpressions().get(0).getOrder();
+      nullOrder = tabDef.getOrder().getExpressions().get(0).getNullOrder();
 
       int numFns = tabDef.getWindowFunctions().size();
       fnOutputs = new ArrayList[numFns];

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestWorker.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestWorker.java b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestWorker.java
index cf7eb70..7f6430f 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestWorker.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestWorker.java
@@ -224,7 +224,7 @@ public class TestWorker extends CompactorTest {
   @Test
   public void sortedTable() throws Exception {
     List<Order> sortCols = new ArrayList<Order>(1);
-    sortCols.add(new Order("b", 1));
+    sortCols.add(new Order("b", 1, 0));
 
     Table t = newTable("default", "st", false, new HashMap<String, String>(), sortCols, false);
 
@@ -249,7 +249,7 @@ public class TestWorker extends CompactorTest {
   @Test
   public void sortedPartition() throws Exception {
     List<Order> sortCols = new ArrayList<Order>(1);
-    sortCols.add(new Order("b", 1));
+    sortCols.add(new Order("b", 1, 0));
 
     Table t = newTable("default", "sp", true, new HashMap<String, String>(), sortCols, false);
     Partition p = newPartition(t, "today", sortCols);

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/queries/clientpositive/keyword_3.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/keyword_3.q b/ql/src/test/queries/clientpositive/keyword_3.q
new file mode 100644
index 0000000..098d0f9
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/keyword_3.q
@@ -0,0 +1,8 @@
+drop table NULLS;
+
+create table NULLS (LAST string);
+
+insert overwrite table NULLS
+  select key from src where key = '238' limit 1;
+
+select LAST from NULLS;

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/queries/clientpositive/order_null.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/order_null.q b/ql/src/test/queries/clientpositive/order_null.q
new file mode 100644
index 0000000..ea833e0
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/order_null.q
@@ -0,0 +1,29 @@
+create table src_null (a int, b string);
+insert into src_null values (1, 'A');
+insert into src_null values (null, null);
+insert into src_null values (3, null);
+insert into src_null values (2, null);
+insert into src_null values (2, 'A');
+insert into src_null values (2, 'B');
+
+SELECT x.* FROM src_null x ORDER BY a asc;
+
+SELECT x.* FROM src_null x ORDER BY a desc;
+
+SELECT x.* FROM src_null x ORDER BY b asc, a asc nulls last;
+
+SELECT x.* FROM src_null x ORDER BY b desc, a asc;
+
+SELECT x.* FROM src_null x ORDER BY a asc nulls first;
+
+SELECT x.* FROM src_null x ORDER BY a desc nulls first;
+
+SELECT x.* FROM src_null x ORDER BY b asc nulls last, a;
+
+SELECT x.* FROM src_null x ORDER BY b desc nulls last, a;
+
+SELECT x.* FROM src_null x ORDER BY a asc nulls last, b desc;
+
+SELECT x.* FROM src_null x ORDER BY b desc nulls last, a desc nulls last;
+
+SELECT x.* FROM src_null x ORDER BY b asc nulls first, a asc nulls last;

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/queries/clientpositive/windowing_order_null.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/windowing_order_null.q b/ql/src/test/queries/clientpositive/windowing_order_null.q
new file mode 100644
index 0000000..6d8ece0
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/windowing_order_null.q
@@ -0,0 +1,35 @@
+drop table over10k;
+
+create table over10k(
+           t tinyint,
+           si smallint,
+           i int,
+           b bigint,
+           f float,
+           d double,
+           bo boolean,
+           s string,
+	   ts timestamp, 
+           dec decimal,  
+           bin binary)
+       row format delimited
+       fields terminated by '|';
+
+load data local inpath '../../data/files/over10k' into table over10k;
+load data local inpath '../../data/files/over4_null' into table over10k;
+
+select i, s, b, sum(b) over (partition by i order by s nulls last,b rows unbounded preceding) from over10k limit 10;
+
+select d, s, f, sum(f) over (partition by d order by s,f desc nulls first rows unbounded preceding) from over10k limit 10;
+
+select ts, s, f, sum(f) over (partition by ts order by f asc nulls first range between current row and unbounded following) from over10k limit 10;
+
+select t, s, d, avg(d) over (partition by t order by s,d desc nulls first rows between 5 preceding and 5 following) from over10k limit 10;
+
+select ts, s, sum(i) over(partition by ts order by s nulls last) from over10k limit 10 offset 3;
+
+select s, i, round(sum(d) over (partition by s order by i desc nulls last) , 3) from over10k limit 5;
+
+select s, i, round(avg(d) over (partition by s order by i desc nulls last) / 10.0 , 3) from over10k limit 5;
+
+select s, i, round((avg(d) over  w1 + 10.0) - (avg(d) over w1 - 10.0),3) from over10k window w1 as (partition by s order by i nulls last) limit 5;

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/alter_numbuckets_partitioned_table2_h23.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/alter_numbuckets_partitioned_table2_h23.q.out b/ql/src/test/results/clientpositive/alter_numbuckets_partitioned_table2_h23.q.out
index 3b71598..9eea7f7 100644
--- a/ql/src/test/results/clientpositive/alter_numbuckets_partitioned_table2_h23.q.out
+++ b/ql/src/test/results/clientpositive/alter_numbuckets_partitioned_table2_h23.q.out
@@ -232,7 +232,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	8                   	 
 Bucket Columns:     	[key]               	 
-Sort Columns:       	[Order(col:key, order:0)]	 
+Sort Columns:       	[Order(col:key, order:0, nullOrder:1)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: INSERT OVERWRITE TABLE tst1 PARTITION (ds = '1') SELECT key, value FROM src
@@ -281,7 +281,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	8                   	 
 Bucket Columns:     	[key]               	 
-Sort Columns:       	[Order(col:key, order:0)]	 
+Sort Columns:       	[Order(col:key, order:0, nullOrder:1)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: -- Test changing the bucket columns
@@ -326,7 +326,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	8                   	 
 Bucket Columns:     	[value]             	 
-Sort Columns:       	[Order(col:key, order:0)]	 
+Sort Columns:       	[Order(col:key, order:0, nullOrder:1)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: INSERT OVERWRITE TABLE tst1 PARTITION (ds = '1') SELECT key, value FROM src
@@ -375,7 +375,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	8                   	 
 Bucket Columns:     	[value]             	 
-Sort Columns:       	[Order(col:key, order:0)]	 
+Sort Columns:       	[Order(col:key, order:0, nullOrder:1)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: -- Test changing the number of buckets
@@ -420,7 +420,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	4                   	 
 Bucket Columns:     	[value]             	 
-Sort Columns:       	[Order(col:key, order:0)]	 
+Sort Columns:       	[Order(col:key, order:0, nullOrder:1)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: INSERT OVERWRITE TABLE tst1 PARTITION (ds = '1') SELECT key, value FROM src
@@ -469,7 +469,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	4                   	 
 Bucket Columns:     	[value]             	 
-Sort Columns:       	[Order(col:key, order:0)]	 
+Sort Columns:       	[Order(col:key, order:0, nullOrder:1)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: -- Test changing the sort columns
@@ -514,7 +514,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	4                   	 
 Bucket Columns:     	[value]             	 
-Sort Columns:       	[Order(col:value, order:0)]	 
+Sort Columns:       	[Order(col:value, order:0, nullOrder:1)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: INSERT OVERWRITE TABLE tst1 PARTITION (ds = '1') SELECT key, value FROM src
@@ -563,7 +563,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	4                   	 
 Bucket Columns:     	[value]             	 
-Sort Columns:       	[Order(col:value, order:0)]	 
+Sort Columns:       	[Order(col:value, order:0, nullOrder:1)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: -- Test changing the sort order
@@ -608,7 +608,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	4                   	 
 Bucket Columns:     	[value]             	 
-Sort Columns:       	[Order(col:value, order:1)]	 
+Sort Columns:       	[Order(col:value, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: INSERT OVERWRITE TABLE tst1 PARTITION (ds = '1') SELECT key, value FROM src
@@ -657,7 +657,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	4                   	 
 Bucket Columns:     	[value]             	 
-Sort Columns:       	[Order(col:value, order:1)]	 
+Sort Columns:       	[Order(col:value, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: -- Test a sorted partition gets converted to unsorted

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/alter_numbuckets_partitioned_table_h23.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/alter_numbuckets_partitioned_table_h23.q.out b/ql/src/test/results/clientpositive/alter_numbuckets_partitioned_table_h23.q.out
index cab3de4..af2f47a 100644
--- a/ql/src/test/results/clientpositive/alter_numbuckets_partitioned_table_h23.q.out
+++ b/ql/src/test/results/clientpositive/alter_numbuckets_partitioned_table_h23.q.out
@@ -408,7 +408,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	12                  	 
 Bucket Columns:     	[key]               	 
-Sort Columns:       	[Order(col:key, order:1)]	 
+Sort Columns:       	[Order(col:key, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: -- Test changing sort order
@@ -455,7 +455,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	12                  	 
 Bucket Columns:     	[key]               	 
-Sort Columns:       	[Order(col:value, order:0)]	 
+Sort Columns:       	[Order(col:value, order:0, nullOrder:1)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: -- Test removing test order

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/alter_partition_clusterby_sortby.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/alter_partition_clusterby_sortby.q.out b/ql/src/test/results/clientpositive/alter_partition_clusterby_sortby.q.out
index 184d2e4..46d5b34 100644
--- a/ql/src/test/results/clientpositive/alter_partition_clusterby_sortby.q.out
+++ b/ql/src/test/results/clientpositive/alter_partition_clusterby_sortby.q.out
@@ -110,7 +110,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	4                   	 
 Bucket Columns:     	[b]                 	 
-Sort Columns:       	[Order(col:b, order:0)]	 
+Sort Columns:       	[Order(col:b, order:0, nullOrder:1)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: -- Turn off clustering for a partition
@@ -200,7 +200,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	4                   	 
 Bucket Columns:     	[a, b]              	 
-Sort Columns:       	[Order(col:a, order:0), Order(col:b, order:1)]	 
+Sort Columns:       	[Order(col:a, order:0, nullOrder:1), Order(col:b, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: drop table alter_table_partition_clusterby_sortby

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/alter_partition_coltype.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/alter_partition_coltype.q.out b/ql/src/test/results/clientpositive/alter_partition_coltype.q.out
index 20ef536..a184372 100644
--- a/ql/src/test/results/clientpositive/alter_partition_coltype.q.out
+++ b/ql/src/test/results/clientpositive/alter_partition_coltype.q.out
@@ -131,6 +131,7 @@ STAGE PLANS:
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
+                  null sort order: 
                   sort order: 
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   tag: -1
@@ -357,6 +358,7 @@ STAGE PLANS:
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
+                  null sort order: 
                   sort order: 
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   tag: -1
@@ -515,6 +517,7 @@ STAGE PLANS:
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
+                  null sort order: 
                   sort order: 
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/alter_table_not_sorted.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/alter_table_not_sorted.q.out b/ql/src/test/results/clientpositive/alter_table_not_sorted.q.out
index 6e1ec59..8a88bc4 100644
--- a/ql/src/test/results/clientpositive/alter_table_not_sorted.q.out
+++ b/ql/src/test/results/clientpositive/alter_table_not_sorted.q.out
@@ -34,7 +34,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	4                   	 
 Bucket Columns:     	[a]                 	 
-Sort Columns:       	[Order(col:a, order:1)]	 
+Sort Columns:       	[Order(col:a, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: alter table alter_table_not_sorted not sorted

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/authorization_index.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/authorization_index.q.out b/ql/src/test/results/clientpositive/authorization_index.q.out
index adc02ad..a6ec998 100644
--- a/ql/src/test/results/clientpositive/authorization_index.q.out
+++ b/ql/src/test/results/clientpositive/authorization_index.q.out
@@ -41,7 +41,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	-1                  	 
 Bucket Columns:     	[]                  	 
-Sort Columns:       	[Order(col:a, order:1)]	 
+Sort Columns:       	[Order(col:a, order:1, nullOrder:0)]	 
 PREHOOK: query: alter index t1_index on t1 rebuild
 PREHOOK: type: ALTERINDEX_REBUILD
 PREHOOK: Input: default@t1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/auto_join_reordering_values.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/auto_join_reordering_values.q.out b/ql/src/test/results/clientpositive/auto_join_reordering_values.q.out
index 22c8300..ac349a4 100644
--- a/ql/src/test/results/clientpositive/auto_join_reordering_values.q.out
+++ b/ql/src/test/results/clientpositive/auto_join_reordering_values.q.out
@@ -192,6 +192,7 @@ STAGE PLANS:
                 Statistics: Num rows: 1 Data size: 36 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col1 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col1 (type: string)
                   Statistics: Num rows: 1 Data size: 36 Basic stats: COMPLETE Column stats: NONE
@@ -212,6 +213,7 @@ STAGE PLANS:
                 Statistics: Num rows: 1 Data size: 36 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: string)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 1 Data size: 36 Basic stats: COMPLETE Column stats: NONE
@@ -301,6 +303,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col0 (type: int)
+              null sort order: a
               sort order: +
               Map-reduce partition columns: _col0 (type: int)
               Statistics: Num rows: 1 Data size: 39 Basic stats: COMPLETE Column stats: NONE
@@ -321,6 +324,7 @@ STAGE PLANS:
                 Statistics: Num rows: 1 Data size: 36 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 1 Data size: 36 Basic stats: COMPLETE Column stats: NONE
@@ -431,6 +435,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col2 (type: int)
+              null sort order: a
               sort order: +
               Map-reduce partition columns: _col2 (type: int)
               Statistics: Num rows: 1 Data size: 42 Basic stats: COMPLETE Column stats: NONE
@@ -451,6 +456,7 @@ STAGE PLANS:
                 Statistics: Num rows: 1 Data size: 36 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 1 Data size: 36 Basic stats: COMPLETE Column stats: NONE
@@ -561,6 +567,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col3 (type: int)
+              null sort order: a
               sort order: +
               Map-reduce partition columns: _col3 (type: int)
               Statistics: Num rows: 1 Data size: 46 Basic stats: COMPLETE Column stats: NONE
@@ -581,6 +588,7 @@ STAGE PLANS:
                 Statistics: Num rows: 100 Data size: 288 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
+                  null sort order: a
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 100 Data size: 288 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/auto_sortmerge_join_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/auto_sortmerge_join_1.q.out b/ql/src/test/results/clientpositive/auto_sortmerge_join_1.q.out
index 3a91947..f768ea4 100644
--- a/ql/src/test/results/clientpositive/auto_sortmerge_join_1.q.out
+++ b/ql/src/test/results/clientpositive/auto_sortmerge_join_1.q.out
@@ -174,6 +174,7 @@ STAGE PLANS:
                     mode: hash
                     outputColumnNames: _col0
                     Reduce Output Operator
+                      null sort order: 
                       sort order: 
                       tag: -1
                       value expressions: _col0 (type: bigint)
@@ -394,6 +395,7 @@ STAGE PLANS:
                     mode: hash
                     outputColumnNames: _col0
                     Reduce Output Operator
+                      null sort order: 
                       sort order: 
                       tag: -1
                       value expressions: _col0 (type: bigint)
@@ -691,6 +693,7 @@ STAGE PLANS:
                     mode: hash
                     outputColumnNames: _col0
                     Reduce Output Operator
+                      null sort order: 
                       sort order: 
                       tag: -1
                       value expressions: _col0 (type: bigint)
@@ -1007,6 +1010,7 @@ STAGE PLANS:
                     mode: hash
                     outputColumnNames: _col0
                     Reduce Output Operator
+                      null sort order: 
                       sort order: 
                       tag: -1
                       value expressions: _col0 (type: bigint)
@@ -1208,6 +1212,7 @@ STAGE PLANS:
                     mode: hash
                     outputColumnNames: _col0
                     Reduce Output Operator
+                      null sort order: 
                       sort order: 
                       tag: -1
                       value expressions: _col0 (type: bigint)

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/auto_sortmerge_join_11.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/auto_sortmerge_join_11.q.out b/ql/src/test/results/clientpositive/auto_sortmerge_join_11.q.out
index efdd94a..f4bbfd0 100644
--- a/ql/src/test/results/clientpositive/auto_sortmerge_join_11.q.out
+++ b/ql/src/test/results/clientpositive/auto_sortmerge_join_11.q.out
@@ -242,6 +242,7 @@ STAGE PLANS:
                     outputColumnNames: _col0
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
+                      null sort order: 
                       sort order: 
                       Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       tag: -1
@@ -589,6 +590,7 @@ STAGE PLANS:
                     outputColumnNames: _col0
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
+                      null sort order: 
                       sort order: 
                       Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       tag: -1
@@ -934,6 +936,7 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
+                    null sort order: 
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                     tag: -1
@@ -1357,6 +1360,7 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
+                    null sort order: 
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                     tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/auto_sortmerge_join_12.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/auto_sortmerge_join_12.q.out b/ql/src/test/results/clientpositive/auto_sortmerge_join_12.q.out
index 33e2def..e5ff904 100644
--- a/ql/src/test/results/clientpositive/auto_sortmerge_join_12.q.out
+++ b/ql/src/test/results/clientpositive/auto_sortmerge_join_12.q.out
@@ -447,6 +447,7 @@ STAGE PLANS:
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
+                        null sort order: 
                         sort order: 
                         Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                         tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/auto_sortmerge_join_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/auto_sortmerge_join_2.q.out b/ql/src/test/results/clientpositive/auto_sortmerge_join_2.q.out
index fbbe9c4..0de079d 100644
--- a/ql/src/test/results/clientpositive/auto_sortmerge_join_2.q.out
+++ b/ql/src/test/results/clientpositive/auto_sortmerge_join_2.q.out
@@ -154,6 +154,7 @@ STAGE PLANS:
                     mode: hash
                     outputColumnNames: _col0
                     Reduce Output Operator
+                      null sort order: 
                       sort order: 
                       tag: -1
                       value expressions: _col0 (type: bigint)
@@ -453,6 +454,7 @@ STAGE PLANS:
                     mode: hash
                     outputColumnNames: _col0
                     Reduce Output Operator
+                      null sort order: 
                       sort order: 
                       tag: -1
                       value expressions: _col0 (type: bigint)
@@ -769,6 +771,7 @@ STAGE PLANS:
                     mode: hash
                     outputColumnNames: _col0
                     Reduce Output Operator
+                      null sort order: 
                       sort order: 
                       tag: -1
                       value expressions: _col0 (type: bigint)
@@ -970,6 +973,7 @@ STAGE PLANS:
                     mode: hash
                     outputColumnNames: _col0
                     Reduce Output Operator
+                      null sort order: 
                       sort order: 
                       tag: -1
                       value expressions: _col0 (type: bigint)

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/auto_sortmerge_join_3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/auto_sortmerge_join_3.q.out b/ql/src/test/results/clientpositive/auto_sortmerge_join_3.q.out
index 52cea36..a161f1c 100644
--- a/ql/src/test/results/clientpositive/auto_sortmerge_join_3.q.out
+++ b/ql/src/test/results/clientpositive/auto_sortmerge_join_3.q.out
@@ -154,6 +154,7 @@ STAGE PLANS:
                     mode: hash
                     outputColumnNames: _col0
                     Reduce Output Operator
+                      null sort order: 
                       sort order: 
                       tag: -1
                       value expressions: _col0 (type: bigint)
@@ -328,6 +329,7 @@ STAGE PLANS:
                     mode: hash
                     outputColumnNames: _col0
                     Reduce Output Operator
+                      null sort order: 
                       sort order: 
                       tag: -1
                       value expressions: _col0 (type: bigint)
@@ -622,6 +624,7 @@ STAGE PLANS:
                     mode: hash
                     outputColumnNames: _col0
                     Reduce Output Operator
+                      null sort order: 
                       sort order: 
                       tag: -1
                       value expressions: _col0 (type: bigint)
@@ -892,6 +895,7 @@ STAGE PLANS:
                     mode: hash
                     outputColumnNames: _col0
                     Reduce Output Operator
+                      null sort order: 
                       sort order: 
                       tag: -1
                       value expressions: _col0 (type: bigint)
@@ -1093,6 +1097,7 @@ STAGE PLANS:
                     mode: hash
                     outputColumnNames: _col0
                     Reduce Output Operator
+                      null sort order: 
                       sort order: 
                       tag: -1
                       value expressions: _col0 (type: bigint)

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/auto_sortmerge_join_4.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/auto_sortmerge_join_4.q.out b/ql/src/test/results/clientpositive/auto_sortmerge_join_4.q.out
index c50865e..3421ab1 100644
--- a/ql/src/test/results/clientpositive/auto_sortmerge_join_4.q.out
+++ b/ql/src/test/results/clientpositive/auto_sortmerge_join_4.q.out
@@ -170,6 +170,7 @@ STAGE PLANS:
                     mode: hash
                     outputColumnNames: _col0
                     Reduce Output Operator
+                      null sort order: 
                       sort order: 
                       tag: -1
                       value expressions: _col0 (type: bigint)
@@ -344,6 +345,7 @@ STAGE PLANS:
                     mode: hash
                     outputColumnNames: _col0
                     Reduce Output Operator
+                      null sort order: 
                       sort order: 
                       tag: -1
                       value expressions: _col0 (type: bigint)
@@ -638,6 +640,7 @@ STAGE PLANS:
                     mode: hash
                     outputColumnNames: _col0
                     Reduce Output Operator
+                      null sort order: 
                       sort order: 
                       tag: -1
                       value expressions: _col0 (type: bigint)
@@ -908,6 +911,7 @@ STAGE PLANS:
                     mode: hash
                     outputColumnNames: _col0
                     Reduce Output Operator
+                      null sort order: 
                       sort order: 
                       tag: -1
                       value expressions: _col0 (type: bigint)
@@ -1109,6 +1113,7 @@ STAGE PLANS:
                     mode: hash
                     outputColumnNames: _col0
                     Reduce Output Operator
+                      null sort order: 
                       sort order: 
                       tag: -1
                       value expressions: _col0 (type: bigint)

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/auto_sortmerge_join_5.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/auto_sortmerge_join_5.q.out b/ql/src/test/results/clientpositive/auto_sortmerge_join_5.q.out
index 2e63698..9c2663c 100644
--- a/ql/src/test/results/clientpositive/auto_sortmerge_join_5.q.out
+++ b/ql/src/test/results/clientpositive/auto_sortmerge_join_5.q.out
@@ -141,6 +141,7 @@ STAGE PLANS:
                     mode: hash
                     outputColumnNames: _col0
                     Reduce Output Operator
+                      null sort order: 
                       sort order: 
                       tag: -1
                       value expressions: _col0 (type: bigint)
@@ -306,6 +307,7 @@ STAGE PLANS:
                     mode: hash
                     outputColumnNames: _col0
                     Reduce Output Operator
+                      null sort order: 
                       sort order: 
                       tag: -1
                       value expressions: _col0 (type: bigint)
@@ -504,6 +506,7 @@ STAGE PLANS:
                     mode: hash
                     outputColumnNames: _col0
                     Reduce Output Operator
+                      null sort order: 
                       sort order: 
                       tag: -1
                       value expressions: _col0 (type: bigint)
@@ -675,6 +678,7 @@ STAGE PLANS:
                     mode: hash
                     outputColumnNames: _col0
                     Reduce Output Operator
+                      null sort order: 
                       sort order: 
                       tag: -1
                       value expressions: _col0 (type: bigint)
@@ -821,6 +825,7 @@ STAGE PLANS:
                     mode: hash
                     outputColumnNames: _col0
                     Reduce Output Operator
+                      null sort order: 
                       sort order: 
                       tag: -1
                       value expressions: _col0 (type: bigint)

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/auto_sortmerge_join_7.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/auto_sortmerge_join_7.q.out b/ql/src/test/results/clientpositive/auto_sortmerge_join_7.q.out
index 2675eca..a25db01 100644
--- a/ql/src/test/results/clientpositive/auto_sortmerge_join_7.q.out
+++ b/ql/src/test/results/clientpositive/auto_sortmerge_join_7.q.out
@@ -187,6 +187,7 @@ STAGE PLANS:
                     mode: hash
                     outputColumnNames: _col0
                     Reduce Output Operator
+                      null sort order: 
                       sort order: 
                       tag: -1
                       value expressions: _col0 (type: bigint)
@@ -409,6 +410,7 @@ STAGE PLANS:
                     mode: hash
                     outputColumnNames: _col0
                     Reduce Output Operator
+                      null sort order: 
                       sort order: 
                       tag: -1
                       value expressions: _col0 (type: bigint)
@@ -751,6 +753,7 @@ STAGE PLANS:
                     mode: hash
                     outputColumnNames: _col0
                     Reduce Output Operator
+                      null sort order: 
                       sort order: 
                       tag: -1
                       value expressions: _col0 (type: bigint)
@@ -1111,6 +1114,7 @@ STAGE PLANS:
                     mode: hash
                     outputColumnNames: _col0
                     Reduce Output Operator
+                      null sort order: 
                       sort order: 
                       tag: -1
                       value expressions: _col0 (type: bigint)
@@ -1357,6 +1361,7 @@ STAGE PLANS:
                     mode: hash
                     outputColumnNames: _col0
                     Reduce Output Operator
+                      null sort order: 
                       sort order: 
                       tag: -1
                       value expressions: _col0 (type: bigint)

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/auto_sortmerge_join_8.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/auto_sortmerge_join_8.q.out b/ql/src/test/results/clientpositive/auto_sortmerge_join_8.q.out
index 9de9d73..38996a7 100644
--- a/ql/src/test/results/clientpositive/auto_sortmerge_join_8.q.out
+++ b/ql/src/test/results/clientpositive/auto_sortmerge_join_8.q.out
@@ -187,6 +187,7 @@ STAGE PLANS:
                     mode: hash
                     outputColumnNames: _col0
                     Reduce Output Operator
+                      null sort order: 
                       sort order: 
                       tag: -1
                       value expressions: _col0 (type: bigint)
@@ -409,6 +410,7 @@ STAGE PLANS:
                     mode: hash
                     outputColumnNames: _col0
                     Reduce Output Operator
+                      null sort order: 
                       sort order: 
                       tag: -1
                       value expressions: _col0 (type: bigint)
@@ -753,6 +755,7 @@ STAGE PLANS:
                     mode: hash
                     outputColumnNames: _col0
                     Reduce Output Operator
+                      null sort order: 
                       sort order: 
                       tag: -1
                       value expressions: _col0 (type: bigint)
@@ -1113,6 +1116,7 @@ STAGE PLANS:
                     mode: hash
                     outputColumnNames: _col0
                     Reduce Output Operator
+                      null sort order: 
                       sort order: 
                       tag: -1
                       value expressions: _col0 (type: bigint)
@@ -1359,6 +1363,7 @@ STAGE PLANS:
                     mode: hash
                     outputColumnNames: _col0
                     Reduce Output Operator
+                      null sort order: 
                       sort order: 
                       tag: -1
                       value expressions: _col0 (type: bigint)

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/bucket1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/bucket1.q.out b/ql/src/test/results/clientpositive/bucket1.q.out
index 9e03236..0680176 100644
--- a/ql/src/test/results/clientpositive/bucket1.q.out
+++ b/ql/src/test/results/clientpositive/bucket1.q.out
@@ -53,6 +53,7 @@ STAGE PLANS:
               outputColumnNames: _col0, _col1
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
+                null sort order: 
                 sort order: 
                 Map-reduce partition columns: UDFToInteger(_col0) (type: int)
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/bucket2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/bucket2.q.out b/ql/src/test/results/clientpositive/bucket2.q.out
index 374b0e1..958e556 100644
--- a/ql/src/test/results/clientpositive/bucket2.q.out
+++ b/ql/src/test/results/clientpositive/bucket2.q.out
@@ -53,6 +53,7 @@ STAGE PLANS:
               outputColumnNames: _col0, _col1
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
+                null sort order: 
                 sort order: 
                 Map-reduce partition columns: UDFToInteger(_col0) (type: int)
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/bucket3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/bucket3.q.out b/ql/src/test/results/clientpositive/bucket3.q.out
index 542bbb7..eca9ba5 100644
--- a/ql/src/test/results/clientpositive/bucket3.q.out
+++ b/ql/src/test/results/clientpositive/bucket3.q.out
@@ -57,6 +57,7 @@ STAGE PLANS:
               outputColumnNames: _col0, _col1
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
+                null sort order: 
                 sort order: 
                 Map-reduce partition columns: UDFToInteger(_col0) (type: int)
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/bucket4.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/bucket4.q.out b/ql/src/test/results/clientpositive/bucket4.q.out
index 5c7126b..dd9ef00 100644
--- a/ql/src/test/results/clientpositive/bucket4.q.out
+++ b/ql/src/test/results/clientpositive/bucket4.q.out
@@ -50,6 +50,7 @@ STAGE PLANS:
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
                 key expressions: UDFToInteger(_col0) (type: int)
+                null sort order: a
                 sort order: +
                 Map-reduce partition columns: UDFToInteger(_col0) (type: int)
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/bucket5.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/bucket5.q.out b/ql/src/test/results/clientpositive/bucket5.q.out
index 46e3d54..aa47ba8 100644
--- a/ql/src/test/results/clientpositive/bucket5.q.out
+++ b/ql/src/test/results/clientpositive/bucket5.q.out
@@ -93,6 +93,7 @@ STAGE PLANS:
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
                 key expressions: UDFToInteger(_col0) (type: int)
+                null sort order: a
                 sort order: +
                 Map-reduce partition columns: UDFToInteger(_col0) (type: int)
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -239,6 +240,7 @@ STAGE PLANS:
             GatherStats: false
             Reduce Output Operator
               key expressions: _col0 (type: string)
+              null sort order: a
               sort order: +
               Map-reduce partition columns: _col0 (type: string)
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -533,7 +535,7 @@ OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 Compressed:         	No                  	 
 Num Buckets:        	2                   	 
 Bucket Columns:     	[key]               	 
-Sort Columns:       	[Order(col:key, order:1)]	 
+Sort Columns:       	[Order(col:key, order:1, nullOrder:0)]	 
 Storage Desc Params:	 	 
 	serialization.format	1                   
 PREHOOK: query: SELECT * FROM bucketed_table TABLESAMPLE (BUCKET 1 OUT OF 2) s LIMIT 10

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/bucket_many.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/bucket_many.q.out b/ql/src/test/results/clientpositive/bucket_many.q.out
index ecd5e29..a9e2f0b 100644
--- a/ql/src/test/results/clientpositive/bucket_many.q.out
+++ b/ql/src/test/results/clientpositive/bucket_many.q.out
@@ -49,6 +49,7 @@ STAGE PLANS:
               outputColumnNames: _col0, _col1
               Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
               Reduce Output Operator
+                null sort order: 
                 sort order: 
                 Map-reduce partition columns: UDFToInteger(_col0) (type: int)
                 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/bucket_map_join_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/bucket_map_join_1.q.out b/ql/src/test/results/clientpositive/bucket_map_join_1.q.out
index 5f1e9b9..05fb434 100644
--- a/ql/src/test/results/clientpositive/bucket_map_join_1.q.out
+++ b/ql/src/test/results/clientpositive/bucket_map_join_1.q.out
@@ -156,6 +156,7 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
+                    null sort order: 
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                     tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/bucket_map_join_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/bucket_map_join_2.q.out b/ql/src/test/results/clientpositive/bucket_map_join_2.q.out
index 07958ce..1173c93 100644
--- a/ql/src/test/results/clientpositive/bucket_map_join_2.q.out
+++ b/ql/src/test/results/clientpositive/bucket_map_join_2.q.out
@@ -156,6 +156,7 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
+                    null sort order: 
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                     tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/bucketcontext_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/bucketcontext_1.q.out b/ql/src/test/results/clientpositive/bucketcontext_1.q.out
index 8b2ebb0..78a0c8c 100644
--- a/ql/src/test/results/clientpositive/bucketcontext_1.q.out
+++ b/ql/src/test/results/clientpositive/bucketcontext_1.q.out
@@ -245,6 +245,7 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
+                    null sort order: 
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                     tag: -1
@@ -471,6 +472,7 @@ STAGE PLANS:
                   mode: hash
                   outputColumnNames: _col0
                   Reduce Output Operator
+                    null sort order: 
                     sort order: 
                     tag: -1
                     value expressions: _col0 (type: bigint)

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/bucketcontext_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/bucketcontext_2.q.out b/ql/src/test/results/clientpositive/bucketcontext_2.q.out
index 9168ee1..bc08fd2 100644
--- a/ql/src/test/results/clientpositive/bucketcontext_2.q.out
+++ b/ql/src/test/results/clientpositive/bucketcontext_2.q.out
@@ -229,6 +229,7 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
+                    null sort order: 
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                     tag: -1
@@ -455,6 +456,7 @@ STAGE PLANS:
                   mode: hash
                   outputColumnNames: _col0
                   Reduce Output Operator
+                    null sort order: 
                     sort order: 
                     tag: -1
                     value expressions: _col0 (type: bigint)

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/bucketcontext_3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/bucketcontext_3.q.out b/ql/src/test/results/clientpositive/bucketcontext_3.q.out
index e9e3406..0cba2ca 100644
--- a/ql/src/test/results/clientpositive/bucketcontext_3.q.out
+++ b/ql/src/test/results/clientpositive/bucketcontext_3.q.out
@@ -272,6 +272,7 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
+                    null sort order: 
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                     tag: -1
@@ -452,6 +453,7 @@ STAGE PLANS:
                   mode: hash
                   outputColumnNames: _col0
                   Reduce Output Operator
+                    null sort order: 
                     sort order: 
                     tag: -1
                     value expressions: _col0 (type: bigint)


[18/18] hive git commit: HIVE-12994: Implement support for NULLS FIRST/NULLS LAST (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
HIVE-12994: Implement support for NULLS FIRST/NULLS LAST (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)


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

Branch: refs/heads/master
Commit: 9350b69341d509d4fc6d6e2f82bf3d6123ee11f3
Parents: 01fd684
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Wed Feb 24 14:19:41 2016 +0100
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Wed Mar 2 11:59:05 2016 +0100

----------------------------------------------------------------------
 data/files/over4_null                           |   5 +
 .../test/resources/testconfiguration.properties |   1 +
 metastore/if/hive_metastore.thrift              |   5 +-
 .../upgrade/derby/034-HIVE-12994.derby.sql      |   2 +
 .../upgrade/derby/hive-schema-2.1.0.derby.sql   |   2 +-
 .../derby/upgrade-2.0.0-to-2.1.0.derby.sql      |   1 +
 .../upgrade/mssql/019-HIVE-12994.mssql.sql      |   2 +
 .../upgrade/mssql/hive-schema-2.1.0.mssql.sql   |   1 +
 .../mssql/upgrade-2.0.0-to-2.1.0.mssql.sql      |   1 +
 .../upgrade/mysql/034-HIVE-12994.mysql.sql      |   2 +
 .../upgrade/mysql/hive-schema-2.1.0.mysql.sql   |   1 +
 .../mysql/upgrade-2.0.0-to-2.1.0.mysql.sql      |   1 +
 .../upgrade/oracle/034-HIVE-12994.oracle.sql    |   2 +
 .../upgrade/oracle/hive-schema-2.1.0.oracle.sql |   1 +
 .../oracle/upgrade-2.0.0-to-2.1.0.oracle.sql    |   1 +
 .../postgres/033-HIVE-12994.postgres.sql        |   2 +
 .../postgres/hive-schema-2.1.0.postgres.sql     |   1 +
 .../upgrade-2.0.0-to-2.1.0.postgres.sql         |   1 +
 .../apache/hadoop/hive/metastore/Metastore.java |  22 +-
 .../metastore/hbase/HbaseMetastoreProto.java    | 175 +++--
 .../gen/thrift/gen-cpp/hive_metastore_types.cpp |  20 +
 .../gen/thrift/gen-cpp/hive_metastore_types.h   |  10 +-
 .../apache/hadoop/hive/metastore/api/Order.java | 106 ++-
 .../src/gen/thrift/gen-php/metastore/Types.php  |  23 +
 .../gen/thrift/gen-py/hive_metastore/ttypes.py  |  15 +-
 .../gen/thrift/gen-rb/hive_metastore_types.rb   |   4 +-
 .../hive/metastore/MetaStoreDirectSql.java      |   6 +-
 .../hadoop/hive/metastore/ObjectStore.java      |   5 +-
 .../hadoop/hive/metastore/hbase/HBaseUtils.java |   2 +-
 .../hadoop/hive/metastore/model/MOrder.java     |  22 +-
 .../metastore/hbase/hbase_metastore_proto.proto |   1 +
 .../hive/metastore/hbase/TestHBaseStore.java    |  31 +-
 .../hbase/TestHBaseStoreBitVector.java          |  36 +-
 .../hbase/TestSharedStorageDescriptor.java      |  18 +-
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |   5 +
 .../hadoop/hive/ql/exec/MapJoinOperator.java    |   2 +
 .../apache/hadoop/hive/ql/exec/MoveTask.java    |   4 +-
 .../persistence/HybridHashTableContainer.java   |  22 +-
 .../persistence/MapJoinBytesTableContainer.java |  36 +-
 .../hive/ql/exec/persistence/MapJoinKey.java    |  12 +-
 .../ql/exec/tez/DynamicPartitionPruner.java     |   2 +-
 .../VectorReduceSinkCommonOperator.java         |  66 +-
 .../apache/hadoop/hive/ql/metadata/Hive.java    |   2 +-
 .../hive/ql/optimizer/AbstractSMBJoinProc.java  |   3 +-
 .../BucketingSortingReduceSinkOptimizer.java    |  85 ++-
 .../ql/optimizer/ReduceSinkMapJoinProc.java     |  15 +-
 .../optimizer/SortedDynPartitionOptimizer.java  |  97 ++-
 .../ql/optimizer/calcite/RelOptHiveTable.java   |  10 +-
 .../calcite/rules/HiveRelFieldTrimmer.java      |   4 +-
 .../calcite/translator/ASTConverter.java        |  49 +-
 .../calcite/translator/ExprNodeConverter.java   |  12 +
 .../calcite/translator/HiveOpConverter.java     |  39 +-
 .../correlation/ReduceSinkDeDuplication.java    |  20 +-
 .../optimizer/physical/BucketingSortingCtx.java |  15 +-
 .../physical/BucketingSortingOpProcFactory.java |  16 +-
 .../spark/SparkReduceSinkMapJoinProc.java       |  23 +-
 .../hive/ql/parse/BaseSemanticAnalyzer.java     |  22 +-
 .../hadoop/hive/ql/parse/CalcitePlanner.java    |  32 +-
 .../org/apache/hadoop/hive/ql/parse/HiveLexer.g |   2 +
 .../apache/hadoop/hive/ql/parse/HiveParser.g    |  48 +-
 .../hadoop/hive/ql/parse/IdentifiersParser.g    |   4 +-
 .../hadoop/hive/ql/parse/PTFInvocationSpec.java |  25 +-
 .../hadoop/hive/ql/parse/PTFTranslator.java     |   1 +
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |  89 ++-
 .../hive/ql/parse/TypeCheckProcFactory.java     |   2 +
 .../apache/hadoop/hive/ql/plan/PlanUtils.java   |  35 +-
 .../hadoop/hive/ql/plan/ReduceSinkDesc.java     |  23 +-
 .../hive/ql/plan/ptf/OrderExpressionDef.java    |  13 +-
 .../plan/ptf/PartitionedTableFunctionDef.java   |  12 +-
 .../hive/ql/udf/ptf/WindowingTableFunction.java |  69 +-
 .../hive/ql/txn/compactor/TestWorker.java       |   4 +-
 ql/src/test/queries/clientpositive/keyword_3.q  |   8 +
 ql/src/test/queries/clientpositive/order_null.q |  29 +
 .../clientpositive/windowing_order_null.q       |  35 +
 ...lter_numbuckets_partitioned_table2_h23.q.out |  20 +-
 ...alter_numbuckets_partitioned_table_h23.q.out |   4 +-
 .../alter_partition_clusterby_sortby.q.out      |   4 +-
 .../alter_partition_coltype.q.out               |   3 +
 .../clientpositive/alter_table_not_sorted.q.out |   2 +-
 .../clientpositive/authorization_index.q.out    |   2 +-
 .../auto_join_reordering_values.q.out           |   8 +
 .../clientpositive/auto_sortmerge_join_1.q.out  |   5 +
 .../clientpositive/auto_sortmerge_join_11.q.out |   4 +
 .../clientpositive/auto_sortmerge_join_12.q.out |   1 +
 .../clientpositive/auto_sortmerge_join_2.q.out  |   4 +
 .../clientpositive/auto_sortmerge_join_3.q.out  |   5 +
 .../clientpositive/auto_sortmerge_join_4.q.out  |   5 +
 .../clientpositive/auto_sortmerge_join_5.q.out  |   5 +
 .../clientpositive/auto_sortmerge_join_7.q.out  |   5 +
 .../clientpositive/auto_sortmerge_join_8.q.out  |   5 +
 .../test/results/clientpositive/bucket1.q.out   |   1 +
 .../test/results/clientpositive/bucket2.q.out   |   1 +
 .../test/results/clientpositive/bucket3.q.out   |   1 +
 .../test/results/clientpositive/bucket4.q.out   |   1 +
 .../test/results/clientpositive/bucket5.q.out   |   4 +-
 .../results/clientpositive/bucket_many.q.out    |   1 +
 .../clientpositive/bucket_map_join_1.q.out      |   1 +
 .../clientpositive/bucket_map_join_2.q.out      |   1 +
 .../clientpositive/bucketcontext_1.q.out        |   2 +
 .../clientpositive/bucketcontext_2.q.out        |   2 +
 .../clientpositive/bucketcontext_3.q.out        |   2 +
 .../clientpositive/bucketcontext_4.q.out        |   2 +
 .../clientpositive/bucketcontext_5.q.out        |   2 +
 .../clientpositive/bucketcontext_6.q.out        |   2 +
 .../clientpositive/bucketcontext_7.q.out        |   2 +
 .../clientpositive/bucketcontext_8.q.out        |   2 +
 .../clientpositive/bucketmapjoin10.q.out        |   1 +
 .../clientpositive/bucketmapjoin11.q.out        |   2 +
 .../clientpositive/bucketmapjoin12.q.out        |   2 +
 .../clientpositive/bucketmapjoin13.q.out        |   4 +
 .../results/clientpositive/bucketmapjoin7.q.out |  19 +-
 .../results/clientpositive/bucketmapjoin8.q.out |   2 +
 .../results/clientpositive/bucketmapjoin9.q.out |   2 +
 .../cbo_rp_outer_join_ppr.q.java1.7.out         |   4 +
 .../clientpositive/columnstats_partlvl.q.out    |   2 +
 .../clientpositive/columnstats_tbllvl.q.out     |   2 +
 .../test/results/clientpositive/combine2.q.out  |   1 +
 .../constantPropagateForSubQuery.q.out          |   2 +
 .../clientpositive/correlationoptimizer12.q.out |   4 +-
 ql/src/test/results/clientpositive/ctas.q.out   |  12 +-
 .../results/clientpositive/ctas_colname.q.out   |   4 +-
 .../disable_merge_for_bucketing.q.out           |   1 +
 .../display_colstats_tbllvl.q.out               |   1 +
 .../dynpart_sort_opt_vectorization.q.out        |  12 +-
 .../dynpart_sort_optimization.q.out             |  12 +-
 .../encryption_join_unencrypted_tbl.q.out       |   2 +
 ...on_join_with_different_encryption_keys.q.out |   2 +
 .../clientpositive/explain_logical.q.out        |   9 +-
 .../clientpositive/filter_join_breaktask.q.out  |   4 +
 .../clientpositive/fouter_join_ppr.q.out        |   8 +
 .../groupby_grouping_window.q.out               |   2 +-
 .../clientpositive/groupby_map_ppr.q.out        |   1 +
 .../groupby_map_ppr_multi_distinct.q.out        |   1 +
 .../results/clientpositive/groupby_ppr.q.out    |   1 +
 .../groupby_ppr_multi_distinct.q.out            |   1 +
 .../clientpositive/groupby_resolution.q.out     |   2 +-
 .../clientpositive/groupby_sort_1_23.q.out      |  11 +
 .../results/clientpositive/groupby_sort_6.q.out |   3 +
 .../clientpositive/groupby_sort_skew_1_23.q.out |  18 +
 .../clientpositive/index_skewtable.q.out        |   2 +-
 .../clientpositive/infer_bucket_sort.q.out      |  42 +-
 .../infer_bucket_sort_bucketed_table.q.out      |   2 +-
 .../infer_bucket_sort_convert_join.q.out        |   2 +-
 .../infer_bucket_sort_dyn_part.q.out            |   4 +-
 .../infer_bucket_sort_grouping_operators.q.out  |  12 +-
 .../infer_bucket_sort_map_operators.q.out       |   4 +-
 .../infer_bucket_sort_merge.q.out               |   2 +-
 .../infer_bucket_sort_multi_insert.q.out        |  10 +-
 .../infer_bucket_sort_reducers_power_two.q.out  |  12 +-
 .../test/results/clientpositive/input23.q.out   |   2 +
 .../results/clientpositive/input_part7.q.out    |  38 +-
 ql/src/test/results/clientpositive/join17.q.out |   2 +
 ql/src/test/results/clientpositive/join35.q.out |   2 +
 ql/src/test/results/clientpositive/join9.q.out  |   2 +
 .../clientpositive/join_filters_overlap.q.out   |  17 +
 .../test/results/clientpositive/keyword_3.q.out |  32 +
 .../test/results/clientpositive/lineage2.q.out  |   2 +-
 .../test/results/clientpositive/lineage3.q.out  |  10 +-
 .../list_bucket_query_multiskew_3.q.out         |   1 +
 .../list_bucket_query_oneskew_2.q.out           |   2 +
 .../llap/tez_join_result_complex.q.out          |   2 +
 .../clientpositive/louter_join_ppr.q.out        |   8 +
 ql/src/test/results/clientpositive/merge3.q.out |   1 +
 .../results/clientpositive/metadataonly1.q.out  |  12 +
 .../clientpositive/optimize_nullscan.q.out      |  14 +
 .../results/clientpositive/orc_analyze.q.out    |  16 +-
 .../results/clientpositive/order_null.q.out     | 222 ++++++
 .../clientpositive/outer_join_ppr.q.java1.7.out |   4 +
 .../partition_coltype_literals.q.out            |   4 +-
 ql/src/test/results/clientpositive/pcr.q.out    | 204 +++--
 ql/src/test/results/clientpositive/pcs.q.out    |  49 +-
 .../results/clientpositive/perf/query12.q.out   |   2 +-
 .../results/clientpositive/perf/query20.q.out   |   2 +-
 .../results/clientpositive/perf/query51.q.out   |   6 +-
 .../results/clientpositive/perf/query67.q.out   |   2 +-
 .../results/clientpositive/perf/query70.q.out   |   4 +-
 .../results/clientpositive/perf/query89.q.out   |   2 +-
 .../results/clientpositive/perf/query98.q.out   |   2 +-
 .../results/clientpositive/pointlookup2.q.out   | 100 ++-
 .../results/clientpositive/pointlookup3.q.out   |  96 ++-
 .../results/clientpositive/pointlookup4.q.out   |  42 +-
 .../clientpositive/ppd_join_filter.q.out        |  12 +
 .../results/clientpositive/ppd_union_view.q.out |   4 +
 ql/src/test/results/clientpositive/ppd_vc.q.out |  18 +-
 .../results/clientpositive/ppd_windowing1.q.out |  66 +-
 ql/src/test/results/clientpositive/ptf.q.out    | 144 ++--
 .../results/clientpositive/ptf_matchpath.q.out  |   6 +-
 .../results/clientpositive/ptf_streaming.q.out  |  96 +--
 .../results/clientpositive/ptfgroupbyjoin.q.out |   2 +-
 .../test/results/clientpositive/push_or.q.out   |  11 +-
 .../results/clientpositive/quotedid_basic.q.out |   4 +-
 .../clientpositive/reduce_deduplicate.q.out     |   2 +
 .../results/clientpositive/regexp_extract.q.out |   2 +
 .../clientpositive/router_join_ppr.q.out        |   8 +
 .../test/results/clientpositive/sample10.q.out  |   7 +-
 .../test/results/clientpositive/sample6.q.out   |  77 +-
 .../test/results/clientpositive/sample8.q.out   |   2 +
 .../test/results/clientpositive/semijoin2.q.out |   4 +-
 .../test/results/clientpositive/semijoin4.q.out |   2 +-
 .../show_create_table_alter.q.out               |  10 +-
 .../results/clientpositive/smb_mapjoin_11.q.out |   1 +
 .../results/clientpositive/smb_mapjoin_13.q.out |  20 +-
 .../results/clientpositive/smb_mapjoin_15.q.out |  40 +-
 .../clientpositive/sort_merge_join_desc_5.q.out |   1 +
 .../clientpositive/sort_merge_join_desc_6.q.out |   1 +
 .../clientpositive/sort_merge_join_desc_7.q.out |   1 +
 .../spark/auto_join_reordering_values.q.out     |   8 +
 .../spark/auto_sortmerge_join_1.q.out           |   3 +
 .../spark/auto_sortmerge_join_12.q.out          |   1 +
 .../spark/auto_sortmerge_join_2.q.out           |   2 +
 .../spark/auto_sortmerge_join_3.q.out           |   3 +
 .../spark/auto_sortmerge_join_4.q.out           |   3 +
 .../spark/auto_sortmerge_join_5.q.out           |   3 +
 .../spark/auto_sortmerge_join_7.q.out           |   3 +
 .../spark/auto_sortmerge_join_8.q.out           |   3 +
 .../results/clientpositive/spark/bucket2.q.out  |   1 +
 .../results/clientpositive/spark/bucket3.q.out  |   1 +
 .../results/clientpositive/spark/bucket4.q.out  |   1 +
 .../results/clientpositive/spark/bucket5.q.out  |   4 +-
 .../spark/bucket_map_join_1.q.out               |   1 +
 .../spark/bucket_map_join_2.q.out               |   1 +
 .../clientpositive/spark/bucketmapjoin10.q.out  |   1 +
 .../clientpositive/spark/bucketmapjoin11.q.out  |   2 +
 .../clientpositive/spark/bucketmapjoin12.q.out  |   2 +
 .../clientpositive/spark/bucketmapjoin13.q.out  |   4 +
 .../clientpositive/spark/bucketmapjoin7.q.out   |  19 +-
 .../clientpositive/spark/bucketmapjoin8.q.out   |   2 +
 .../clientpositive/spark/bucketmapjoin9.q.out   |   2 +
 .../results/clientpositive/spark/ctas.q.out     |  12 +-
 .../spark/disable_merge_for_bucketing.q.out     |   1 +
 .../spark/filter_join_breaktask.q.out           |   4 +
 .../clientpositive/spark/groupby_map_ppr.q.out  |   1 +
 .../spark/groupby_map_ppr_multi_distinct.q.out  |   1 +
 .../clientpositive/spark/groupby_ppr.q.out      |   1 +
 .../spark/groupby_ppr_multi_distinct.q.out      |   1 +
 .../spark/groupby_resolution.q.out              |   2 +-
 .../spark/groupby_sort_1_23.q.out               |  11 +
 .../spark/groupby_sort_skew_1_23.q.out          |  18 +
 .../infer_bucket_sort_bucketed_table.q.out      |   2 +-
 .../results/clientpositive/spark/join17.q.out   |   2 +
 .../results/clientpositive/spark/join34.q.out   |   3 +
 .../results/clientpositive/spark/join35.q.out   |   5 +
 .../results/clientpositive/spark/join9.q.out    |   2 +
 .../spark/join_filters_overlap.q.out            |  17 +
 .../clientpositive/spark/louter_join_ppr.q.out  |   8 +
 .../spark/optimize_nullscan.q.out               |  14 +
 .../spark/outer_join_ppr.q.java1.7.out          |   4 +
 .../test/results/clientpositive/spark/pcr.q.out | 204 +++--
 .../clientpositive/spark/ppd_join_filter.q.out  |  12 +
 .../test/results/clientpositive/spark/ptf.q.out | 146 ++--
 .../clientpositive/spark/ptf_matchpath.q.out    |   6 +-
 .../clientpositive/spark/ptf_streaming.q.out    |  96 +--
 .../spark/reduce_deduplicate.q.out              |   2 +
 .../clientpositive/spark/router_join_ppr.q.out  |   8 +
 .../results/clientpositive/spark/sample10.q.out |   7 +-
 .../results/clientpositive/spark/sample6.q.out  |  77 +-
 .../results/clientpositive/spark/sample8.q.out  |   2 +
 .../clientpositive/spark/smb_mapjoin_11.q.out   |   1 +
 .../clientpositive/spark/smb_mapjoin_12.q.out   |   2 +
 .../clientpositive/spark/smb_mapjoin_13.q.out   |  20 +-
 .../clientpositive/spark/smb_mapjoin_15.q.out   |  40 +-
 .../clientpositive/spark/subquery_in.q.out      |   4 +-
 .../clientpositive/spark/transform_ppr1.q.out   |   1 +
 .../clientpositive/spark/transform_ppr2.q.out   |   1 +
 .../results/clientpositive/spark/union24.q.out  |   6 +
 .../clientpositive/spark/union_ppr.q.out        |  38 +-
 .../spark/union_remove_6_subq.q.out             |   2 +-
 .../clientpositive/spark/vectorized_ptf.q.out   | 747 +++++++++++--------
 .../results/clientpositive/subquery_in.q.out    |   4 +-
 .../clientpositive/subquery_in_having.q.out     |   2 +-
 .../results/clientpositive/subquery_notin.q.out |  12 +-
 .../subquery_unqualcolumnrefs.q.out             |   8 +-
 .../temp_table_display_colstats_tbllvl.q.out    |   1 +
 .../tez/auto_sortmerge_join_1.q.out             |   6 +
 .../tez/auto_sortmerge_join_11.q.out            |   9 +
 .../tez/auto_sortmerge_join_12.q.out            |   4 +
 .../tez/auto_sortmerge_join_2.q.out             |   4 +
 .../tez/auto_sortmerge_join_3.q.out             |   6 +
 .../tez/auto_sortmerge_join_4.q.out             |   6 +
 .../tez/auto_sortmerge_join_5.q.out             |   4 +
 .../tez/auto_sortmerge_join_7.q.out             |   6 +
 .../tez/auto_sortmerge_join_8.q.out             |   6 +
 .../results/clientpositive/tez/bucket2.q.out    |   1 +
 .../results/clientpositive/tez/bucket3.q.out    |   1 +
 .../results/clientpositive/tez/bucket4.q.out    |   1 +
 .../test/results/clientpositive/tez/ctas.q.out  |  12 +-
 .../tez/disable_merge_for_bucketing.q.out       |   1 +
 .../tez/dynpart_sort_opt_vectorization.q.out    |  12 +-
 .../tez/dynpart_sort_optimization.q.out         |  12 +-
 .../clientpositive/tez/explainuser_1.q.out      |  94 +--
 .../tez/filter_join_breaktask.q.out             |   4 +
 .../clientpositive/tez/mapjoin_mapjoin.q.out    |   2 +
 .../clientpositive/tez/metadataonly1.q.out      |  12 +
 .../clientpositive/tez/optimize_nullscan.q.out  |  14 +
 .../clientpositive/tez/orc_analyze.q.out        |  16 +-
 .../results/clientpositive/tez/order_null.q.out | 222 ++++++
 .../test/results/clientpositive/tez/ptf.q.out   | 144 ++--
 .../clientpositive/tez/ptf_matchpath.q.out      |   6 +-
 .../clientpositive/tez/ptf_streaming.q.out      |  96 +--
 .../clientpositive/tez/subquery_in.q.out        |   4 +-
 .../tez/tez_join_result_complex.q.out           |   2 +
 .../clientpositive/tez/transform_ppr1.q.out     |   1 +
 .../clientpositive/tez/transform_ppr2.q.out     |   1 +
 .../clientpositive/tez/unionDistinct_1.q.out    |  21 +
 .../clientpositive/tez/vectorized_ptf.q.out     | 745 +++++++++++-------
 .../clientpositive/tez/windowing_gby.q.out      |   2 +-
 .../results/clientpositive/transform_ppr1.q.out |   1 +
 .../results/clientpositive/transform_ppr2.q.out |   1 +
 .../results/clientpositive/udf_explode.q.out    |   2 +
 .../results/clientpositive/udtf_explode.q.out   |   4 +
 .../test/results/clientpositive/union22.q.out   |   2 +
 .../test/results/clientpositive/union24.q.out   |   6 +
 .../clientpositive/unionDistinct_1.q.out        |  20 +
 .../test/results/clientpositive/union_ppr.q.out |  38 +-
 .../clientpositive/union_remove_6_subq.q.out    |   2 +-
 .../results/clientpositive/vectorized_ptf.q.out | 745 +++++++++++-------
 .../clientpositive/windowing_order_null.q.out   | 183 +++++
 .../clientpositive/windowing_streaming.q.out    |   6 +-
 serde/if/serde.thrift                           |   1 +
 .../src/gen/thrift/gen-cpp/serde_constants.cpp  |   2 +
 serde/src/gen/thrift/gen-cpp/serde_constants.h  |   1 +
 .../hadoop/hive/serde/serdeConstants.java       |   2 +
 .../org/apache/hadoop/hive/serde/Types.php      |   5 +
 .../org_apache_hadoop_hive_serde/constants.py   |   1 +
 serde/src/gen/thrift/gen-rb/serde_constants.rb  |   2 +
 .../binarysortable/BinarySortableSerDe.java     | 110 ++-
 .../BinarySortableSerDeWithEndPrefix.java       |   2 +-
 .../fast/BinarySortableSerializeWrite.java      | 130 +++-
 .../binarysortable/TestBinarySortableFast.java  |  30 +-
 .../binarysortable/TestBinarySortableSerDe.java |  18 +-
 330 files changed, 5399 insertions(+), 2340 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/data/files/over4_null
----------------------------------------------------------------------
diff --git a/data/files/over4_null b/data/files/over4_null
new file mode 100644
index 0000000..7f79531
--- /dev/null
+++ b/data/files/over4_null
@@ -0,0 +1,5 @@
+124|336|65534|\N|\N|0.01|true|\N|2013-03-01 09:11:58.703302|45.40|^Ayard duty^B
+19|442|65534|\N|\N|\N|true|calvin miller|2013-03-01 09:11:58.703217|29.62|^Ahistory^B
+35|387|65536|\N|\N|0.02|false|\N|2013-03-01 09:11:58.70307|27.32|^Ahistory^B
+111|372|\N|\N|\N|0.01|false|\N|2013-03-01 09:11:58.703310|23.91|^Atopology^B
+54|317|\N|\N|\N|\N|false|alice ichabod|2013-03-01 09:11:58.703103|90.21|^Ageology^B

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------
diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index 68dbd0c..f991d49 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -160,6 +160,7 @@ minitez.query.files.shared=acid_globallimit.q,\
   metadata_only_queries.q,\
   metadata_only_queries_with_filters.q,\
   nonmr_fetch_threshold.q,\
+  order_null.q,\
   optimize_nullscan.q,\
   orc_analyze.q,\
   orc_merge1.q,\

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/metastore/if/hive_metastore.thrift
----------------------------------------------------------------------
diff --git a/metastore/if/hive_metastore.thrift b/metastore/if/hive_metastore.thrift
index e8f0a68..a4fb612 100755
--- a/metastore/if/hive_metastore.thrift
+++ b/metastore/if/hive_metastore.thrift
@@ -226,8 +226,9 @@ struct SerDeInfo {
 
 // sort order of a column (column name along with asc(1)/desc(0))
 struct Order {
-  1: string col,  // sort column name
-  2: i32    order // asc(1) or desc(0)
+  1: string col,      // sort column name
+  2: i32    order,    // asc(1) or desc(0)
+  3: i32    nullOrder // nulls first(0) or nulls last(1)
 }
 
 // this object holds all the information about skewed table

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/metastore/scripts/upgrade/derby/034-HIVE-12994.derby.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/derby/034-HIVE-12994.derby.sql b/metastore/scripts/upgrade/derby/034-HIVE-12994.derby.sql
new file mode 100644
index 0000000..a8b48bf
--- /dev/null
+++ b/metastore/scripts/upgrade/derby/034-HIVE-12994.derby.sql
@@ -0,0 +1,2 @@
+ALTER TABLE "APP".SORT_COLS ADD "NULL_ORDER" INTEGER NOT NULL DEFAULT 0;
+UPDATE "APP".SORT_COLS SET "NULL_ORDER" = 1 WHERE "ORDER" = 0;

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/metastore/scripts/upgrade/derby/hive-schema-2.1.0.derby.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/derby/hive-schema-2.1.0.derby.sql b/metastore/scripts/upgrade/derby/hive-schema-2.1.0.derby.sql
index 42f4eb6..00c49ae 100644
--- a/metastore/scripts/upgrade/derby/hive-schema-2.1.0.derby.sql
+++ b/metastore/scripts/upgrade/derby/hive-schema-2.1.0.derby.sql
@@ -28,7 +28,7 @@ CREATE TABLE "APP"."SERDE_PARAMS" ("SERDE_ID" BIGINT NOT NULL, "PARAM_KEY" VARCH
 
 CREATE TABLE "APP"."COLUMNS_V2" ("CD_ID" BIGINT NOT NULL, "COMMENT" VARCHAR(4000), "COLUMN_NAME" VARCHAR(1000) NOT NULL, "TYPE_NAME" VARCHAR(4000), "INTEGER_IDX" INTEGER NOT NULL);
 
-CREATE TABLE "APP"."SORT_COLS" ("SD_ID" BIGINT NOT NULL, "COLUMN_NAME" VARCHAR(1000), "ORDER" INTEGER NOT NULL, "INTEGER_IDX" INTEGER NOT NULL);
+CREATE TABLE "APP"."SORT_COLS" ("SD_ID" BIGINT NOT NULL, "COLUMN_NAME" VARCHAR(1000), "ORDER" INTEGER NOT NULL, "NULL_ORDER" INTEGER NOT NULL, "INTEGER_IDX" INTEGER NOT NULL);
 
 CREATE TABLE "APP"."CDS" ("CD_ID" BIGINT NOT NULL);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/metastore/scripts/upgrade/derby/upgrade-2.0.0-to-2.1.0.derby.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/derby/upgrade-2.0.0-to-2.1.0.derby.sql b/metastore/scripts/upgrade/derby/upgrade-2.0.0-to-2.1.0.derby.sql
index a0bac3c..13aeabb 100644
--- a/metastore/scripts/upgrade/derby/upgrade-2.0.0-to-2.1.0.derby.sql
+++ b/metastore/scripts/upgrade/derby/upgrade-2.0.0-to-2.1.0.derby.sql
@@ -1,4 +1,5 @@
 -- Upgrade MetaStore schema from 2.0.0 to 2.1.0
 RUN '033-HIVE-12892.derby.sql';
+RUN '034-HIVE-12994.derby.sql';
 
 UPDATE "APP".VERSION SET SCHEMA_VERSION='2.1.0', VERSION_COMMENT='Hive release version 2.1.0' where VER_ID=1;

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/metastore/scripts/upgrade/mssql/019-HIVE-12994.mssql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mssql/019-HIVE-12994.mssql.sql b/metastore/scripts/upgrade/mssql/019-HIVE-12994.mssql.sql
new file mode 100644
index 0000000..51e2c43
--- /dev/null
+++ b/metastore/scripts/upgrade/mssql/019-HIVE-12994.mssql.sql
@@ -0,0 +1,2 @@
+ALTER TABLE SORT_COLS ADD "NULL_ORDER" int NOT NULL DEFAULT 0;
+UPDATE SORT_COLS SET "NULL_ORDER" = 1 WHERE "ORDER" = 0;

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/metastore/scripts/upgrade/mssql/hive-schema-2.1.0.mssql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mssql/hive-schema-2.1.0.mssql.sql b/metastore/scripts/upgrade/mssql/hive-schema-2.1.0.mssql.sql
index cf5a662..c98c7ef 100644
--- a/metastore/scripts/upgrade/mssql/hive-schema-2.1.0.mssql.sql
+++ b/metastore/scripts/upgrade/mssql/hive-schema-2.1.0.mssql.sql
@@ -411,6 +411,7 @@ CREATE TABLE SORT_COLS
     SD_ID bigint NOT NULL,
     "COLUMN_NAME" nvarchar(1000) NULL,
     "ORDER" int NOT NULL,
+    "NULL_ORDER" int NOT NULL,
     INTEGER_IDX int NOT NULL
 );
 

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/metastore/scripts/upgrade/mssql/upgrade-2.0.0-to-2.1.0.mssql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mssql/upgrade-2.0.0-to-2.1.0.mssql.sql b/metastore/scripts/upgrade/mssql/upgrade-2.0.0-to-2.1.0.mssql.sql
index f25daf2..3b4a2a3 100644
--- a/metastore/scripts/upgrade/mssql/upgrade-2.0.0-to-2.1.0.mssql.sql
+++ b/metastore/scripts/upgrade/mssql/upgrade-2.0.0-to-2.1.0.mssql.sql
@@ -1,6 +1,7 @@
 SELECT 'Upgrading MetaStore schema from 2.0.0 to 2.1.0' AS MESSAGE;
 
 :r 018-HIVE-12892.mssql.sql;
+:r 019-HIVE-12994.mssql.sql;
 
 UPDATE VERSION SET SCHEMA_VERSION='2.1.0', VERSION_COMMENT='Hive release version 2.1.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 2.0.0 to 2.1.0' AS MESSAGE;

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/metastore/scripts/upgrade/mysql/034-HIVE-12994.mysql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mysql/034-HIVE-12994.mysql.sql b/metastore/scripts/upgrade/mysql/034-HIVE-12994.mysql.sql
new file mode 100644
index 0000000..ff4cf37
--- /dev/null
+++ b/metastore/scripts/upgrade/mysql/034-HIVE-12994.mysql.sql
@@ -0,0 +1,2 @@
+ALTER TABLE `SORT_COLS` ADD `NULL_ORDER` INTEGER NOT NULL DEFAULT 0;
+UPDATE `SORT_COLS` SET `NULL_ORDER` = 1 WHERE `ORDER` = 0;

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/metastore/scripts/upgrade/mysql/hive-schema-2.1.0.mysql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mysql/hive-schema-2.1.0.mysql.sql b/metastore/scripts/upgrade/mysql/hive-schema-2.1.0.mysql.sql
index 6fd3209..1369349 100644
--- a/metastore/scripts/upgrade/mysql/hive-schema-2.1.0.mysql.sql
+++ b/metastore/scripts/upgrade/mysql/hive-schema-2.1.0.mysql.sql
@@ -546,6 +546,7 @@ CREATE TABLE IF NOT EXISTS `SORT_COLS` (
   `SD_ID` bigint(20) NOT NULL,
   `COLUMN_NAME` varchar(1000) CHARACTER SET latin1 COLLATE latin1_bin DEFAULT NULL,
   `ORDER` int(11) NOT NULL,
+  `NULL_ORDER` int(11) NOT NULL,
   `INTEGER_IDX` int(11) NOT NULL,
   PRIMARY KEY (`SD_ID`,`INTEGER_IDX`),
   KEY `SORT_COLS_N49` (`SD_ID`),

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/metastore/scripts/upgrade/mysql/upgrade-2.0.0-to-2.1.0.mysql.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/mysql/upgrade-2.0.0-to-2.1.0.mysql.sql b/metastore/scripts/upgrade/mysql/upgrade-2.0.0-to-2.1.0.mysql.sql
index e790636..09bb8b8 100644
--- a/metastore/scripts/upgrade/mysql/upgrade-2.0.0-to-2.1.0.mysql.sql
+++ b/metastore/scripts/upgrade/mysql/upgrade-2.0.0-to-2.1.0.mysql.sql
@@ -1,6 +1,7 @@
 SELECT 'Upgrading MetaStore schema from 2.0.0 to 2.1.0' AS ' ';
 
 SOURCE 033-HIVE-12892.mysql.sql;
+SOURCE 034-HIVE-12994.mysql.sql;
 
 UPDATE VERSION SET SCHEMA_VERSION='2.1.0', VERSION_COMMENT='Hive release version 2.1.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 2.0.0 to 2.1.0' AS ' ';

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/metastore/scripts/upgrade/oracle/034-HIVE-12994.oracle.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/oracle/034-HIVE-12994.oracle.sql b/metastore/scripts/upgrade/oracle/034-HIVE-12994.oracle.sql
new file mode 100644
index 0000000..0b57ef8
--- /dev/null
+++ b/metastore/scripts/upgrade/oracle/034-HIVE-12994.oracle.sql
@@ -0,0 +1,2 @@
+ALTER TABLE SORT_COLS ADD "NULL_ORDER" NUMBER (10) NOT NULL DEFAULT 0;
+UPDATE SORT_COLS SET "NULL_ORDER" = 1 WHERE "ORDER" = 0;

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/metastore/scripts/upgrade/oracle/hive-schema-2.1.0.oracle.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/oracle/hive-schema-2.1.0.oracle.sql b/metastore/scripts/upgrade/oracle/hive-schema-2.1.0.oracle.sql
index 774f6be..198f0bd 100644
--- a/metastore/scripts/upgrade/oracle/hive-schema-2.1.0.oracle.sql
+++ b/metastore/scripts/upgrade/oracle/hive-schema-2.1.0.oracle.sql
@@ -274,6 +274,7 @@ CREATE TABLE SORT_COLS
     SD_ID NUMBER NOT NULL,
     "COLUMN_NAME" VARCHAR2(1000) NULL,
     "ORDER" NUMBER (10) NOT NULL,
+    "NULL_ORDER" NUMBER (10) NOT NULL,
     INTEGER_IDX NUMBER(10) NOT NULL
 );
 

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/metastore/scripts/upgrade/oracle/upgrade-2.0.0-to-2.1.0.oracle.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/oracle/upgrade-2.0.0-to-2.1.0.oracle.sql b/metastore/scripts/upgrade/oracle/upgrade-2.0.0-to-2.1.0.oracle.sql
index 8368d08..a729baa 100644
--- a/metastore/scripts/upgrade/oracle/upgrade-2.0.0-to-2.1.0.oracle.sql
+++ b/metastore/scripts/upgrade/oracle/upgrade-2.0.0-to-2.1.0.oracle.sql
@@ -1,6 +1,7 @@
 SELECT 'Upgrading MetaStore schema from 2.0.0 to 2.1.0' AS Status from dual;
 
 @033-HIVE-12892.oracle.sql;
+@034-HIVE-12994.oracle.sql;
 
 UPDATE VERSION SET SCHEMA_VERSION='2.1.0', VERSION_COMMENT='Hive release version 2.1.0' where VER_ID=1;
 SELECT 'Finished upgrading MetaStore schema from 2.0.0 to 2.1.0' AS Status from dual;

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/metastore/scripts/upgrade/postgres/033-HIVE-12994.postgres.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/postgres/033-HIVE-12994.postgres.sql b/metastore/scripts/upgrade/postgres/033-HIVE-12994.postgres.sql
new file mode 100644
index 0000000..0894fed
--- /dev/null
+++ b/metastore/scripts/upgrade/postgres/033-HIVE-12994.postgres.sql
@@ -0,0 +1,2 @@
+ALTER TABLE SORT_COLS ADD COLUMN "NULL_ORDER" bigint NOT NULL DEFAULT 0;
+UPDATE SORT_COLS SET "NULL_ORDER" = 1 WHERE "ORDER" = 0;

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/metastore/scripts/upgrade/postgres/hive-schema-2.1.0.postgres.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/postgres/hive-schema-2.1.0.postgres.sql b/metastore/scripts/upgrade/postgres/hive-schema-2.1.0.postgres.sql
index 7463a37..48d16de 100644
--- a/metastore/scripts/upgrade/postgres/hive-schema-2.1.0.postgres.sql
+++ b/metastore/scripts/upgrade/postgres/hive-schema-2.1.0.postgres.sql
@@ -342,6 +342,7 @@ CREATE TABLE "SORT_COLS" (
     "SD_ID" bigint NOT NULL,
     "COLUMN_NAME" character varying(1000) DEFAULT NULL::character varying,
     "ORDER" bigint NOT NULL,
+    "NULL_ORDER" bigint NOT NULL,
     "INTEGER_IDX" bigint NOT NULL
 );
 

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/metastore/scripts/upgrade/postgres/upgrade-2.0.0-to-2.1.0.postgres.sql
----------------------------------------------------------------------
diff --git a/metastore/scripts/upgrade/postgres/upgrade-2.0.0-to-2.1.0.postgres.sql b/metastore/scripts/upgrade/postgres/upgrade-2.0.0-to-2.1.0.postgres.sql
index 6172407..30d070d 100644
--- a/metastore/scripts/upgrade/postgres/upgrade-2.0.0-to-2.1.0.postgres.sql
+++ b/metastore/scripts/upgrade/postgres/upgrade-2.0.0-to-2.1.0.postgres.sql
@@ -1,6 +1,7 @@
 SELECT 'Upgrading MetaStore schema from 2.0.0 to 2.1.0';
 
 \i 032-HIVE-12892.postgres.sql;
+\i 033-HIVE-12994.postgres.sql;
 
 UPDATE "VERSION" SET "SCHEMA_VERSION"='2.1.0', "VERSION_COMMENT"='Hive release version 2.1.0' where "VER_ID"=1;
 SELECT 'Finished upgrading MetaStore schema from 2.0.0 to 2.1.0';

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/metastore/src/gen/protobuf/gen-java/org/apache/hadoop/hive/metastore/Metastore.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/protobuf/gen-java/org/apache/hadoop/hive/metastore/Metastore.java b/metastore/src/gen/protobuf/gen-java/org/apache/hadoop/hive/metastore/Metastore.java
index 416ae9d..ca928b9 100644
--- a/metastore/src/gen/protobuf/gen-java/org/apache/hadoop/hive/metastore/Metastore.java
+++ b/metastore/src/gen/protobuf/gen-java/org/apache/hadoop/hive/metastore/Metastore.java
@@ -447,15 +447,15 @@ public final class Metastore {
 
       public final boolean isInitialized() {
         if (!hasOffset()) {
-
+          
           return false;
         }
         if (!hasLength()) {
-
+          
           return false;
         }
         if (!hasIndex()) {
-
+          
           return false;
         }
         return true;
@@ -597,7 +597,7 @@ public final class Metastore {
     /**
      * <code>repeated .org.apache.hadoop.hive.metastore.SplitInfo infos = 1;</code>
      */
-    java.util.List<org.apache.hadoop.hive.metastore.Metastore.SplitInfo>
+    java.util.List<org.apache.hadoop.hive.metastore.Metastore.SplitInfo> 
         getInfosList();
     /**
      * <code>repeated .org.apache.hadoop.hive.metastore.SplitInfo infos = 1;</code>
@@ -610,7 +610,7 @@ public final class Metastore {
     /**
      * <code>repeated .org.apache.hadoop.hive.metastore.SplitInfo infos = 1;</code>
      */
-    java.util.List<? extends org.apache.hadoop.hive.metastore.Metastore.SplitInfoOrBuilder>
+    java.util.List<? extends org.apache.hadoop.hive.metastore.Metastore.SplitInfoOrBuilder> 
         getInfosOrBuilderList();
     /**
      * <code>repeated .org.apache.hadoop.hive.metastore.SplitInfo infos = 1;</code>
@@ -731,7 +731,7 @@ public final class Metastore {
     /**
      * <code>repeated .org.apache.hadoop.hive.metastore.SplitInfo infos = 1;</code>
      */
-    public java.util.List<? extends org.apache.hadoop.hive.metastore.Metastore.SplitInfoOrBuilder>
+    public java.util.List<? extends org.apache.hadoop.hive.metastore.Metastore.SplitInfoOrBuilder> 
         getInfosOrBuilderList() {
       return infos_;
     }
@@ -984,7 +984,7 @@ public final class Metastore {
               infosBuilder_ = null;
               infos_ = other.infos_;
               bitField0_ = (bitField0_ & ~0x00000001);
-              infosBuilder_ =
+              infosBuilder_ = 
                 com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
                    getInfosFieldBuilder() : null;
             } else {
@@ -999,7 +999,7 @@ public final class Metastore {
       public final boolean isInitialized() {
         for (int i = 0; i < getInfosCount(); i++) {
           if (!getInfos(i).isInitialized()) {
-
+            
             return false;
           }
         }
@@ -1220,7 +1220,7 @@ public final class Metastore {
       /**
        * <code>repeated .org.apache.hadoop.hive.metastore.SplitInfo infos = 1;</code>
        */
-      public java.util.List<? extends org.apache.hadoop.hive.metastore.Metastore.SplitInfoOrBuilder>
+      public java.util.List<? extends org.apache.hadoop.hive.metastore.Metastore.SplitInfoOrBuilder> 
            getInfosOrBuilderList() {
         if (infosBuilder_ != null) {
           return infosBuilder_.getMessageOrBuilderList();
@@ -1246,12 +1246,12 @@ public final class Metastore {
       /**
        * <code>repeated .org.apache.hadoop.hive.metastore.SplitInfo infos = 1;</code>
        */
-      public java.util.List<org.apache.hadoop.hive.metastore.Metastore.SplitInfo.Builder>
+      public java.util.List<org.apache.hadoop.hive.metastore.Metastore.SplitInfo.Builder> 
            getInfosBuilderList() {
         return getInfosFieldBuilder().getBuilderList();
       }
       private com.google.protobuf.RepeatedFieldBuilder<
-          org.apache.hadoop.hive.metastore.Metastore.SplitInfo, org.apache.hadoop.hive.metastore.Metastore.SplitInfo.Builder, org.apache.hadoop.hive.metastore.Metastore.SplitInfoOrBuilder>
+          org.apache.hadoop.hive.metastore.Metastore.SplitInfo, org.apache.hadoop.hive.metastore.Metastore.SplitInfo.Builder, org.apache.hadoop.hive.metastore.Metastore.SplitInfoOrBuilder> 
           getInfosFieldBuilder() {
         if (infosBuilder_ == null) {
           infosBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/metastore/src/gen/protobuf/gen-java/org/apache/hadoop/hive/metastore/hbase/HbaseMetastoreProto.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/protobuf/gen-java/org/apache/hadoop/hive/metastore/hbase/HbaseMetastoreProto.java b/metastore/src/gen/protobuf/gen-java/org/apache/hadoop/hive/metastore/hbase/HbaseMetastoreProto.java
index 3b2d7b5..3057fff 100644
--- a/metastore/src/gen/protobuf/gen-java/org/apache/hadoop/hive/metastore/hbase/HbaseMetastoreProto.java
+++ b/metastore/src/gen/protobuf/gen-java/org/apache/hadoop/hive/metastore/hbase/HbaseMetastoreProto.java
@@ -22582,6 +22582,16 @@ public final class HbaseMetastoreProto {
        * <code>optional sint32 order = 2 [default = 1];</code>
        */
       int getOrder();
+
+      // optional sint32 nullOrder = 3 [default = 0];
+      /**
+       * <code>optional sint32 nullOrder = 3 [default = 0];</code>
+       */
+      boolean hasNullOrder();
+      /**
+       * <code>optional sint32 nullOrder = 3 [default = 0];</code>
+       */
+      int getNullOrder();
     }
     /**
      * Protobuf type {@code org.apache.hadoop.hive.metastore.hbase.StorageDescriptor.Order}
@@ -22644,6 +22654,11 @@ public final class HbaseMetastoreProto {
                 order_ = input.readSInt32();
                 break;
               }
+              case 24: {
+                bitField0_ |= 0x00000004;
+                nullOrder_ = input.readSInt32();
+                break;
+              }
             }
           }
         } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -22743,9 +22758,26 @@ public final class HbaseMetastoreProto {
         return order_;
       }
 
+      // optional sint32 nullOrder = 3 [default = 0];
+      public static final int NULLORDER_FIELD_NUMBER = 3;
+      private int nullOrder_;
+      /**
+       * <code>optional sint32 nullOrder = 3 [default = 0];</code>
+       */
+      public boolean hasNullOrder() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      /**
+       * <code>optional sint32 nullOrder = 3 [default = 0];</code>
+       */
+      public int getNullOrder() {
+        return nullOrder_;
+      }
+
       private void initFields() {
         columnName_ = "";
         order_ = 1;
+        nullOrder_ = 0;
       }
       private byte memoizedIsInitialized = -1;
       public final boolean isInitialized() {
@@ -22769,6 +22801,9 @@ public final class HbaseMetastoreProto {
         if (((bitField0_ & 0x00000002) == 0x00000002)) {
           output.writeSInt32(2, order_);
         }
+        if (((bitField0_ & 0x00000004) == 0x00000004)) {
+          output.writeSInt32(3, nullOrder_);
+        }
         getUnknownFields().writeTo(output);
       }
 
@@ -22786,6 +22821,10 @@ public final class HbaseMetastoreProto {
           size += com.google.protobuf.CodedOutputStream
             .computeSInt32Size(2, order_);
         }
+        if (((bitField0_ & 0x00000004) == 0x00000004)) {
+          size += com.google.protobuf.CodedOutputStream
+            .computeSInt32Size(3, nullOrder_);
+        }
         size += getUnknownFields().getSerializedSize();
         memoizedSerializedSize = size;
         return size;
@@ -22906,6 +22945,8 @@ public final class HbaseMetastoreProto {
           bitField0_ = (bitField0_ & ~0x00000001);
           order_ = 1;
           bitField0_ = (bitField0_ & ~0x00000002);
+          nullOrder_ = 0;
+          bitField0_ = (bitField0_ & ~0x00000004);
           return this;
         }
 
@@ -22942,6 +22983,10 @@ public final class HbaseMetastoreProto {
             to_bitField0_ |= 0x00000002;
           }
           result.order_ = order_;
+          if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+            to_bitField0_ |= 0x00000004;
+          }
+          result.nullOrder_ = nullOrder_;
           result.bitField0_ = to_bitField0_;
           onBuilt();
           return result;
@@ -22966,6 +23011,9 @@ public final class HbaseMetastoreProto {
           if (other.hasOrder()) {
             setOrder(other.getOrder());
           }
+          if (other.hasNullOrder()) {
+            setNullOrder(other.getNullOrder());
+          }
           this.mergeUnknownFields(other.getUnknownFields());
           return this;
         }
@@ -23104,6 +23152,39 @@ public final class HbaseMetastoreProto {
           return this;
         }
 
+        // optional sint32 nullOrder = 3 [default = 0];
+        private int nullOrder_ ;
+        /**
+         * <code>optional sint32 nullOrder = 3 [default = 0];</code>
+         */
+        public boolean hasNullOrder() {
+          return ((bitField0_ & 0x00000004) == 0x00000004);
+        }
+        /**
+         * <code>optional sint32 nullOrder = 3 [default = 0];</code>
+         */
+        public int getNullOrder() {
+          return nullOrder_;
+        }
+        /**
+         * <code>optional sint32 nullOrder = 3 [default = 0];</code>
+         */
+        public Builder setNullOrder(int value) {
+          bitField0_ |= 0x00000004;
+          nullOrder_ = value;
+          onChanged();
+          return this;
+        }
+        /**
+         * <code>optional sint32 nullOrder = 3 [default = 0];</code>
+         */
+        public Builder clearNullOrder() {
+          bitField0_ = (bitField0_ & ~0x00000004);
+          nullOrder_ = 0;
+          onChanged();
+          return this;
+        }
+
         // @@protoc_insertion_point(builder_scope:org.apache.hadoop.hive.metastore.hbase.StorageDescriptor.Order)
       }
 
@@ -34747,7 +34828,7 @@ public final class HbaseMetastoreProto {
       "ant_info\030\001 \003(\01325.org.apache.hadoop.hive." +
       "metastore.hbase.RoleGrantInfo\"\030\n\010RoleLis",
       "t\022\014\n\004role\030\001 \003(\t\"/\n\004Role\022\023\n\013create_time\030\001" +
-      " \001(\003\022\022\n\nowner_name\030\002 \001(\t\"\254\010\n\021StorageDesc" +
+      " \001(\003\022\022\n\nowner_name\030\002 \001(\t\"\302\010\n\021StorageDesc" +
       "riptor\022A\n\004cols\030\001 \003(\01323.org.apache.hadoop" +
       ".hive.metastore.hbase.FieldSchema\022\024\n\014inp" +
       "ut_format\030\002 \001(\t\022\025\n\routput_format\030\003 \001(\t\022\025" +
@@ -34760,51 +34841,51 @@ public final class HbaseMetastoreProto {
       "skewed_info\030\t \001(\0132D.org.apache.hadoop.hi" +
       "ve.metastore.hbase.StorageDescriptor.Ske" +
       "wedInfo\022!\n\031stored_as_sub_directories\030\n \001" +
-      "(\010\032.\n\005Order\022\023\n\013column_name\030\001 \002(\t\022\020\n\005orde" +
-      "r\030\002 \001(\021:\0011\032|\n\tSerDeInfo\022\014\n\004name\030\001 \001(\t\022\031\n" +
-      "\021serialization_lib\030\002 \001(\t\022F\n\nparameters\030\003" +
-      " \001(\01322.org.apache.hadoop.hive.metastore." +
-      "hbase.Parameters\032\214\003\n\nSkewedInfo\022\030\n\020skewe" +
-      "d_col_names\030\001 \003(\t\022r\n\021skewed_col_values\030\002",
-      " \003(\0132W.org.apache.hadoop.hive.metastore." +
-      "hbase.StorageDescriptor.SkewedInfo.Skewe" +
-      "dColValueList\022\206\001\n\036skewed_col_value_locat" +
-      "ion_maps\030\003 \003(\0132^.org.apache.hadoop.hive." +
-      "metastore.hbase.StorageDescriptor.Skewed" +
-      "Info.SkewedColValueLocationMap\032.\n\022Skewed" +
-      "ColValueList\022\030\n\020skewed_col_value\030\001 \003(\t\0327" +
-      "\n\031SkewedColValueLocationMap\022\013\n\003key\030\001 \003(\t" +
-      "\022\r\n\005value\030\002 \002(\t\"\220\004\n\005Table\022\r\n\005owner\030\001 \001(\t" +
-      "\022\023\n\013create_time\030\002 \001(\003\022\030\n\020last_access_tim",
-      "e\030\003 \001(\003\022\021\n\tretention\030\004 \001(\003\022\020\n\010location\030\005" +
-      " \001(\t\022I\n\rsd_parameters\030\006 \001(\01322.org.apache" +
-      ".hadoop.hive.metastore.hbase.Parameters\022" +
-      "\017\n\007sd_hash\030\007 \002(\014\022K\n\016partition_keys\030\010 \003(\013" +
-      "23.org.apache.hadoop.hive.metastore.hbas" +
-      "e.FieldSchema\022F\n\nparameters\030\t \001(\01322.org." +
-      "apache.hadoop.hive.metastore.hbase.Param" +
-      "eters\022\032\n\022view_original_text\030\n \001(\t\022\032\n\022vie" +
-      "w_expanded_text\030\013 \001(\t\022\022\n\ntable_type\030\014 \001(" +
-      "\t\022Q\n\nprivileges\030\r \001(\0132=.org.apache.hadoo",
-      "p.hive.metastore.hbase.PrincipalPrivileg" +
-      "eSet\022\024\n\014is_temporary\030\016 \001(\010\"\353\004\n\026Partition" +
-      "KeyComparator\022\r\n\005names\030\001 \002(\t\022\r\n\005types\030\002 " +
-      "\002(\t\022S\n\002op\030\003 \003(\0132G.org.apache.hadoop.hive" +
-      ".metastore.hbase.PartitionKeyComparator." +
-      "Operator\022S\n\005range\030\004 \003(\0132D.org.apache.had" +
-      "oop.hive.metastore.hbase.PartitionKeyCom" +
-      "parator.Range\032(\n\004Mark\022\r\n\005value\030\001 \002(\t\022\021\n\t" +
-      "inclusive\030\002 \002(\010\032\272\001\n\005Range\022\013\n\003key\030\001 \002(\t\022R" +
-      "\n\005start\030\002 \001(\0132C.org.apache.hadoop.hive.m",
-      "etastore.hbase.PartitionKeyComparator.Ma" +
-      "rk\022P\n\003end\030\003 \001(\0132C.org.apache.hadoop.hive" +
-      ".metastore.hbase.PartitionKeyComparator." +
-      "Mark\032\241\001\n\010Operator\022Z\n\004type\030\001 \002(\0162L.org.ap" +
-      "ache.hadoop.hive.metastore.hbase.Partiti" +
-      "onKeyComparator.Operator.Type\022\013\n\003key\030\002 \002" +
-      "(\t\022\013\n\003val\030\003 \002(\t\"\037\n\004Type\022\010\n\004LIKE\020\000\022\r\n\tNOT" +
-      "EQUALS\020\001*#\n\rPrincipalType\022\010\n\004USER\020\000\022\010\n\004R" +
-      "OLE\020\001"
+      "(\010\032D\n\005Order\022\023\n\013column_name\030\001 \002(\t\022\020\n\005orde" +
+      "r\030\002 \001(\021:\0011\022\024\n\tnullOrder\030\003 \001(\021:\0010\032|\n\tSerD" +
+      "eInfo\022\014\n\004name\030\001 \001(\t\022\031\n\021serialization_lib" +
+      "\030\002 \001(\t\022F\n\nparameters\030\003 \001(\01322.org.apache." +
+      "hadoop.hive.metastore.hbase.Parameters\032\214" +
+      "\003\n\nSkewedInfo\022\030\n\020skewed_col_names\030\001 \003(\t\022",
+      "r\n\021skewed_col_values\030\002 \003(\0132W.org.apache." +
+      "hadoop.hive.metastore.hbase.StorageDescr" +
+      "iptor.SkewedInfo.SkewedColValueList\022\206\001\n\036" +
+      "skewed_col_value_location_maps\030\003 \003(\0132^.o" +
+      "rg.apache.hadoop.hive.metastore.hbase.St" +
+      "orageDescriptor.SkewedInfo.SkewedColValu" +
+      "eLocationMap\032.\n\022SkewedColValueList\022\030\n\020sk" +
+      "ewed_col_value\030\001 \003(\t\0327\n\031SkewedColValueLo" +
+      "cationMap\022\013\n\003key\030\001 \003(\t\022\r\n\005value\030\002 \002(\t\"\220\004" +
+      "\n\005Table\022\r\n\005owner\030\001 \001(\t\022\023\n\013create_time\030\002 ",
+      "\001(\003\022\030\n\020last_access_time\030\003 \001(\003\022\021\n\tretenti" +
+      "on\030\004 \001(\003\022\020\n\010location\030\005 \001(\t\022I\n\rsd_paramet" +
+      "ers\030\006 \001(\01322.org.apache.hadoop.hive.metas" +
+      "tore.hbase.Parameters\022\017\n\007sd_hash\030\007 \002(\014\022K" +
+      "\n\016partition_keys\030\010 \003(\01323.org.apache.hado" +
+      "op.hive.metastore.hbase.FieldSchema\022F\n\np" +
+      "arameters\030\t \001(\01322.org.apache.hadoop.hive" +
+      ".metastore.hbase.Parameters\022\032\n\022view_orig" +
+      "inal_text\030\n \001(\t\022\032\n\022view_expanded_text\030\013 " +
+      "\001(\t\022\022\n\ntable_type\030\014 \001(\t\022Q\n\nprivileges\030\r ",
+      "\001(\0132=.org.apache.hadoop.hive.metastore.h" +
+      "base.PrincipalPrivilegeSet\022\024\n\014is_tempora" +
+      "ry\030\016 \001(\010\"\353\004\n\026PartitionKeyComparator\022\r\n\005n" +
+      "ames\030\001 \002(\t\022\r\n\005types\030\002 \002(\t\022S\n\002op\030\003 \003(\0132G." +
+      "org.apache.hadoop.hive.metastore.hbase.P" +
+      "artitionKeyComparator.Operator\022S\n\005range\030" +
+      "\004 \003(\0132D.org.apache.hadoop.hive.metastore" +
+      ".hbase.PartitionKeyComparator.Range\032(\n\004M" +
+      "ark\022\r\n\005value\030\001 \002(\t\022\021\n\tinclusive\030\002 \002(\010\032\272\001" +
+      "\n\005Range\022\013\n\003key\030\001 \002(\t\022R\n\005start\030\002 \001(\0132C.or",
+      "g.apache.hadoop.hive.metastore.hbase.Par" +
+      "titionKeyComparator.Mark\022P\n\003end\030\003 \001(\0132C." +
+      "org.apache.hadoop.hive.metastore.hbase.P" +
+      "artitionKeyComparator.Mark\032\241\001\n\010Operator\022" +
+      "Z\n\004type\030\001 \002(\0162L.org.apache.hadoop.hive.m" +
+      "etastore.hbase.PartitionKeyComparator.Op" +
+      "erator.Type\022\013\n\003key\030\002 \002(\t\022\013\n\003val\030\003 \002(\t\"\037\n" +
+      "\004Type\022\010\n\004LIKE\020\000\022\r\n\tNOTEQUALS\020\001*#\n\rPrinci" +
+      "palType\022\010\n\004USER\020\000\022\010\n\004ROLE\020\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -34990,7 +35071,7 @@ public final class HbaseMetastoreProto {
           internal_static_org_apache_hadoop_hive_metastore_hbase_StorageDescriptor_Order_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_org_apache_hadoop_hive_metastore_hbase_StorageDescriptor_Order_descriptor,
-              new java.lang.String[] { "ColumnName", "Order", });
+              new java.lang.String[] { "ColumnName", "Order", "NullOrder", });
           internal_static_org_apache_hadoop_hive_metastore_hbase_StorageDescriptor_SerDeInfo_descriptor =
             internal_static_org_apache_hadoop_hive_metastore_hbase_StorageDescriptor_descriptor.getNestedTypes().get(1);
           internal_static_org_apache_hadoop_hive_metastore_hbase_StorageDescriptor_SerDeInfo_fieldAccessorTable = new

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
index 2695ffa..6534d68 100644
--- a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
+++ b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.cpp
@@ -3101,6 +3101,10 @@ void Order::__set_order(const int32_t val) {
   this->order = val;
 }
 
+void Order::__set_nullOrder(const int32_t val) {
+  this->nullOrder = val;
+}
+
 uint32_t Order::read(::apache::thrift::protocol::TProtocol* iprot) {
 
   apache::thrift::protocol::TInputRecursionTracker tracker(*iprot);
@@ -3138,6 +3142,14 @@ uint32_t Order::read(::apache::thrift::protocol::TProtocol* iprot) {
           xfer += iprot->skip(ftype);
         }
         break;
+      case 3:
+        if (ftype == ::apache::thrift::protocol::T_I32) {
+          xfer += iprot->readI32(this->nullOrder);
+          this->__isset.nullOrder = true;
+        } else {
+          xfer += iprot->skip(ftype);
+        }
+        break;
       default:
         xfer += iprot->skip(ftype);
         break;
@@ -3163,6 +3175,10 @@ uint32_t Order::write(::apache::thrift::protocol::TProtocol* oprot) const {
   xfer += oprot->writeI32(this->order);
   xfer += oprot->writeFieldEnd();
 
+  xfer += oprot->writeFieldBegin("nullOrder", ::apache::thrift::protocol::T_I32, 3);
+  xfer += oprot->writeI32(this->nullOrder);
+  xfer += oprot->writeFieldEnd();
+
   xfer += oprot->writeFieldStop();
   xfer += oprot->writeStructEnd();
   return xfer;
@@ -3172,17 +3188,20 @@ void swap(Order &a, Order &b) {
   using ::std::swap;
   swap(a.col, b.col);
   swap(a.order, b.order);
+  swap(a.nullOrder, b.nullOrder);
   swap(a.__isset, b.__isset);
 }
 
 Order::Order(const Order& other139) {
   col = other139.col;
   order = other139.order;
+  nullOrder = other139.nullOrder;
   __isset = other139.__isset;
 }
 Order& Order::operator=(const Order& other140) {
   col = other140.col;
   order = other140.order;
+  nullOrder = other140.nullOrder;
   __isset = other140.__isset;
   return *this;
 }
@@ -3191,6 +3210,7 @@ void Order::printTo(std::ostream& out) const {
   out << "Order(";
   out << "col=" << to_string(col);
   out << ", " << "order=" << to_string(order);
+  out << ", " << "nullOrder=" << to_string(nullOrder);
   out << ")";
 }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
index 97c07a5..3fd2543 100644
--- a/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
+++ b/metastore/src/gen/thrift/gen-cpp/hive_metastore_types.h
@@ -1575,9 +1575,10 @@ inline std::ostream& operator<<(std::ostream& out, const SerDeInfo& obj)
 }
 
 typedef struct _Order__isset {
-  _Order__isset() : col(false), order(false) {}
+  _Order__isset() : col(false), order(false), nullOrder(false) {}
   bool col :1;
   bool order :1;
+  bool nullOrder :1;
 } _Order__isset;
 
 class Order {
@@ -1585,12 +1586,13 @@ class Order {
 
   Order(const Order&);
   Order& operator=(const Order&);
-  Order() : col(), order(0) {
+  Order() : col(), order(0), nullOrder(0) {
   }
 
   virtual ~Order() throw();
   std::string col;
   int32_t order;
+  int32_t nullOrder;
 
   _Order__isset __isset;
 
@@ -1598,12 +1600,16 @@ class Order {
 
   void __set_order(const int32_t val);
 
+  void __set_nullOrder(const int32_t val);
+
   bool operator == (const Order & rhs) const
   {
     if (!(col == rhs.col))
       return false;
     if (!(order == rhs.order))
       return false;
+    if (!(nullOrder == rhs.nullOrder))
+      return false;
     return true;
   }
   bool operator != (const Order &rhs) const {

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Order.java
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Order.java b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Order.java
index cc0e2dd..fd05de5 100644
--- a/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Order.java
+++ b/metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/Order.java
@@ -40,6 +40,7 @@ public class Order implements org.apache.thrift.TBase<Order, Order._Fields>, jav
 
   private static final org.apache.thrift.protocol.TField COL_FIELD_DESC = new org.apache.thrift.protocol.TField("col", org.apache.thrift.protocol.TType.STRING, (short)1);
   private static final org.apache.thrift.protocol.TField ORDER_FIELD_DESC = new org.apache.thrift.protocol.TField("order", org.apache.thrift.protocol.TType.I32, (short)2);
+  private static final org.apache.thrift.protocol.TField NULL_ORDER_FIELD_DESC = new org.apache.thrift.protocol.TField("nullOrder", org.apache.thrift.protocol.TType.I32, (short)3);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -49,11 +50,13 @@ public class Order implements org.apache.thrift.TBase<Order, Order._Fields>, jav
 
   private String col; // required
   private int order; // required
+  private int nullOrder; // required
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
     COL((short)1, "col"),
-    ORDER((short)2, "order");
+    ORDER((short)2, "order"),
+    NULL_ORDER((short)3, "nullOrder");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -72,6 +75,8 @@ public class Order implements org.apache.thrift.TBase<Order, Order._Fields>, jav
           return COL;
         case 2: // ORDER
           return ORDER;
+        case 3: // NULL_ORDER
+          return NULL_ORDER;
         default:
           return null;
       }
@@ -113,6 +118,7 @@ public class Order implements org.apache.thrift.TBase<Order, Order._Fields>, jav
 
   // isset id assignments
   private static final int __ORDER_ISSET_ID = 0;
+  private static final int __NULLORDER_ISSET_ID = 1;
   private byte __isset_bitfield = 0;
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
@@ -121,6 +127,8 @@ public class Order implements org.apache.thrift.TBase<Order, Order._Fields>, jav
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.ORDER, new org.apache.thrift.meta_data.FieldMetaData("order", org.apache.thrift.TFieldRequirementType.DEFAULT, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
+    tmpMap.put(_Fields.NULL_ORDER, new org.apache.thrift.meta_data.FieldMetaData("nullOrder", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(Order.class, metaDataMap);
   }
@@ -130,12 +138,15 @@ public class Order implements org.apache.thrift.TBase<Order, Order._Fields>, jav
 
   public Order(
     String col,
-    int order)
+    int order,
+    int nullOrder)
   {
     this();
     this.col = col;
     this.order = order;
     setOrderIsSet(true);
+    this.nullOrder = nullOrder;
+    setNullOrderIsSet(true);
   }
 
   /**
@@ -147,6 +158,7 @@ public class Order implements org.apache.thrift.TBase<Order, Order._Fields>, jav
       this.col = other.col;
     }
     this.order = other.order;
+    this.nullOrder = other.nullOrder;
   }
 
   public Order deepCopy() {
@@ -158,6 +170,8 @@ public class Order implements org.apache.thrift.TBase<Order, Order._Fields>, jav
     this.col = null;
     setOrderIsSet(false);
     this.order = 0;
+    setNullOrderIsSet(false);
+    this.nullOrder = 0;
   }
 
   public String getCol() {
@@ -205,6 +219,28 @@ public class Order implements org.apache.thrift.TBase<Order, Order._Fields>, jav
     __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ORDER_ISSET_ID, value);
   }
 
+  public int getNullOrder() {
+    return this.nullOrder;
+  }
+
+  public void setNullOrder(int nullOrder) {
+    this.nullOrder = nullOrder;
+    setNullOrderIsSet(true);
+  }
+
+  public void unsetNullOrder() {
+    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __NULLORDER_ISSET_ID);
+  }
+
+  /** Returns true if field nullOrder is set (has been assigned a value) and false otherwise */
+  public boolean isSetNullOrder() {
+    return EncodingUtils.testBit(__isset_bitfield, __NULLORDER_ISSET_ID);
+  }
+
+  public void setNullOrderIsSet(boolean value) {
+    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __NULLORDER_ISSET_ID, value);
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case COL:
@@ -223,6 +259,14 @@ public class Order implements org.apache.thrift.TBase<Order, Order._Fields>, jav
       }
       break;
 
+    case NULL_ORDER:
+      if (value == null) {
+        unsetNullOrder();
+      } else {
+        setNullOrder((Integer)value);
+      }
+      break;
+
     }
   }
 
@@ -234,6 +278,9 @@ public class Order implements org.apache.thrift.TBase<Order, Order._Fields>, jav
     case ORDER:
       return getOrder();
 
+    case NULL_ORDER:
+      return getNullOrder();
+
     }
     throw new IllegalStateException();
   }
@@ -249,6 +296,8 @@ public class Order implements org.apache.thrift.TBase<Order, Order._Fields>, jav
       return isSetCol();
     case ORDER:
       return isSetOrder();
+    case NULL_ORDER:
+      return isSetNullOrder();
     }
     throw new IllegalStateException();
   }
@@ -284,6 +333,15 @@ public class Order implements org.apache.thrift.TBase<Order, Order._Fields>, jav
         return false;
     }
 
+    boolean this_present_nullOrder = true;
+    boolean that_present_nullOrder = true;
+    if (this_present_nullOrder || that_present_nullOrder) {
+      if (!(this_present_nullOrder && that_present_nullOrder))
+        return false;
+      if (this.nullOrder != that.nullOrder)
+        return false;
+    }
+
     return true;
   }
 
@@ -301,6 +359,11 @@ public class Order implements org.apache.thrift.TBase<Order, Order._Fields>, jav
     if (present_order)
       list.add(order);
 
+    boolean present_nullOrder = true;
+    list.add(present_nullOrder);
+    if (present_nullOrder)
+      list.add(nullOrder);
+
     return list.hashCode();
   }
 
@@ -332,6 +395,16 @@ public class Order implements org.apache.thrift.TBase<Order, Order._Fields>, jav
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetNullOrder()).compareTo(other.isSetNullOrder());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetNullOrder()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nullOrder, other.nullOrder);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -363,6 +436,10 @@ public class Order implements org.apache.thrift.TBase<Order, Order._Fields>, jav
     sb.append("order:");
     sb.append(this.order);
     first = false;
+    if (!first) sb.append(", ");
+    sb.append("nullOrder:");
+    sb.append(this.nullOrder);
+    first = false;
     sb.append(")");
     return sb.toString();
   }
@@ -424,6 +501,14 @@ public class Order implements org.apache.thrift.TBase<Order, Order._Fields>, jav
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 3: // NULL_ORDER
+            if (schemeField.type == org.apache.thrift.protocol.TType.I32) {
+              struct.nullOrder = iprot.readI32();
+              struct.setNullOrderIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -445,6 +530,9 @@ public class Order implements org.apache.thrift.TBase<Order, Order._Fields>, jav
       oprot.writeFieldBegin(ORDER_FIELD_DESC);
       oprot.writeI32(struct.order);
       oprot.writeFieldEnd();
+      oprot.writeFieldBegin(NULL_ORDER_FIELD_DESC);
+      oprot.writeI32(struct.nullOrder);
+      oprot.writeFieldEnd();
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -469,19 +557,25 @@ public class Order implements org.apache.thrift.TBase<Order, Order._Fields>, jav
       if (struct.isSetOrder()) {
         optionals.set(1);
       }
-      oprot.writeBitSet(optionals, 2);
+      if (struct.isSetNullOrder()) {
+        optionals.set(2);
+      }
+      oprot.writeBitSet(optionals, 3);
       if (struct.isSetCol()) {
         oprot.writeString(struct.col);
       }
       if (struct.isSetOrder()) {
         oprot.writeI32(struct.order);
       }
+      if (struct.isSetNullOrder()) {
+        oprot.writeI32(struct.nullOrder);
+      }
     }
 
     @Override
     public void read(org.apache.thrift.protocol.TProtocol prot, Order struct) throws org.apache.thrift.TException {
       TTupleProtocol iprot = (TTupleProtocol) prot;
-      BitSet incoming = iprot.readBitSet(2);
+      BitSet incoming = iprot.readBitSet(3);
       if (incoming.get(0)) {
         struct.col = iprot.readString();
         struct.setColIsSet(true);
@@ -490,6 +584,10 @@ public class Order implements org.apache.thrift.TBase<Order, Order._Fields>, jav
         struct.order = iprot.readI32();
         struct.setOrderIsSet(true);
       }
+      if (incoming.get(2)) {
+        struct.nullOrder = iprot.readI32();
+        struct.setNullOrderIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/metastore/src/gen/thrift/gen-php/metastore/Types.php
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-php/metastore/Types.php b/metastore/src/gen/thrift/gen-php/metastore/Types.php
index 488a920..4da4707 100644
--- a/metastore/src/gen/thrift/gen-php/metastore/Types.php
+++ b/metastore/src/gen/thrift/gen-php/metastore/Types.php
@@ -3076,6 +3076,10 @@ class Order {
    * @var int
    */
   public $order = null;
+  /**
+   * @var int
+   */
+  public $nullOrder = null;
 
   public function __construct($vals=null) {
     if (!isset(self::$_TSPEC)) {
@@ -3088,6 +3092,10 @@ class Order {
           'var' => 'order',
           'type' => TType::I32,
           ),
+        3 => array(
+          'var' => 'nullOrder',
+          'type' => TType::I32,
+          ),
         );
     }
     if (is_array($vals)) {
@@ -3097,6 +3105,9 @@ class Order {
       if (isset($vals['order'])) {
         $this->order = $vals['order'];
       }
+      if (isset($vals['nullOrder'])) {
+        $this->nullOrder = $vals['nullOrder'];
+      }
     }
   }
 
@@ -3133,6 +3144,13 @@ class Order {
             $xfer += $input->skip($ftype);
           }
           break;
+        case 3:
+          if ($ftype == TType::I32) {
+            $xfer += $input->readI32($this->nullOrder);
+          } else {
+            $xfer += $input->skip($ftype);
+          }
+          break;
         default:
           $xfer += $input->skip($ftype);
           break;
@@ -3156,6 +3174,11 @@ class Order {
       $xfer += $output->writeI32($this->order);
       $xfer += $output->writeFieldEnd();
     }
+    if ($this->nullOrder !== null) {
+      $xfer += $output->writeFieldBegin('nullOrder', TType::I32, 3);
+      $xfer += $output->writeI32($this->nullOrder);
+      $xfer += $output->writeFieldEnd();
+    }
     $xfer += $output->writeFieldStop();
     $xfer += $output->writeStructEnd();
     return $xfer;

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py b/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
index 10eaf4a..4b20da9 100644
--- a/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
+++ b/metastore/src/gen/thrift/gen-py/hive_metastore/ttypes.py
@@ -2237,17 +2237,20 @@ class Order:
   Attributes:
    - col
    - order
+   - nullOrder
   """
 
   thrift_spec = (
     None, # 0
     (1, TType.STRING, 'col', None, None, ), # 1
     (2, TType.I32, 'order', None, None, ), # 2
+    (3, TType.I32, 'nullOrder', None, None, ), # 3
   )
 
-  def __init__(self, col=None, order=None,):
+  def __init__(self, col=None, order=None, nullOrder=None,):
     self.col = col
     self.order = order
+    self.nullOrder = nullOrder
 
   def read(self, iprot):
     if iprot.__class__ == TBinaryProtocol.TBinaryProtocolAccelerated and isinstance(iprot.trans, TTransport.CReadableTransport) and self.thrift_spec is not None and fastbinary is not None:
@@ -2268,6 +2271,11 @@ class Order:
           self.order = iprot.readI32()
         else:
           iprot.skip(ftype)
+      elif fid == 3:
+        if ftype == TType.I32:
+          self.nullOrder = iprot.readI32()
+        else:
+          iprot.skip(ftype)
       else:
         iprot.skip(ftype)
       iprot.readFieldEnd()
@@ -2286,6 +2294,10 @@ class Order:
       oprot.writeFieldBegin('order', TType.I32, 2)
       oprot.writeI32(self.order)
       oprot.writeFieldEnd()
+    if self.nullOrder is not None:
+      oprot.writeFieldBegin('nullOrder', TType.I32, 3)
+      oprot.writeI32(self.nullOrder)
+      oprot.writeFieldEnd()
     oprot.writeFieldStop()
     oprot.writeStructEnd()
 
@@ -2297,6 +2309,7 @@ class Order:
     value = 17
     value = (value * 31) ^ hash(self.col)
     value = (value * 31) ^ hash(self.order)
+    value = (value * 31) ^ hash(self.nullOrder)
     return value
 
   def __repr__(self):

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
----------------------------------------------------------------------
diff --git a/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb b/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
index 1cf40ae..556c380 100644
--- a/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
+++ b/metastore/src/gen/thrift/gen-rb/hive_metastore_types.rb
@@ -557,10 +557,12 @@ class Order
   include ::Thrift::Struct, ::Thrift::Struct_Union
   COL = 1
   ORDER = 2
+  NULLORDER = 3
 
   FIELDS = {
     COL => {:type => ::Thrift::Types::STRING, :name => 'col'},
-    ORDER => {:type => ::Thrift::Types::I32, :name => 'order'}
+    ORDER => {:type => ::Thrift::Types::I32, :name => 'order'},
+    NULLORDER => {:type => ::Thrift::Types::I32, :name => 'nullOrder'}
   }
 
   def struct_fields; FIELDS; end

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java b/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
index 6da295e..3b31ee1 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
@@ -672,14 +672,16 @@ class MetaStoreDirectSql {
       t.setParameters(MetaStoreUtils.trimMapNulls(t.getParameters(), convertMapNullsToEmptyStrings));
     }
 
-    queryText = "select \"SD_ID\", \"COLUMN_NAME\", \"SORT_COLS\".\"ORDER\" from \"SORT_COLS\""
+    queryText = "select \"SD_ID\", \"COLUMN_NAME\", \"SORT_COLS\".\"ORDER\", \"SORT_COLS\".\"NULL_ORDER\""
+        + " from \"SORT_COLS\""
         + " where \"SD_ID\" in (" + sdIds + ") and \"INTEGER_IDX\" >= 0"
         + " order by \"SD_ID\" asc, \"INTEGER_IDX\" asc";
     loopJoinOrderedResult(sds, queryText, 0, new ApplyFunc<StorageDescriptor>() {
       @Override
       public void apply(StorageDescriptor t, Object[] fields) {
         if (fields[2] == null) return;
-        t.addToSortCols(new Order((String)fields[1], extractSqlInt(fields[2])));
+        assert fields[3] != null;
+        t.addToSortCols(new Order((String)fields[1], extractSqlInt(fields[2]), extractSqlInt(fields[3])));
       }});
 
     queryText = "select \"SD_ID\", \"BUCKET_COL_NAME\" from \"BUCKETING_COLS\""

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
index d4852b0..1d04ef2 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@ -1367,7 +1367,8 @@ public class ObjectStore implements RawStore, Configurable {
     if (keys != null) {
       mkeys = new ArrayList<MOrder>(keys.size());
       for (Order part : keys) {
-        mkeys.add(new MOrder(HiveStringUtils.normalizeIdentifier(part.getCol()), part.getOrder()));
+        mkeys.add(new MOrder(HiveStringUtils.normalizeIdentifier(part.getCol()), part.getOrder(),
+                part.getNullOrder()));
       }
     }
     return mkeys;
@@ -1378,7 +1379,7 @@ public class ObjectStore implements RawStore, Configurable {
     if (mkeys != null) {
       keys = new ArrayList<Order>(mkeys.size());
       for (MOrder part : mkeys) {
-        keys.add(new Order(part.getCol(), part.getOrder()));
+        keys.add(new Order(part.getCol(), part.getOrder(), part.getNullOrder()));
       }
     }
     return keys;

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java
index d6d01bd..a16997b 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java
@@ -707,7 +707,7 @@ class HBaseUtils {
     sd.setBucketCols(new ArrayList<>(proto.getBucketColsList()));
     List<Order> sortCols = new ArrayList<>();
     for (HbaseMetastoreProto.StorageDescriptor.Order protoOrder : proto.getSortColsList()) {
-      sortCols.add(new Order(protoOrder.getColumnName(), protoOrder.getOrder()));
+      sortCols.add(new Order(protoOrder.getColumnName(), protoOrder.getOrder(), protoOrder.getNullOrder()));
     }
     sd.setSortCols(sortCols);
     if (proto.hasSkewedInfo()) {

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/metastore/src/model/org/apache/hadoop/hive/metastore/model/MOrder.java
----------------------------------------------------------------------
diff --git a/metastore/src/model/org/apache/hadoop/hive/metastore/model/MOrder.java b/metastore/src/model/org/apache/hadoop/hive/metastore/model/MOrder.java
index 1fa82a4..5370c02 100644
--- a/metastore/src/model/org/apache/hadoop/hive/metastore/model/MOrder.java
+++ b/metastore/src/model/org/apache/hadoop/hive/metastore/model/MOrder.java
@@ -21,14 +21,16 @@ package org.apache.hadoop.hive.metastore.model;
 public class MOrder {
   private String col;
   private int order;
-  
+  private int nullOrder;
+
   /**
    * @param col
    * @param order
    */
-  public MOrder(String col, int order) {
+  public MOrder(String col, int order, int nullOrder) {
     this.col = col;
     this.order = order;
+    this.nullOrder = nullOrder;
   }
 
   /**
@@ -58,5 +60,19 @@ public class MOrder {
   public void setOrder(int order) {
     this.order = order;
   }
-  
+
+  /**
+   * @return the null order
+   */
+  public int getNullOrder() {
+    return nullOrder;
+  }
+
+  /**
+   * @param nullOrder the null order to set
+   */
+  public void setNullOrder(int nullOrder) {
+    this.nullOrder = nullOrder;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/metastore/src/protobuf/org/apache/hadoop/hive/metastore/hbase/hbase_metastore_proto.proto
----------------------------------------------------------------------
diff --git a/metastore/src/protobuf/org/apache/hadoop/hive/metastore/hbase/hbase_metastore_proto.proto b/metastore/src/protobuf/org/apache/hadoop/hive/metastore/hbase/hbase_metastore_proto.proto
index 466fdf9..552097b 100644
--- a/metastore/src/protobuf/org/apache/hadoop/hive/metastore/hbase/hbase_metastore_proto.proto
+++ b/metastore/src/protobuf/org/apache/hadoop/hive/metastore/hbase/hbase_metastore_proto.proto
@@ -205,6 +205,7 @@ message StorageDescriptor {
   message Order {
     required string column_name = 1;
     optional sint32 order = 2 [default = 1];
+    optional sint32 nullOrder = 3 [default = 0];
   }
 
   message SerDeInfo {

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStore.java b/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStore.java
index 29d5a64..d938a03 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStore.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStore.java
@@ -18,8 +18,16 @@
  */
 package org.apache.hadoop.hive.metastore.hbase;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.security.MessageDigest;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -58,16 +66,8 @@ import org.junit.Test;
 import org.junit.rules.ExpectedException;
 import org.mockito.Mock;
 import org.mockito.MockitoAnnotations;
-
-import java.io.IOException;
-import java.security.MessageDigest;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.SortedMap;
-import java.util.TreeMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  *
@@ -402,7 +402,7 @@ public class TestHBaseStore {
     Map<String, String> params = new HashMap<String, String>();
     params.put("key", "value");
     StorageDescriptor sd = new StorageDescriptor(cols, "file:/tmp", "input", "output", false, 17,
-        serde, Arrays.asList("bucketcol"), Arrays.asList(new Order("sortcol", 1)), params);
+        serde, Arrays.asList("bucketcol"), Arrays.asList(new Order("sortcol", 1, 0)), params);
     Table table = new Table(tableName, "default", "me", startTime, startTime, 0, sd, null,
         emptyParameters, null, null, null);
     store.createTable(table);
@@ -424,6 +424,7 @@ public class TestHBaseStore {
     Assert.assertEquals(1, t.getSd().getSortColsSize());
     Assert.assertEquals("sortcol", t.getSd().getSortCols().get(0).getCol());
     Assert.assertEquals(1, t.getSd().getSortCols().get(0).getOrder());
+    Assert.assertEquals(0, t.getSd().getSortCols().get(0).getNullOrder());
     Assert.assertEquals(1, t.getSd().getParametersSize());
     Assert.assertEquals("value", t.getSd().getParameters().get("key"));
     Assert.assertEquals("me", t.getOwner());
@@ -1273,7 +1274,7 @@ public class TestHBaseStore {
     Map<String, String> params = new HashMap<String, String>();
     params.put("key", "value");
     StorageDescriptor sd = new StorageDescriptor(cols, "file:/tmp", "input", "output", false, 17,
-        serde, Arrays.asList("bucketcol"), Arrays.asList(new Order("sortcol", 1)), params);
+        serde, Arrays.asList("bucketcol"), Arrays.asList(new Order("sortcol", 1, 0)), params);
     int currentTime = (int)(System.currentTimeMillis() / 1000);
     Table table = new Table(TBL, DB, "me", currentTime, currentTime, 0, sd, cols,
         emptyParameters, null, null, null);
@@ -1291,7 +1292,7 @@ public class TestHBaseStore {
     Map<String, String> params = new HashMap<String, String>();
     params.put("key", "value");
     StorageDescriptor sd = new StorageDescriptor(cols, "file:/tmp", "input", "output", false, 17,
-        serde, Arrays.asList("bucketcol"), Arrays.asList(new Order("sortcol", 1)), params);
+        serde, Arrays.asList("bucketcol"), Arrays.asList(new Order("sortcol", 1, 0)), params);
     int currentTime = (int)(System.currentTimeMillis() / 1000);
     Table table = new Table(TBL, DB, "me", currentTime, currentTime, 0, sd, cols,
         emptyParameters, null, null, null);

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStoreBitVector.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStoreBitVector.java b/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStoreBitVector.java
index b0d7662..570d023 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStoreBitVector.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseStoreBitVector.java
@@ -18,34 +18,30 @@
  */
 package org.apache.hadoop.hive.metastore.hbase;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.api.BinaryColumnStatsData;
-import org.apache.hadoop.hive.metastore.api.BooleanColumnStatsData;
 import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsDesc;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
-import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.Decimal;
 import org.apache.hadoop.hive.metastore.api.DecimalColumnStatsData;
 import org.apache.hadoop.hive.metastore.api.DoubleColumnStatsData;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
-import org.apache.hadoop.hive.metastore.api.Function;
-import org.apache.hadoop.hive.metastore.api.FunctionType;
 import org.apache.hadoop.hive.metastore.api.LongColumnStatsData;
-import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 import org.apache.hadoop.hive.metastore.api.Order;
 import org.apache.hadoop.hive.metastore.api.Partition;
-import org.apache.hadoop.hive.metastore.api.PrincipalType;
-import org.apache.hadoop.hive.metastore.api.ResourceType;
-import org.apache.hadoop.hive.metastore.api.ResourceUri;
-import org.apache.hadoop.hive.metastore.api.Role;
 import org.apache.hadoop.hive.metastore.api.SerDeInfo;
-import org.apache.hadoop.hive.metastore.api.SkewedInfo;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.StringColumnStatsData;
 import org.apache.hadoop.hive.metastore.api.Table;
@@ -58,16 +54,8 @@ import org.junit.Test;
 import org.junit.rules.ExpectedException;
 import org.mockito.Mock;
 import org.mockito.MockitoAnnotations;
-
-import java.io.IOException;
-import java.security.MessageDigest;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.SortedMap;
-import java.util.TreeMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  *
@@ -595,7 +583,7 @@ public class TestHBaseStoreBitVector {
     Map<String, String> params = new HashMap<String, String>();
     params.put("key", "value");
     StorageDescriptor sd = new StorageDescriptor(cols, "file:/tmp", "input", "output", false, 17,
-        serde, Arrays.asList("bucketcol"), Arrays.asList(new Order("sortcol", 1)), params);
+        serde, Arrays.asList("bucketcol"), Arrays.asList(new Order("sortcol", 1, 0)), params);
     int currentTime = (int)(System.currentTimeMillis() / 1000);
     Table table = new Table(TBL, DB, "me", currentTime, currentTime, 0, sd, cols,
         emptyParameters, null, null, null);

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestSharedStorageDescriptor.java
----------------------------------------------------------------------
diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestSharedStorageDescriptor.java b/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestSharedStorageDescriptor.java
index e0d8ce4..8e856a1 100644
--- a/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestSharedStorageDescriptor.java
+++ b/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestSharedStorageDescriptor.java
@@ -18,8 +18,9 @@
  */
 package org.apache.hadoop.hive.metastore.hbase;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import java.util.Iterator;
+import java.util.List;
+
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.Order;
 import org.apache.hadoop.hive.metastore.api.SerDeInfo;
@@ -27,10 +28,8 @@ import org.apache.hadoop.hive.metastore.api.SkewedInfo;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.junit.Assert;
 import org.junit.Test;
-
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
 /**
@@ -80,19 +79,22 @@ public class TestSharedStorageDescriptor {
   @Test
   public void changeOrder() {
     StorageDescriptor sd = new StorageDescriptor();
-    sd.addToSortCols(new Order("fred", 1));
+    sd.addToSortCols(new Order("fred", 1, 0));
     SharedStorageDescriptor ssd = new SharedStorageDescriptor();
     ssd.setShared(sd);
     ssd.getSortCols().get(0).setOrder(2);
+    ssd.getSortCols().get(0).setNullOrder(3);
     Assert.assertFalse(sd.getSortCols() == ssd.getSortCols());
     Assert.assertEquals(2, ssd.getSortCols().get(0).getOrder());
     Assert.assertEquals(1, sd.getSortCols().get(0).getOrder());
+    Assert.assertEquals(3, ssd.getSortCols().get(0).getNullOrder());
+    Assert.assertEquals(0, sd.getSortCols().get(0).getNullOrder());
   }
 
   @Test
   public void unsetOrder() {
     StorageDescriptor sd = new StorageDescriptor();
-    sd.addToSortCols(new Order("fred", 1));
+    sd.addToSortCols(new Order("fred", 1, 0));
     SharedStorageDescriptor ssd = new SharedStorageDescriptor();
     ssd.setShared(sd);
     ssd.unsetSortCols();

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
index c51cfd6..70afe16 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
@@ -2140,6 +2140,11 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
             else if (sortCol.getOrder() == BaseSemanticAnalyzer.HIVE_COLUMN_ORDER_DESC) {
               sortKeyDesc = sortKeyDesc + "DESC";
             }
+            if (sortCol.getNullOrder() == BaseSemanticAnalyzer.HIVE_COLUMN_NULLS_FIRST) {
+              sortKeyDesc = sortKeyDesc + " NULLS FIRST";
+            } else if (sortCol.getNullOrder() == BaseSemanticAnalyzer.HIVE_COLUMN_NULLS_LAST) {
+              sortKeyDesc = sortKeyDesc + " NULLS LAST";
+            }
             sortKeys.add(sortKeyDesc);
           }
           tbl_sort_bucket += StringUtils.join(sortKeys, ", \n");

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java
index 4d2447b..e79fccd 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/MapJoinOperator.java
@@ -653,6 +653,8 @@ public class MapJoinOperator extends AbstractMapJoinOperator<MapJoinDesc> implem
     spilledMapJoinTables[pos] = new MapJoinBytesTableContainer(restoredHashMap);
     spilledMapJoinTables[pos].setInternalValueOi(container.getInternalValueOi());
     spilledMapJoinTables[pos].setSortableSortOrders(container.getSortableSortOrders());
+    spilledMapJoinTables[pos].setNullMarkers(container.getNullMarkers());
+    spilledMapJoinTables[pos].setNotNullMarkers(container.getNotNullMarkers());
   }
 
   /**


[10/18] hive git commit: HIVE-12994: Implement support for NULLS FIRST/NULLS LAST (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/outer_join_ppr.q.java1.7.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/outer_join_ppr.q.java1.7.out b/ql/src/test/results/clientpositive/spark/outer_join_ppr.q.java1.7.out
index 4edb4a9..9b1a503 100644
--- a/ql/src/test/results/clientpositive/spark/outer_join_ppr.q.java1.7.out
+++ b/ql/src/test/results/clientpositive/spark/outer_join_ppr.q.java1.7.out
@@ -129,6 +129,7 @@ STAGE PLANS:
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: string)
+                      null sort order: a
                       sort order: +
                       Map-reduce partition columns: _col0 (type: string)
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -196,6 +197,7 @@ STAGE PLANS:
                     Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: string)
+                      null sort order: a
                       sort order: +
                       Map-reduce partition columns: _col0 (type: string)
                       Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
@@ -614,6 +616,7 @@ STAGE PLANS:
                       Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE
@@ -685,6 +688,7 @@ STAGE PLANS:
                       Statistics: Num rows: 111 Data size: 1179 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 111 Data size: 1179 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/pcr.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/pcr.q.out b/ql/src/test/results/clientpositive/spark/pcr.q.out
index 681251d..3886272 100644
--- a/ql/src/test/results/clientpositive/spark/pcr.q.out
+++ b/ql/src/test/results/clientpositive/spark/pcr.q.out
@@ -85,11 +85,13 @@ TOK_QUERY
                5
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               ds
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  ds
 
 
 STAGE DEPENDENCIES:
@@ -119,6 +121,7 @@ STAGE PLANS:
                       Statistics: Num rows: 13 Data size: 104 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: int), _col2 (type: string)
+                        null sort order: aa
                         sort order: ++
                         Statistics: Num rows: 13 Data size: 104 Basic stats: COMPLETE Column stats: NONE
                         tag: -1
@@ -312,8 +315,9 @@ TOK_QUERY
                5
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
 
 
 STAGE DEPENDENCIES:
@@ -343,6 +347,7 @@ STAGE PLANS:
                       Statistics: Num rows: 40 Data size: 320 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: int)
+                        null sort order: a
                         sort order: +
                         Statistics: Num rows: 40 Data size: 320 Basic stats: COMPLETE Column stats: NONE
                         tag: -1
@@ -627,11 +632,13 @@ TOK_QUERY
                'val_2'
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               ds
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  ds
 
 
 STAGE DEPENDENCIES:
@@ -661,6 +668,7 @@ STAGE PLANS:
                       Statistics: Num rows: 13 Data size: 104 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: int), _col2 (type: string)
+                        null sort order: aa
                         sort order: ++
                         Statistics: Num rows: 13 Data size: 104 Basic stats: COMPLETE Column stats: NONE
                         tag: -1
@@ -869,11 +877,13 @@ TOK_QUERY
                   'val_5'
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               ds
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  ds
 
 
 STAGE DEPENDENCIES:
@@ -903,6 +913,7 @@ STAGE PLANS:
                       Statistics: Num rows: 10 Data size: 80 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: int), _col2 (type: string)
+                        null sort order: aa
                         sort order: ++
                         Statistics: Num rows: 10 Data size: 80 Basic stats: COMPLETE Column stats: NONE
                         tag: -1
@@ -1113,11 +1124,13 @@ TOK_QUERY
                   'val_5'
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               ds
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  ds
 
 
 STAGE DEPENDENCIES:
@@ -1147,6 +1160,7 @@ STAGE PLANS:
                       Statistics: Num rows: 16 Data size: 128 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: int), _col2 (type: string)
+                        null sort order: aa
                         sort order: ++
                         Statistics: Num rows: 16 Data size: 128 Basic stats: COMPLETE Column stats: NONE
                         tag: -1
@@ -1413,11 +1427,13 @@ TOK_QUERY
                   'val_5'
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               ds
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  ds
 
 
 STAGE DEPENDENCIES:
@@ -1447,6 +1463,7 @@ STAGE PLANS:
                       Statistics: Num rows: 33 Data size: 264 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: int), _col2 (type: string)
+                        null sort order: aa
                         sort order: ++
                         Statistics: Num rows: 33 Data size: 264 Basic stats: COMPLETE Column stats: NONE
                         tag: -1
@@ -1713,11 +1730,13 @@ TOK_QUERY
                14
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               value
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  value
 
 
 STAGE DEPENDENCIES:
@@ -1747,6 +1766,7 @@ STAGE PLANS:
                       Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: 14 (type: int), _col1 (type: string)
+                        null sort order: aa
                         sort order: ++
                         Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
                         tag: -1
@@ -1929,11 +1949,13 @@ TOK_QUERY
                '2000-04-09'
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               value
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  value
 
 
 STAGE DEPENDENCIES:
@@ -1959,6 +1981,7 @@ STAGE PLANS:
                     Statistics: Num rows: 40 Data size: 320 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: int), _col1 (type: string)
+                      null sort order: aa
                       sort order: ++
                       Statistics: Num rows: 40 Data size: 320 Basic stats: COMPLETE Column stats: NONE
                       tag: -1
@@ -2181,11 +2204,13 @@ TOK_QUERY
                '2000-04-10'
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               value
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  value
 
 
 STAGE DEPENDENCIES:
@@ -2211,6 +2236,7 @@ STAGE PLANS:
                     Statistics: Num rows: 60 Data size: 480 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: int), _col1 (type: string)
+                      null sort order: aa
                       sort order: ++
                       Statistics: Num rows: 60 Data size: 480 Basic stats: COMPLETE Column stats: NONE
                       tag: -1
@@ -2514,14 +2540,17 @@ TOK_QUERY
                   2
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               value
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  value
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               ds
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  ds
 
 
 STAGE DEPENDENCIES:
@@ -2551,6 +2580,7 @@ STAGE PLANS:
                       Statistics: Num rows: 40 Data size: 320 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string)
+                        null sort order: aaa
                         sort order: +++
                         Statistics: Num rows: 40 Data size: 320 Basic stats: COMPLETE Column stats: NONE
                         tag: -1
@@ -2749,10 +2779,11 @@ TOK_QUERY
             TOK_ALLCOLREF
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            .
-               TOK_TABLE_OR_COL
-                  t1
-               key
+            TOK_NULLS_FIRST
+               .
+                  TOK_TABLE_OR_COL
+                     t1
+                  key
 
 
 STAGE DEPENDENCIES:
@@ -2783,6 +2814,7 @@ STAGE PLANS:
                       Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: int)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
@@ -2855,6 +2887,7 @@ STAGE PLANS:
                       Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: int)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
@@ -2924,6 +2957,7 @@ STAGE PLANS:
                 Statistics: Num rows: 22 Data size: 176 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
+                  null sort order: a
                   sort order: +
                   Statistics: Num rows: 22 Data size: 176 Basic stats: COMPLETE Column stats: NONE
                   tag: -1
@@ -3062,10 +3096,11 @@ TOK_QUERY
             TOK_ALLCOLREF
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            .
-               TOK_TABLE_OR_COL
-                  t1
-               key
+            TOK_NULLS_FIRST
+               .
+                  TOK_TABLE_OR_COL
+                     t1
+                  key
 
 
 STAGE DEPENDENCIES:
@@ -3096,6 +3131,7 @@ STAGE PLANS:
                       Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: int)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
@@ -3168,6 +3204,7 @@ STAGE PLANS:
                       Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: int)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: int)
                         Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE
@@ -3237,6 +3274,7 @@ STAGE PLANS:
                 Statistics: Num rows: 22 Data size: 176 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
                   key expressions: _col0 (type: int)
+                  null sort order: a
                   sort order: +
                   Statistics: Num rows: 22 Data size: 176 Basic stats: COMPLETE Column stats: NONE
                   tag: -1
@@ -3390,14 +3428,17 @@ TOK_QUERY
                   2
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               value
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  value
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               ds
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  ds
 
 
 STAGE DEPENDENCIES:
@@ -3427,6 +3468,7 @@ STAGE PLANS:
                       Statistics: Num rows: 48 Data size: 384 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string)
+                        null sort order: aaa
                         sort order: +++
                         Statistics: Num rows: 48 Data size: 384 Basic stats: COMPLETE Column stats: NONE
                         tag: -1
@@ -3760,14 +3802,17 @@ TOK_QUERY
                   2
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               value
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  value
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               ds
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  ds
 
 
 STAGE DEPENDENCIES:
@@ -3797,6 +3842,7 @@ STAGE PLANS:
                       Statistics: Num rows: 30 Data size: 240 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string)
+                        null sort order: aaa
                         sort order: +++
                         Statistics: Num rows: 30 Data size: 240 Basic stats: COMPLETE Column stats: NONE
                         tag: -1
@@ -4630,8 +4676,9 @@ TOK_QUERY
                11
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
       TOK_LIMIT
          10
 
@@ -4659,6 +4706,7 @@ STAGE PLANS:
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: string)
+                      null sort order: a
                       sort order: +
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                       tag: -1
@@ -4814,14 +4862,17 @@ TOK_QUERY
                11
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               ds
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  ds
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               hr
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  hr
 
 
 STAGE DEPENDENCIES:
@@ -4851,6 +4902,7 @@ STAGE PLANS:
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: '11' (type: string), '2008-04-08' (type: string), _col3 (type: string)
+                        null sort order: aaa
                         sort order: +++
                         Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                         tag: -1
@@ -5044,14 +5096,17 @@ TOK_QUERY
                11
       TOK_ORDERBY
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               key
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  key
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               ds
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  ds
          TOK_TABSORTCOLNAMEASC
-            TOK_TABLE_OR_COL
-               hr
+            TOK_NULLS_FIRST
+               TOK_TABLE_OR_COL
+                  hr
 
 
 STAGE DEPENDENCIES:
@@ -5081,6 +5136,7 @@ STAGE PLANS:
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: '11' (type: string), _col2 (type: string), '11' (type: string)
+                        null sort order: aaa
                         sort order: +++
                         Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                         tag: -1

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/ppd_join_filter.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/ppd_join_filter.q.out b/ql/src/test/results/clientpositive/spark/ppd_join_filter.q.out
index e87ded7..9f94f16 100644
--- a/ql/src/test/results/clientpositive/spark/ppd_join_filter.q.out
+++ b/ql/src/test/results/clientpositive/spark/ppd_join_filter.q.out
@@ -147,6 +147,7 @@ STAGE PLANS:
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -219,6 +220,7 @@ STAGE PLANS:
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -330,6 +332,7 @@ STAGE PLANS:
                     Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: string)
+                      null sort order: a
                       sort order: +
                       Map-reduce partition columns: _col0 (type: string)
                       Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
@@ -526,6 +529,7 @@ STAGE PLANS:
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -598,6 +602,7 @@ STAGE PLANS:
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -709,6 +714,7 @@ STAGE PLANS:
                     Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: string)
+                      null sort order: a
                       sort order: +
                       Map-reduce partition columns: _col0 (type: string)
                       Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
@@ -905,6 +911,7 @@ STAGE PLANS:
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -977,6 +984,7 @@ STAGE PLANS:
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -1088,6 +1096,7 @@ STAGE PLANS:
                     Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: string)
+                      null sort order: a
                       sort order: +
                       Map-reduce partition columns: _col0 (type: string)
                       Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
@@ -1284,6 +1293,7 @@ STAGE PLANS:
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -1356,6 +1366,7 @@ STAGE PLANS:
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: _col0 (type: string)
+                        null sort order: a
                         sort order: +
                         Map-reduce partition columns: _col0 (type: string)
                         Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -1467,6 +1478,7 @@ STAGE PLANS:
                     Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: string)
+                      null sort order: a
                       sort order: +
                       Map-reduce partition columns: _col0 (type: string)
                       Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/ptf.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/ptf.q.out b/ql/src/test/results/clientpositive/spark/ptf.q.out
index a826bc3..0640ca3 100644
--- a/ql/src/test/results/clientpositive/spark/ptf.q.out
+++ b/ql/src/test/results/clientpositive/spark/ptf.q.out
@@ -62,7 +62,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -88,7 +88,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -266,7 +266,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
@@ -292,7 +292,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -417,7 +417,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
@@ -540,7 +540,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: abc
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -566,7 +566,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -717,7 +717,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
@@ -743,7 +743,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -897,7 +897,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
@@ -932,7 +932,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col0
                         raw input shape:
                         window functions:
@@ -1092,7 +1092,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: abc
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col0: int, _col1: string, _col2: string, _col3: string, _col4: string, _col5: int, _col6: string, _col7: double, _col8: string
                         partition by: _col2
                         raw input shape:
@@ -1262,7 +1262,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: abc
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col0: int, _col1: string, _col2: string, _col3: string, _col4: string, _col5: int, _col6: string, _col7: double, _col8: string
                         partition by: _col2
                         raw input shape:
@@ -1369,7 +1369,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_1
                           name: noopwithmap
-                          order by: p_name, p_size(DESC)
+                          order by: p_name ASC NULLS FIRST, p_size DESC NULLS LAST
                           output shape: p_name: string, p_mfgr: string, p_size: int
                           partition by: p_mfgr
                           raw input shape:
@@ -1396,7 +1396,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopwithmap
-                        order by: _col1, _col5(DESC)
+                        order by: _col1 ASC NULLS FIRST, _col5 DESC NULLS LAST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
@@ -1422,7 +1422,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1, _col5(DESC)
+                        order by: _col1 ASC NULLS FIRST, _col5 DESC NULLS LAST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -1540,7 +1540,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_1
                           name: noopwithmap
-                          order by: p_name
+                          order by: p_name ASC NULLS FIRST
                           output shape: p_name: string, p_mfgr: string, p_size: int, p_retailprice: double
                           partition by: p_mfgr
                           raw input shape:
@@ -1568,7 +1568,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopwithmap
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -1595,7 +1595,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -1742,7 +1742,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -1768,7 +1768,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -1918,7 +1918,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col2(DESC), _col1
+                        order by: _col2 DESC NULLS LAST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -1932,7 +1932,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_1
                           name: noopwithmap
-                          order by: _col2(DESC), _col1
+                          order by: _col2 DESC NULLS LAST, _col1 ASC NULLS FIRST
                           output shape: _col1: string, _col2: string, _col5: int, _col7: double
                           partition by: _col2
                           raw input shape:
@@ -1940,7 +1940,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_2
                           name: noop
-                          order by: _col2(DESC), _col1
+                          order by: _col2 DESC NULLS LAST, _col1 ASC NULLS FIRST
                           output shape: _col1: string, _col2: string, _col5: int, _col7: double
                           partition by: _col2
                           raw input shape:
@@ -1967,7 +1967,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopwithmap
-                        order by: _col2(DESC), _col1
+                        order by: _col2 DESC NULLS LAST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -1975,7 +1975,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_2
                         name: noop
-                        order by: _col2(DESC), _col1
+                        order by: _col2 DESC NULLS LAST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -2001,7 +2001,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -2158,7 +2158,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -2184,7 +2184,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -2351,7 +2351,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: abc
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col0: int, _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -2396,7 +2396,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -2558,7 +2558,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
@@ -2723,7 +2723,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col0
+                        order by: _col0 ASC NULLS FIRST
                         output shape: _col0: string, _col1: string, _col2: double
                         partition by: _col0
                         raw input shape:
@@ -2749,7 +2749,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col0
                         raw input shape:
                         window functions:
@@ -2936,7 +2936,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -2988,7 +2988,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col5
+                        order by: _col5 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -3024,7 +3024,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col3, _col2
+                        order by: _col3 ASC NULLS FIRST, _col2 ASC NULLS FIRST
                         partition by: _col3
                         raw input shape:
                         window functions:
@@ -3082,7 +3082,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                   Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
@@ -3106,7 +3106,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                   Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
@@ -3336,14 +3336,14 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col2
+                        order by: _col2 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
                       Partition table definition
                         input alias: ptf_2
                         name: noop
-                        order by: _col2
+                        order by: _col2 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
@@ -3357,7 +3357,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_1
                           name: noopwithmap
-                          order by: _col2, _col1
+                          order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                           output shape: _col1: string, _col2: string, _col5: int
                           partition by: _col2, _col1
                           raw input shape:
@@ -3365,7 +3365,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_2
                           name: noop
-                          order by: _col2, _col1
+                          order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                           output shape: _col1: string, _col2: string, _col5: int
                           partition by: _col2, _col1
                           raw input shape:
@@ -3392,7 +3392,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopwithmap
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
@@ -3400,7 +3400,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_2
                         name: noop
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
@@ -3426,7 +3426,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         partition by: _col2, _col1
                         raw input shape:
                         window functions:
@@ -3607,14 +3607,14 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col2
+                        order by: _col2 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
                       Partition table definition
                         input alias: ptf_2
                         name: noop
-                        order by: _col2
+                        order by: _col2 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
@@ -3640,7 +3640,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
@@ -3666,7 +3666,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col2
+                        order by: _col2 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
@@ -3692,7 +3692,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -3868,14 +3868,14 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
                       Partition table definition
                         input alias: ptf_2
                         name: noop
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
@@ -3901,14 +3901,14 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col2
+                        order by: _col2 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
                       Partition table definition
                         input alias: ptf_2
                         name: noop
-                        order by: _col2
+                        order by: _col2 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
@@ -3934,7 +3934,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -4111,14 +4111,14 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
                       Partition table definition
                         input alias: ptf_2
                         name: noop
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
@@ -4144,7 +4144,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col2
+                        order by: _col2 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
@@ -4158,7 +4158,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_1
                           name: noopwithmap
-                          order by: _col2, _col1
+                          order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                           output shape: _col1: string, _col2: string, _col5: int
                           partition by: _col2, _col1
                           raw input shape:
@@ -4186,7 +4186,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopwithmap
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
@@ -4213,7 +4213,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         partition by: _col2, _col1
                         raw input shape:
                         window functions:
@@ -4391,7 +4391,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
@@ -4405,7 +4405,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_1
                           name: noopwithmap
-                          order by: _col2
+                          order by: _col2 ASC NULLS FIRST
                           output shape: _col1: string, _col2: string, _col5: int
                           partition by: _col2
                           raw input shape:
@@ -4413,7 +4413,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_2
                           name: noop
-                          order by: _col2
+                          order by: _col2 ASC NULLS FIRST
                           output shape: _col1: string, _col2: string, _col5: int
                           partition by: _col2
                           raw input shape:
@@ -4440,7 +4440,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopwithmap
-                        order by: _col2
+                        order by: _col2 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
@@ -4448,7 +4448,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_2
                         name: noop
-                        order by: _col2
+                        order by: _col2 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
@@ -4474,7 +4474,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         partition by: _col2, _col1
                         raw input shape:
                         window functions:
@@ -4646,14 +4646,14 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
                       Partition table definition
                         input alias: ptf_2
                         name: noop
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
@@ -4667,7 +4667,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_1
                           name: noopwithmap
-                          order by: _col2, _col1
+                          order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                           output shape: _col1: string, _col2: string, _col5: int
                           partition by: _col2, _col1
                           raw input shape:
@@ -4695,7 +4695,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopwithmap
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
@@ -4722,7 +4722,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/ptf_matchpath.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/ptf_matchpath.q.out b/ql/src/test/results/clientpositive/spark/ptf_matchpath.q.out
index fc35091..1098240 100644
--- a/ql/src/test/results/clientpositive/spark/ptf_matchpath.q.out
+++ b/ql/src/test/results/clientpositive/spark/ptf_matchpath.q.out
@@ -100,7 +100,7 @@ STAGE PLANS:
                         input alias: ptf_1
                         arguments: 'LATE.LATE+', 'LATE', (_col5 > 15.0), 'origin_city_name, fl_num, year, month, day_of_month, size(tpath) as sz, tpath[0].day_of_month as tpath'
                         name: matchpath
-                        order by: _col2, _col3, _col4
+                        order by: _col2 ASC NULLS FIRST, _col3 ASC NULLS FIRST, _col4 ASC NULLS FIRST
                         output shape: tpath: int
                         partition by: _col6
                         raw input shape:
@@ -226,7 +226,7 @@ STAGE PLANS:
                         input alias: ptf_1
                         arguments: 'LATE.LATE+', 'LATE', (_col5 > 15.0), 'origin_city_name, fl_num, year, month, day_of_month, size(tpath) as sz, tpath[0].day_of_month as tpath'
                         name: matchpath
-                        order by: _col6, _col2, _col3, _col4
+                        order by: _col6 ASC NULLS FIRST, _col2 ASC NULLS FIRST, _col3 ASC NULLS FIRST, _col4 ASC NULLS FIRST
                         output shape: tpath: int
                         partition by: 0
                         raw input shape:
@@ -349,7 +349,7 @@ STAGE PLANS:
                         input alias: ptf_1
                         arguments: 'LATE.LATE+', 'LATE', (_col5 > 15.0), 'origin_city_name, fl_num, year, month, day_of_month, size(tpath) as sz, tpath[0].day_of_month as tpath'
                         name: matchpath
-                        order by: _col6, _col2, _col3, _col4
+                        order by: _col6 ASC NULLS FIRST, _col2 ASC NULLS FIRST, _col3 ASC NULLS FIRST, _col4 ASC NULLS FIRST
                         output shape: tpath: int
                         partition by: 0
                         raw input shape:

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/ptf_streaming.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/ptf_streaming.q.out b/ql/src/test/results/clientpositive/spark/ptf_streaming.q.out
index ecaa779..8e47b11 100644
--- a/ql/src/test/results/clientpositive/spark/ptf_streaming.q.out
+++ b/ql/src/test/results/clientpositive/spark/ptf_streaming.q.out
@@ -62,7 +62,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopstreaming
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -88,7 +88,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -266,7 +266,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopstreaming
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
@@ -292,7 +292,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -433,7 +433,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: abc
                         name: noopstreaming
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col0: int, _col1: string, _col2: string, _col3: string, _col4: string, _col5: int, _col6: string, _col7: double, _col8: string
                         partition by: _col2
                         raw input shape:
@@ -557,7 +557,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_1
                           name: noopwithmapstreaming
-                          order by: p_name, p_size(DESC)
+                          order by: p_name ASC NULLS FIRST, p_size DESC NULLS LAST
                           output shape: p_name: string, p_mfgr: string, p_size: int
                           partition by: p_mfgr
                           raw input shape:
@@ -584,7 +584,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopwithmapstreaming
-                        order by: _col1, _col5(DESC)
+                        order by: _col1 ASC NULLS FIRST, _col5 DESC NULLS LAST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
@@ -610,7 +610,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1, _col5(DESC)
+                        order by: _col1 ASC NULLS FIRST, _col5 DESC NULLS LAST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -728,7 +728,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_1
                           name: noopwithmapstreaming
-                          order by: p_name
+                          order by: p_name ASC NULLS FIRST
                           output shape: p_name: string, p_mfgr: string, p_size: int, p_retailprice: double
                           partition by: p_mfgr
                           raw input shape:
@@ -756,7 +756,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopwithmapstreaming
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -783,7 +783,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -933,7 +933,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopstreaming
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -947,7 +947,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_1
                           name: noopwithmapstreaming
-                          order by: _col2, _col1
+                          order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                           output shape: _col1: string, _col2: string, _col5: int, _col7: double
                           partition by: _col2
                           raw input shape:
@@ -955,7 +955,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_2
                           name: noopstreaming
-                          order by: _col2, _col1
+                          order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                           output shape: _col1: string, _col2: string, _col5: int, _col7: double
                           partition by: _col2
                           raw input shape:
@@ -982,7 +982,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopwithmapstreaming
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -990,7 +990,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_2
                         name: noopstreaming
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -1016,7 +1016,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -1168,7 +1168,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopstreaming
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -1182,7 +1182,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_1
                           name: noopwithmap
-                          order by: _col2, _col1
+                          order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                           output shape: _col1: string, _col2: string, _col5: int, _col7: double
                           partition by: _col2
                           raw input shape:
@@ -1190,7 +1190,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_2
                           name: noopstreaming
-                          order by: _col2, _col1
+                          order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                           output shape: _col1: string, _col2: string, _col5: int, _col7: double
                           partition by: _col2
                           raw input shape:
@@ -1217,7 +1217,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopwithmap
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -1225,7 +1225,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_2
                         name: noopstreaming
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -1251,7 +1251,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -1403,7 +1403,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopstreaming
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -1417,7 +1417,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_1
                           name: noopwithmapstreaming
-                          order by: _col2, _col1
+                          order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                           output shape: _col1: string, _col2: string, _col5: int, _col7: double
                           partition by: _col2
                           raw input shape:
@@ -1425,7 +1425,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_2
                           name: noop
-                          order by: _col2, _col1
+                          order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                           output shape: _col1: string, _col2: string, _col5: int, _col7: double
                           partition by: _col2
                           raw input shape:
@@ -1452,7 +1452,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopwithmapstreaming
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -1460,7 +1460,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_2
                         name: noop
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -1486,7 +1486,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -1655,7 +1655,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: abc
                         name: noopstreaming
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         output shape: _col0: int, _col1: string, _col2: string, _col5: int, _col7: double
                         partition by: _col2
                         raw input shape:
@@ -1700,7 +1700,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -1885,14 +1885,14 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopstreaming
-                        order by: _col2
+                        order by: _col2 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
                       Partition table definition
                         input alias: ptf_2
                         name: noop
-                        order by: _col2
+                        order by: _col2 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
@@ -1906,7 +1906,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_1
                           name: noopwithmap
-                          order by: _col2, _col1
+                          order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                           output shape: _col1: string, _col2: string, _col5: int
                           partition by: _col2, _col1
                           raw input shape:
@@ -1914,7 +1914,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_2
                           name: noopstreaming
-                          order by: _col2, _col1
+                          order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                           output shape: _col1: string, _col2: string, _col5: int
                           partition by: _col2, _col1
                           raw input shape:
@@ -1941,7 +1941,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopwithmap
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
@@ -1949,7 +1949,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_2
                         name: noopstreaming
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
@@ -1975,7 +1975,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         partition by: _col2, _col1
                         raw input shape:
                         window functions:
@@ -2156,14 +2156,14 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopstreaming
-                        order by: _col2
+                        order by: _col2 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
                       Partition table definition
                         input alias: ptf_2
                         name: noop
-                        order by: _col2
+                        order by: _col2 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
@@ -2189,7 +2189,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopstreaming
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
@@ -2215,7 +2215,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noop
-                        order by: _col2
+                        order by: _col2 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2
                         raw input shape:
@@ -2241,7 +2241,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:
@@ -2415,14 +2415,14 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopstreaming
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
                       Partition table definition
                         input alias: ptf_2
                         name: noop
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
@@ -2436,7 +2436,7 @@ STAGE PLANS:
                         Partition table definition
                           input alias: ptf_1
                           name: noopwithmapstreaming
-                          order by: _col2, _col1
+                          order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                           output shape: _col1: string, _col2: string, _col5: int
                           partition by: _col2, _col1
                           raw input shape:
@@ -2464,7 +2464,7 @@ STAGE PLANS:
                       Partition table definition
                         input alias: ptf_1
                         name: noopwithmapstreaming
-                        order by: _col2, _col1
+                        order by: _col2 ASC NULLS FIRST, _col1 ASC NULLS FIRST
                         output shape: _col1: string, _col2: string, _col5: int
                         partition by: _col2, _col1
                         raw input shape:
@@ -2491,7 +2491,7 @@ STAGE PLANS:
                       Windowing table definition
                         input alias: ptf_1
                         name: windowingtablefunction
-                        order by: _col1
+                        order by: _col1 ASC NULLS FIRST
                         partition by: _col2
                         raw input shape:
                         window functions:

http://git-wip-us.apache.org/repos/asf/hive/blob/9350b693/ql/src/test/results/clientpositive/spark/reduce_deduplicate.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/reduce_deduplicate.q.out b/ql/src/test/results/clientpositive/spark/reduce_deduplicate.q.out
index b38a96b..83988d3 100644
--- a/ql/src/test/results/clientpositive/spark/reduce_deduplicate.q.out
+++ b/ql/src/test/results/clientpositive/spark/reduce_deduplicate.q.out
@@ -58,6 +58,7 @@ STAGE PLANS:
                     Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: string)
+                      null sort order: a
                       sort order: +
                       Map-reduce partition columns: _col0 (type: string)
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -421,6 +422,7 @@ STAGE PLANS:
                         Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
                         Reduce Output Operator
                           key expressions: _col1 (type: string)
+                          null sort order: a
                           sort order: +
                           Map-reduce partition columns: _col1 (type: string)
                           Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE