You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by se...@apache.org on 2016/10/11 23:09:56 UTC

[01/22] hive git commit: HIVE-14882: Lean qtest initialization for CliDrivers (Prasanth Jayachandran reviewed by Siddharth Seth)

Repository: hive
Updated Branches:
  refs/heads/hive-14535 0ce24b93e -> b9e815722


HIVE-14882: Lean qtest initialization for CliDrivers (Prasanth Jayachandran reviewed by Siddharth Seth)


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

Branch: refs/heads/hive-14535
Commit: 481d7cd5c469217b7c8c2ad50e06242beefec17f
Parents: efe9c84
Author: Prasanth Jayachandran <pr...@apache.org>
Authored: Tue Oct 4 17:47:03 2016 -0700
Committer: Prasanth Jayachandran <pr...@apache.org>
Committed: Tue Oct 4 17:47:03 2016 -0700

----------------------------------------------------------------------
 data/scripts/q_test_cleanup_compare.sql         |  1 +
 data/scripts/q_test_cleanup_contrib.sql         |  2 +
 data/scripts/q_test_cleanup_for_encryption.sql  |  1 -
 data/scripts/q_test_cleanup_src.sql             |  1 +
 data/scripts/q_test_cleanup_tez.sql             |  4 +
 data/scripts/q_test_init_compare.sql            | 26 +++++++
 data/scripts/q_test_init_contrib.sql            | 29 ++++++++
 data/scripts/q_test_init_for_encryption.sql     |  5 --
 data/scripts/q_test_init_src.sql                |  5 ++
 data/scripts/q_test_init_src_with_stats.sql     |  9 +++
 data/scripts/q_test_init_tez.sql                | 78 ++++++++++++++++++++
 .../apache/hadoop/hive/ql/TestMTQueries.java    |  3 +-
 .../hadoop/hive/cli/control/CliConfigs.java     | 36 ++++-----
 .../org/apache/hadoop/hive/ql/QTestUtil.java    |  6 +-
 .../clientpositive/tez/explainuser_3.q.out      |  6 +-
 15 files changed, 182 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/481d7cd5/data/scripts/q_test_cleanup_compare.sql
----------------------------------------------------------------------
diff --git a/data/scripts/q_test_cleanup_compare.sql b/data/scripts/q_test_cleanup_compare.sql
new file mode 100644
index 0000000..0c6ab14
--- /dev/null
+++ b/data/scripts/q_test_cleanup_compare.sql
@@ -0,0 +1 @@
+DROP TABLE IF EXISTS alltypesorc;

http://git-wip-us.apache.org/repos/asf/hive/blob/481d7cd5/data/scripts/q_test_cleanup_contrib.sql
----------------------------------------------------------------------
diff --git a/data/scripts/q_test_cleanup_contrib.sql b/data/scripts/q_test_cleanup_contrib.sql
new file mode 100644
index 0000000..95d4ba0
--- /dev/null
+++ b/data/scripts/q_test_cleanup_contrib.sql
@@ -0,0 +1,2 @@
+DROP TABLE IF EXISTS src;
+DROP TABLE IF EXISTS src_thrift;

http://git-wip-us.apache.org/repos/asf/hive/blob/481d7cd5/data/scripts/q_test_cleanup_for_encryption.sql
----------------------------------------------------------------------
diff --git a/data/scripts/q_test_cleanup_for_encryption.sql b/data/scripts/q_test_cleanup_for_encryption.sql
deleted file mode 100644
index 070cdbe..0000000
--- a/data/scripts/q_test_cleanup_for_encryption.sql
+++ /dev/null
@@ -1 +0,0 @@
-DROP TABLE IF EXISTS src PURGE;

http://git-wip-us.apache.org/repos/asf/hive/blob/481d7cd5/data/scripts/q_test_cleanup_src.sql
----------------------------------------------------------------------
diff --git a/data/scripts/q_test_cleanup_src.sql b/data/scripts/q_test_cleanup_src.sql
new file mode 100644
index 0000000..08264ee
--- /dev/null
+++ b/data/scripts/q_test_cleanup_src.sql
@@ -0,0 +1 @@
+DROP TABLE IF EXISTS src;

http://git-wip-us.apache.org/repos/asf/hive/blob/481d7cd5/data/scripts/q_test_cleanup_tez.sql
----------------------------------------------------------------------
diff --git a/data/scripts/q_test_cleanup_tez.sql b/data/scripts/q_test_cleanup_tez.sql
new file mode 100644
index 0000000..0e50ef3
--- /dev/null
+++ b/data/scripts/q_test_cleanup_tez.sql
@@ -0,0 +1,4 @@
+DROP TABLE IF EXISTS src;
+DROP TABLE IF EXISTS src1;
+DROP TABLE IF EXISTS srcpart;
+DROP TABLE IF EXISTS alltypesorc;

http://git-wip-us.apache.org/repos/asf/hive/blob/481d7cd5/data/scripts/q_test_init_compare.sql
----------------------------------------------------------------------
diff --git a/data/scripts/q_test_init_compare.sql b/data/scripts/q_test_init_compare.sql
new file mode 100644
index 0000000..c554250
--- /dev/null
+++ b/data/scripts/q_test_init_compare.sql
@@ -0,0 +1,26 @@
+set hive.stats.dbclass=fs;
+--
+-- Table alltypesorc
+--
+DROP TABLE IF EXISTS alltypesorc;
+CREATE TABLE alltypesorc(
+    ctinyint TINYINT,
+    csmallint SMALLINT,
+    cint INT,
+    cbigint BIGINT,
+    cfloat FLOAT,
+    cdouble DOUBLE,
+    cstring1 STRING,
+    cstring2 STRING,
+    ctimestamp1 TIMESTAMP,
+    ctimestamp2 TIMESTAMP,
+    cboolean1 BOOLEAN,
+    cboolean2 BOOLEAN)
+    STORED AS ORC;
+
+LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/alltypesorc"
+OVERWRITE INTO  TABLE alltypesorc;
+
+ANALYZE TABLE alltypesorc COMPUTE STATISTICS;
+
+ANALYZE TABLE alltypesorc COMPUTE STATISTICS FOR COLUMNS ctinyint,csmallint,cint,cbigint,cfloat,cdouble,cstring1,cstring2,ctimestamp1,ctimestamp2,cboolean1,cboolean2;

http://git-wip-us.apache.org/repos/asf/hive/blob/481d7cd5/data/scripts/q_test_init_contrib.sql
----------------------------------------------------------------------
diff --git a/data/scripts/q_test_init_contrib.sql b/data/scripts/q_test_init_contrib.sql
new file mode 100644
index 0000000..b7bb5ab
--- /dev/null
+++ b/data/scripts/q_test_init_contrib.sql
@@ -0,0 +1,29 @@
+set hive.stats.dbclass=fs;
+--
+-- Table src
+--
+DROP TABLE IF EXISTS src;
+
+CREATE TABLE src (key STRING COMMENT 'default', value STRING COMMENT 'default') STORED AS TEXTFILE;
+
+LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/kv1.txt" INTO TABLE src;
+
+ANALYZE TABLE src COMPUTE STATISTICS;
+
+ANALYZE TABLE src COMPUTE STATISTICS FOR COLUMNS key,value;
+
+--
+-- Table src_thrift
+--
+DROP TABLE IF EXISTS src_thrift;
+
+CREATE TABLE src_thrift
+ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer'
+WITH SERDEPROPERTIES (
+  'serialization.class' = 'org.apache.hadoop.hive.serde2.thrift.test.Complex',
+  'serialization.format' = 'org.apache.thrift.protocol.TBinaryProtocol')
+STORED AS SEQUENCEFILE;
+
+LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/complex.seq" INTO TABLE src_thrift;
+
+ANALYZE TABLE src_thrift COMPUTE STATISTICS;

http://git-wip-us.apache.org/repos/asf/hive/blob/481d7cd5/data/scripts/q_test_init_for_encryption.sql
----------------------------------------------------------------------
diff --git a/data/scripts/q_test_init_for_encryption.sql b/data/scripts/q_test_init_for_encryption.sql
deleted file mode 100644
index 56b44e0..0000000
--- a/data/scripts/q_test_init_for_encryption.sql
+++ /dev/null
@@ -1,5 +0,0 @@
-DROP TABLE IF EXISTS src PURGE;
-
-CREATE TABLE src(key STRING COMMENT 'default', value STRING COMMENT 'default') STORED AS TEXTFILE;
-
-LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/kv1.txt" OVERWRITE INTO TABLE src;

http://git-wip-us.apache.org/repos/asf/hive/blob/481d7cd5/data/scripts/q_test_init_src.sql
----------------------------------------------------------------------
diff --git a/data/scripts/q_test_init_src.sql b/data/scripts/q_test_init_src.sql
new file mode 100644
index 0000000..56b44e0
--- /dev/null
+++ b/data/scripts/q_test_init_src.sql
@@ -0,0 +1,5 @@
+DROP TABLE IF EXISTS src PURGE;
+
+CREATE TABLE src(key STRING COMMENT 'default', value STRING COMMENT 'default') STORED AS TEXTFILE;
+
+LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/kv1.txt" OVERWRITE INTO TABLE src;

http://git-wip-us.apache.org/repos/asf/hive/blob/481d7cd5/data/scripts/q_test_init_src_with_stats.sql
----------------------------------------------------------------------
diff --git a/data/scripts/q_test_init_src_with_stats.sql b/data/scripts/q_test_init_src_with_stats.sql
new file mode 100644
index 0000000..765d6c7
--- /dev/null
+++ b/data/scripts/q_test_init_src_with_stats.sql
@@ -0,0 +1,9 @@
+DROP TABLE IF EXISTS src PURGE;
+
+CREATE TABLE src(key STRING COMMENT 'default', value STRING COMMENT 'default') STORED AS TEXTFILE;
+
+LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/kv1.txt" OVERWRITE INTO TABLE src;
+
+ANALYZE TABLE src COMPUTE STATISTICS;
+
+ANALYZE TABLE src COMPUTE STATISTICS FOR COLUMNS key,value;

http://git-wip-us.apache.org/repos/asf/hive/blob/481d7cd5/data/scripts/q_test_init_tez.sql
----------------------------------------------------------------------
diff --git a/data/scripts/q_test_init_tez.sql b/data/scripts/q_test_init_tez.sql
new file mode 100644
index 0000000..4e6176b
--- /dev/null
+++ b/data/scripts/q_test_init_tez.sql
@@ -0,0 +1,78 @@
+set hive.stats.dbclass=fs;
+
+--
+-- Table src
+--
+DROP TABLE IF EXISTS src;
+
+CREATE TABLE src(key STRING COMMENT 'default', value STRING COMMENT 'default') STORED AS TEXTFILE;
+
+LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/kv1.txt" OVERWRITE INTO TABLE src;
+
+ANALYZE TABLE src COMPUTE STATISTICS;
+
+ANALYZE TABLE src COMPUTE STATISTICS FOR COLUMNS key,value;
+
+--
+-- Table src1
+--
+DROP TABLE IF EXISTS src1;
+
+CREATE TABLE src1 (key STRING COMMENT 'default', value STRING COMMENT 'default') STORED AS TEXTFILE;
+
+LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/kv3.txt" INTO TABLE src1;
+
+ANALYZE TABLE src1 COMPUTE STATISTICS;
+
+ANALYZE TABLE src1 COMPUTE STATISTICS FOR COLUMNS key,value;
+
+--
+-- Table srcpart
+--
+DROP TABLE IF EXISTS srcpart;
+
+CREATE TABLE srcpart (key STRING COMMENT 'default', value STRING COMMENT 'default')
+PARTITIONED BY (ds STRING, hr STRING)
+STORED AS TEXTFILE;
+
+LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/kv1.txt"
+OVERWRITE INTO TABLE srcpart PARTITION (ds="2008-04-08", hr="11");
+
+LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/kv1.txt"
+OVERWRITE INTO TABLE srcpart PARTITION (ds="2008-04-08", hr="12");
+
+LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/kv1.txt"
+OVERWRITE INTO TABLE srcpart PARTITION (ds="2008-04-09", hr="11");
+
+LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/kv1.txt"
+OVERWRITE INTO TABLE srcpart PARTITION (ds="2008-04-09", hr="12");
+
+ANALYZE TABLE srcpart PARTITION(ds, hr) COMPUTE STATISTICS;
+
+ANALYZE TABLE srcpart PARTITION(ds, hr) COMPUTE STATISTICS FOR COLUMNS key,value;
+
+--
+-- Table alltypesorc
+--
+DROP TABLE IF EXISTS alltypesorc;
+CREATE TABLE alltypesorc(
+    ctinyint TINYINT,
+    csmallint SMALLINT,
+    cint INT,
+    cbigint BIGINT,
+    cfloat FLOAT,
+    cdouble DOUBLE,
+    cstring1 STRING,
+    cstring2 STRING,
+    ctimestamp1 TIMESTAMP,
+    ctimestamp2 TIMESTAMP,
+    cboolean1 BOOLEAN,
+    cboolean2 BOOLEAN)
+    STORED AS ORC;
+
+LOAD DATA LOCAL INPATH "${hiveconf:test.data.dir}/alltypesorc"
+OVERWRITE INTO  TABLE alltypesorc;
+
+ANALYZE TABLE alltypesorc COMPUTE STATISTICS;
+
+ANALYZE TABLE alltypesorc COMPUTE STATISTICS FOR COLUMNS ctinyint,csmallint,cint,cbigint,cfloat,cdouble,cstring1,cstring2,ctimestamp1,ctimestamp2,cboolean1,cboolean2;

http://git-wip-us.apache.org/repos/asf/hive/blob/481d7cd5/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestMTQueries.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestMTQueries.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestMTQueries.java
index 198fe48..ad2baa2 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestMTQueries.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestMTQueries.java
@@ -36,7 +36,8 @@ public class TestMTQueries extends BaseTestQueries {
     String[] testNames = new String[] {"join2.q", "groupby1.q", "input1.q", "input19.q"};
 
     File[] qfiles = setupQFiles(testNames);
-    QTestUtil[] qts = QTestUtil.queryListRunnerSetup(qfiles, resDir, logDir);
+    QTestUtil[] qts = QTestUtil.queryListRunnerSetup(qfiles, resDir, logDir, "q_test_init_src_with_stats.sql",
+      "q_test_cleanup_src_with_stats.sql");
     for (QTestUtil util : qts) {
       // derby fails creating multiple stats aggregator concurrently
       util.getConf().setBoolean("hive.exec.submitviachild", true);

http://git-wip-us.apache.org/repos/asf/hive/blob/481d7cd5/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java
index 201c3d3..e7748b7 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java
@@ -124,8 +124,8 @@ public class CliConfigs {
         setResultsDir("ql/src/test/results/clientpositive/tez");
         setLogDir("itests/qtest/target/qfile-results/clientpositive");
 
-        setInitScript("q_test_init.sql");
-        setCleanupScript("q_test_cleanup.sql");
+        setInitScript("q_test_init_tez.sql");
+        setCleanupScript("q_test_cleanup_tez.sql");
 
         setHiveConfDir("data/conf/tez");
         setClusterType(MiniClusterType.tez);
@@ -198,8 +198,8 @@ public class CliConfigs {
         setResultsDir("ql/src/test/results/clientpositive/encrypted");
         setLogDir("itests/qtest/target/qfile-results/clientpositive");
 
-        setInitScript("q_test_init_for_encryption.sql");
-        setCleanupScript("q_test_cleanup_for_encryption.sql");
+        setInitScript("q_test_init_src.sql");
+        setCleanupScript("q_test_cleanup_src.sql");
 
 
         setClusterType(MiniClusterType.mr);
@@ -225,8 +225,8 @@ public class CliConfigs {
         setResultsDir("contrib/src/test/results/clientpositive");
         setLogDir("itests/qtest/target/qfile-results/contribclientpositive");
 
-        setInitScript("q_test_init.sql");
-        setCleanupScript("q_test_cleanup.sql");
+        setInitScript("q_test_init_contrib.sql");
+        setCleanupScript("q_test_cleanup_contrib.sql");
 
         setHiveConfDir("");
       } catch (Exception e) {
@@ -268,8 +268,8 @@ public class CliConfigs {
         setResultsDir("ql/src/test/results/clientcompare");
         setLogDir("itests/qtest/target/qfile-results/clientcompare");
 
-        setInitScript("q_test_init.sql");
-        setCleanupScript("q_test_cleanup.sql");
+        setInitScript("q_test_init_compare.sql");
+        setCleanupScript("q_test_cleanup_compare.sql");
 
         setHiveConfDir("");
         setClusterType(MiniClusterType.none);
@@ -313,8 +313,8 @@ public class CliConfigs {
         setResultsDir("ql/src/test/results/clientnegative");
         setLogDir("itests/qtest/target/qfile-results/clientnegative");
 
-        setInitScript("q_test_init_for_encryption.sql");
-        setCleanupScript("q_test_cleanup_for_encryption.sql");
+        setInitScript("q_test_init_src.sql");
+        setCleanupScript("q_test_cleanup_src.sql");
 
         setHiveConfDir("");
         setClusterType(MiniClusterType.mr);
@@ -333,8 +333,8 @@ public class CliConfigs {
         setResultsDir("hbase-handler/src/test/results/positive");
         setLogDir("itests/qtest/target/qfile-results/hbase-handler/positive");
 
-        setInitScript("q_test_init.sql");
-        setCleanupScript("q_test_cleanup.sql");
+        setInitScript("q_test_init_src_with_stats.sql");
+        setCleanupScript("q_test_cleanup_src.sql");
 
         setHiveConfDir("");
         setClusterType(MiniClusterType.none);
@@ -353,8 +353,8 @@ public class CliConfigs {
         setResultsDir("ql/src/test/results/clientcompare");
         setLogDir("itests/qtest/target/qfile-results/clientcompare");
 
-        setInitScript("q_test_init.sql");
-        setCleanupScript("q_test_cleanup.sql");
+        setInitScript("q_test_init_compare.sql");
+        setCleanupScript("q_test_cleanup_compare.sql");
 
         setHiveConfDir("");
         setClusterType(MiniClusterType.none);
@@ -373,8 +373,8 @@ public class CliConfigs {
         setResultsDir("hbase-handler/src/test/results/negative");
         setLogDir("itests/qtest/target/qfile-results/hbase-handler/negative");
 
-        setInitScript("q_test_init.sql");
-        setCleanupScript("q_test_cleanup.sql");
+        setInitScript("q_test_init_src.sql");
+        setCleanupScript("q_test_cleanup_src.sql");
 
         setHiveConfDir("");
         setClusterType(MiniClusterType.none);
@@ -439,8 +439,8 @@ public class CliConfigs {
         setResultsDir("accumulo-handler/src/test/results/positive");
         setLogDir("itests/qtest/target/qfile-results/accumulo-handler/positive");
 
-        setInitScript("q_test_init.sql");
-        setCleanupScript("q_test_cleanup.sql");
+        setInitScript("q_test_init_src_with_stats.sql");
+        setCleanupScript("q_test_cleanup_src.sql");
 
         setHiveConfDir("");
         setClusterType(MiniClusterType.none);

http://git-wip-us.apache.org/repos/asf/hive/blob/481d7cd5/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
index e49ecd9..5f683f0 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
@@ -1965,12 +1965,14 @@ public class QTestUtil {
    * @return one QTestUtil for each query file
    */
   public static QTestUtil[] queryListRunnerSetup(File[] qfiles, String resDir,
-      String logDir) throws Exception
+      String logDir, String initScript, String cleanupScript) throws Exception
   {
     QTestUtil[] qt = new QTestUtil[qfiles.length];
     for (int i = 0; i < qfiles.length; i++) {
       qt[i] = new QTestUtil(resDir, logDir, MiniClusterType.none, null, "0.20",
-          defaultInitScript, defaultCleanupScript, false, false);
+        initScript == null ? defaultInitScript : initScript,
+        cleanupScript == null ? defaultCleanupScript : cleanupScript,
+        false, false);
       qt[i].addFile(qfiles[i]);
       qt[i].clearTestSideEffects();
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/481d7cd5/ql/src/test/results/clientpositive/tez/explainuser_3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/explainuser_3.q.out b/ql/src/test/results/clientpositive/tez/explainuser_3.q.out
index b08f890..5ba02f3 100644
--- a/ql/src/test/results/clientpositive/tez/explainuser_3.q.out
+++ b/ql/src/test/results/clientpositive/tez/explainuser_3.q.out
@@ -31,13 +31,13 @@ Stage-0
     Stage-1
       Reducer 2 vectorized
       File Output Operator [FS_8]
-        Select Operator [SEL_7] (rows=16 width=106)
+        Select Operator [SEL_7] (rows=16 width=107)
           Output:["_col0","_col1"]
         <-Map 1 [SIMPLE_EDGE] vectorized
           SHUFFLE [RS_6]
-            Select Operator [SEL_5] (rows=16 width=106)
+            Select Operator [SEL_5] (rows=16 width=107)
               Output:["_col0","_col1"]
-              TableScan [TS_0] (rows=16 width=106)
+              TableScan [TS_0] (rows=16 width=107)
                 default@acid_vectorized,acid_vectorized, ACID table,Tbl:COMPLETE,Col:NONE,Output:["a","b"]
 
 PREHOOK: query: explain select key, value


[11/22] hive git commit: HIVE-14861: Support precedence for set operator using parentheses (Pengcheng Xiong, reviewed by Ashutosh Chauhan)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/2435e702/ql/src/test/results/clientpositive/tez/explainanalyze_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/explainanalyze_2.q.out b/ql/src/test/results/clientpositive/tez/explainanalyze_2.q.out
index 37fc7d3..0cbddde 100644
--- a/ql/src/test/results/clientpositive/tez/explainanalyze_2.q.out
+++ b/ql/src/test/results/clientpositive/tez/explainanalyze_2.q.out
@@ -380,143 +380,141 @@ POSTHOOK: type: QUERY
 Plan optimized by CBO.
 
 Vertex dependency in root stage
-Reducer 10 <- Map 14 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE)
-Reducer 11 <- Reducer 10 (SIMPLE_EDGE), Reducer 16 (SIMPLE_EDGE)
-Reducer 16 <- Map 15 (SIMPLE_EDGE), Map 17 (SIMPLE_EDGE)
-Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 6 (SIMPLE_EDGE)
-Reducer 3 <- Reducer 11 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
-Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
-Reducer 5 <- Reducer 4 (SIMPLE_EDGE)
-Reducer 8 <- Map 12 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE)
-Reducer 9 <- Map 13 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE)
+Reducer 10 <- Reducer 9 (SIMPLE_EDGE)
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 11 (SIMPLE_EDGE)
+Reducer 3 <- Map 12 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+Reducer 4 <- Map 13 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+Reducer 5 <- Map 14 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
+Reducer 6 <- Map 15 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
+Reducer 7 <- Map 16 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
+Reducer 8 <- Map 17 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE)
+Reducer 9 <- Reducer 8 (SIMPLE_EDGE)
 
 Stage-0
   Fetch Operator
     limit:100
     Stage-1
-      Reducer 5
-      File Output Operator [FS_55]
-        Limit [LIM_54] (rows=24/0 width=285)
+      Reducer 10
+      File Output Operator [FS_53]
+        Limit [LIM_52] (rows=100/0 width=18)
           Number of rows:100
-          Select Operator [SEL_53] (rows=24/0 width=285)
+          Select Operator [SEL_51] (rows=550/0 width=18)
             Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
-          <-Reducer 4 [SIMPLE_EDGE]
-            SHUFFLE [RS_52]
-              Group By Operator [GBY_50] (rows=24/0 width=285)
+          <-Reducer 9 [SIMPLE_EDGE]
+            SHUFFLE [RS_50]
+              Group By Operator [GBY_48] (rows=550/0 width=18)
                 Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["count(VALUE._col0)","count(VALUE._col1)","count(VALUE._col2)"],keys:KEY._col0, KEY._col1, KEY._col2
-              <-Reducer 3 [SIMPLE_EDGE]
-                SHUFFLE [RS_49]
+              <-Reducer 8 [SIMPLE_EDGE]
+                SHUFFLE [RS_47]
                   PartitionCols:_col0, _col1, _col2
-                  Group By Operator [GBY_48] (rows=24/0 width=285)
-                    Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["count(_col13)","count(_col21)","count(_col3)"],keys:_col12, _col20, _col2
-                    Select Operator [SEL_47] (rows=650/0 width=534)
-                      Output:["_col12","_col20","_col2","_col13","_col21","_col3"]
-                      Merge Join Operator [MERGEJOIN_97] (rows=650/0 width=534)
-                        Conds:RS_44._col1, _col3=RS_45._col15, _col17(Inner),Output:["_col2","_col3","_col12","_col13","_col20","_col21"]
-                      <-Reducer 11 [SIMPLE_EDGE]
-                        SHUFFLE [RS_45]
-                          PartitionCols:_col15, _col17
-                          Select Operator [SEL_40] (rows=190/0 width=447)
-                            Output:["_col14","_col15","_col17","_col6","_col7"]
-                            Merge Join Operator [MERGEJOIN_96] (rows=190/0 width=447)
-                              Conds:RS_37._col6, _col4=RS_38._col4, _col2(Inner),Output:["_col2","_col3","_col14","_col15","_col17"]
-                            <-Reducer 10 [SIMPLE_EDGE]
-                              SHUFFLE [RS_37]
-                                PartitionCols:_col6, _col4
-                                Merge Join Operator [MERGEJOIN_94] (rows=40/0 width=352)
-                                  Conds:RS_34._col3=RS_35._col1(Inner),Output:["_col2","_col3","_col4","_col6"]
-                                <-Map 14 [SIMPLE_EDGE]
-                                  SHUFFLE [RS_35]
-                                    PartitionCols:_col1
-                                    Select Operator [SEL_17] (rows=2/0 width=180)
-                                      Output:["_col1"]
-                                      Filter Operator [FIL_88] (rows=2/0 width=175)
-                                        predicate:((key = 'src1key') and value is not null)
-                                        TableScan [TS_15] (rows=25/25 width=175)
-                                          default@src1,src1,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
-                                <-Reducer 9 [SIMPLE_EDGE]
-                                  SHUFFLE [RS_34]
-                                    PartitionCols:_col3
-                                    Merge Join Operator [MERGEJOIN_93] (rows=40/0 width=352)
-                                      Conds:RS_31._col2=RS_32._col0(Inner),Output:["_col2","_col3","_col4","_col6"]
-                                    <-Map 13 [SIMPLE_EDGE]
-                                      SHUFFLE [RS_32]
-                                        PartitionCols:_col0
-                                        Select Operator [SEL_14] (rows=2/0 width=178)
-                                          Output:["_col0"]
-                                          Filter Operator [FIL_87] (rows=2/0 width=178)
-                                            predicate:((value = 'd1value') and key is not null)
-                                            TableScan [TS_12] (rows=500/500 width=178)
-                                              default@src,d1,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
-                                    <-Reducer 8 [SIMPLE_EDGE]
-                                      SHUFFLE [RS_31]
-                                        PartitionCols:_col2
-                                        Merge Join Operator [MERGEJOIN_92] (rows=40/0 width=352)
-                                          Conds:RS_28._col1=RS_29._col3(Inner),Output:["_col2","_col3","_col4","_col6"]
-                                        <-Map 12 [SIMPLE_EDGE]
-                                          SHUFFLE [RS_29]
-                                            PartitionCols:_col3
-                                            Select Operator [SEL_11] (rows=8/0 width=531)
-                                              Output:["_col0","_col1","_col2","_col3","_col4"]
-                                              Filter Operator [FIL_86] (rows=8/0 width=534)
-                                                predicate:((v3 = 'ssv3') and k2 is not null and k3 is not null and k1 is not null and v1 is not null and v2 is not null)
-                                                TableScan [TS_9] (rows=85/85 width=534)
-                                                  default@ss,ss,Tbl:COMPLETE,Col:COMPLETE,Output:["k1","v1","k2","v2","k3","v3"]
-                                        <-Map 7 [SIMPLE_EDGE]
-                                          SHUFFLE [RS_28]
-                                            PartitionCols:_col1
-                                            Select Operator [SEL_8] (rows=10/0 width=185)
-                                              Output:["_col1"]
-                                              Filter Operator [FIL_85] (rows=10/0 width=178)
-                                                predicate:((key = 'srcpartkey') and value is not null)
-                                                TableScan [TS_6] (rows=2000/2000 width=178)
-                                                  default@srcpart,srcpart,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
-                            <-Reducer 16 [SIMPLE_EDGE]
-                              SHUFFLE [RS_38]
-                                PartitionCols:_col4, _col2
-                                Merge Join Operator [MERGEJOIN_95] (rows=19/0 width=356)
-                                  Conds:RS_24._col0=RS_25._col0(Inner),Output:["_col2","_col3","_col4","_col5"]
-                                <-Map 15 [SIMPLE_EDGE]
-                                  SHUFFLE [RS_24]
-                                    PartitionCols:_col0
-                                    Select Operator [SEL_20] (rows=8/0 width=531)
-                                      Output:["_col0","_col2","_col3","_col4","_col5"]
-                                      Filter Operator [FIL_89] (rows=8/0 width=534)
-                                        predicate:((v1 = 'srv1') and k2 is not null and k3 is not null and v2 is not null and v3 is not null and k1 is not null)
-                                        TableScan [TS_18] (rows=85/85 width=534)
-                                          default@sr,sr,Tbl:COMPLETE,Col:COMPLETE,Output:["k1","v1","k2","v2","k3","v3"]
-                                <-Map 17 [SIMPLE_EDGE]
-                                  SHUFFLE [RS_25]
-                                    PartitionCols:_col0
-                                    Select Operator [SEL_23] (rows=500/0 width=178)
-                                      Output:["_col0"]
-                                      Filter Operator [FIL_90] (rows=500/0 width=178)
-                                        predicate:((value) IN ('2000Q1', '2000Q2', '2000Q3') and key is not null)
-                                        TableScan [TS_21] (rows=500/500 width=178)
-                                          default@src,d2,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
-                      <-Reducer 2 [SIMPLE_EDGE]
-                        SHUFFLE [RS_44]
-                          PartitionCols:_col1, _col3
-                          Merge Join Operator [MERGEJOIN_91] (rows=414/0 width=269)
-                            Conds:RS_41._col0=RS_42._col0(Inner),Output:["_col1","_col2","_col3"]
-                          <-Map 1 [SIMPLE_EDGE]
-                            SHUFFLE [RS_41]
-                              PartitionCols:_col0
-                              Select Operator [SEL_2] (rows=170/170 width=356)
-                                Output:["_col0","_col1","_col2","_col3"]
-                                Filter Operator [FIL_83] (rows=170/170 width=356)
-                                  predicate:(v2 is not null and v3 is not null and k1 is not null)
-                                  TableScan [TS_0] (rows=170/170 width=356)
-                                    default@cs,cs,Tbl:COMPLETE,Col:COMPLETE,Output:["k1","v2","k3","v3"]
-                          <-Map 6 [SIMPLE_EDGE]
-                            SHUFFLE [RS_42]
-                              PartitionCols:_col0
-                              Select Operator [SEL_5] (rows=500/0 width=178)
-                                Output:["_col0"]
-                                Filter Operator [FIL_84] (rows=500/0 width=178)
-                                  predicate:((value) IN ('2000Q1', '2000Q2', '2000Q3') and key is not null)
-                                  TableScan [TS_3] (rows=500/500 width=178)
-                                    default@src,d3,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
+                  Group By Operator [GBY_46] (rows=1100/0 width=18)
+                    Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["count(_col1)","count(_col9)","count(_col15)"],keys:_col0, _col8, _col14
+                    Select Operator [SEL_45] (rows=1100/0 width=18)
+                      Output:["_col0","_col8","_col14","_col1","_col9","_col15"]
+                      Merge Join Operator [MERGEJOIN_103] (rows=1100/0 width=18)
+                        Conds:RS_42._col3=RS_43._col1(Inner),Output:["_col0","_col1","_col8","_col9","_col14","_col15"]
+                      <-Map 17 [SIMPLE_EDGE]
+                        SHUFFLE [RS_43]
+                          PartitionCols:_col1
+                          Select Operator [SEL_23] (rows=1000/0 width=18)
+                            Output:["_col1"]
+                            Filter Operator [FIL_96] (rows=1000/0 width=18)
+                              predicate:((key = 'srcpartkey') and value is not null)
+                              TableScan [TS_21] (rows=2000/2000 width=18)
+                                default@srcpart,srcpart,Tbl:COMPLETE,Col:NONE,Output:["key","value"]
+                      <-Reducer 7 [SIMPLE_EDGE]
+                        SHUFFLE [RS_42]
+                          PartitionCols:_col3
+                          Merge Join Operator [MERGEJOIN_102] (rows=129/0 width=625)
+                            Conds:RS_39._col1=RS_40._col1(Inner),Output:["_col0","_col1","_col3","_col8","_col9","_col14","_col15"]
+                          <-Map 16 [SIMPLE_EDGE]
+                            SHUFFLE [RS_40]
+                              PartitionCols:_col1
+                              Select Operator [SEL_20] (rows=2/0 width=180)
+                                Output:["_col1"]
+                                Filter Operator [FIL_95] (rows=2/0 width=175)
+                                  predicate:((key = 'src1key') and value is not null)
+                                  TableScan [TS_18] (rows=25/25 width=175)
+                                    default@src1,src1,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
+                          <-Reducer 6 [SIMPLE_EDGE]
+                            SHUFFLE [RS_39]
+                              PartitionCols:_col1
+                              Merge Join Operator [MERGEJOIN_101] (rows=129/0 width=625)
+                                Conds:RS_36._col12=RS_37._col0(Inner),Output:["_col0","_col1","_col3","_col8","_col9","_col14","_col15"]
+                              <-Map 15 [SIMPLE_EDGE]
+                                SHUFFLE [RS_37]
+                                  PartitionCols:_col0
+                                  Select Operator [SEL_17] (rows=500/0 width=178)
+                                    Output:["_col0"]
+                                    Filter Operator [FIL_94] (rows=500/0 width=178)
+                                      predicate:((value) IN ('2000Q1', '2000Q2', '2000Q3') and key is not null)
+                                      TableScan [TS_15] (rows=500/500 width=178)
+                                        default@src,d3,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
+                              <-Reducer 5 [SIMPLE_EDGE]
+                                SHUFFLE [RS_36]
+                                  PartitionCols:_col12
+                                  Merge Join Operator [MERGEJOIN_100] (rows=53/0 width=712)
+                                    Conds:RS_33._col6=RS_34._col0(Inner),Output:["_col0","_col1","_col3","_col8","_col9","_col12","_col14","_col15"]
+                                  <-Map 14 [SIMPLE_EDGE]
+                                    SHUFFLE [RS_34]
+                                      PartitionCols:_col0
+                                      Select Operator [SEL_14] (rows=500/0 width=178)
+                                        Output:["_col0"]
+                                        Filter Operator [FIL_93] (rows=500/0 width=178)
+                                          predicate:((value) IN ('2000Q1', '2000Q2', '2000Q3') and key is not null)
+                                          TableScan [TS_12] (rows=500/500 width=178)
+                                            default@src,d2,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
+                                  <-Reducer 4 [SIMPLE_EDGE]
+                                    SHUFFLE [RS_33]
+                                      PartitionCols:_col6
+                                      Merge Join Operator [MERGEJOIN_99] (rows=22/0 width=799)
+                                        Conds:RS_30._col0=RS_31._col0(Inner),Output:["_col0","_col1","_col3","_col6","_col8","_col9","_col12","_col14","_col15"]
+                                      <-Map 13 [SIMPLE_EDGE]
+                                        SHUFFLE [RS_31]
+                                          PartitionCols:_col0
+                                          Select Operator [SEL_11] (rows=2/0 width=178)
+                                            Output:["_col0"]
+                                            Filter Operator [FIL_92] (rows=2/0 width=178)
+                                              predicate:((value = 'd1value') and key is not null)
+                                              TableScan [TS_9] (rows=500/500 width=178)
+                                                default@src,d1,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
+                                      <-Reducer 3 [SIMPLE_EDGE]
+                                        SHUFFLE [RS_30]
+                                          PartitionCols:_col0
+                                          Merge Join Operator [MERGEJOIN_98] (rows=22/0 width=799)
+                                            Conds:RS_27._col9, _col11=RS_28._col1, _col3(Inner),Output:["_col0","_col1","_col3","_col6","_col8","_col9","_col12","_col14","_col15"]
+                                          <-Map 12 [SIMPLE_EDGE]
+                                            SHUFFLE [RS_28]
+                                              PartitionCols:_col1, _col3
+                                              Select Operator [SEL_8] (rows=170/170 width=356)
+                                                Output:["_col0","_col1","_col2","_col3"]
+                                                Filter Operator [FIL_91] (rows=170/170 width=356)
+                                                  predicate:(v2 is not null and v3 is not null and k1 is not null)
+                                                  TableScan [TS_6] (rows=170/170 width=356)
+                                                    default@cs,cs,Tbl:COMPLETE,Col:COMPLETE,Output:["k1","v2","k3","v3"]
+                                          <-Reducer 2 [SIMPLE_EDGE]
+                                            SHUFFLE [RS_27]
+                                              PartitionCols:_col9, _col11
+                                              Merge Join Operator [MERGEJOIN_97] (rows=16/0 width=625)
+                                                Conds:RS_24._col2, _col4=RS_25._col2, _col4(Inner),Output:["_col0","_col1","_col3","_col6","_col8","_col9","_col11"]
+                                              <-Map 1 [SIMPLE_EDGE]
+                                                SHUFFLE [RS_24]
+                                                  PartitionCols:_col2, _col4
+                                                  Select Operator [SEL_2] (rows=8/0 width=531)
+                                                    Output:["_col0","_col1","_col2","_col3","_col4"]
+                                                    Filter Operator [FIL_89] (rows=8/0 width=534)
+                                                      predicate:((v3 = 'ssv3') and k2 is not null and k3 is not null and k1 is not null and v1 is not null and v2 is not null)
+                                                      TableScan [TS_0] (rows=85/85 width=534)
+                                                        default@ss,ss,Tbl:COMPLETE,Col:COMPLETE,Output:["k1","v1","k2","v2","k3","v3"]
+                                              <-Map 11 [SIMPLE_EDGE]
+                                                SHUFFLE [RS_25]
+                                                  PartitionCols:_col2, _col4
+                                                  Select Operator [SEL_5] (rows=8/0 width=531)
+                                                    Output:["_col0","_col2","_col3","_col4","_col5"]
+                                                    Filter Operator [FIL_90] (rows=8/0 width=534)
+                                                      predicate:((v1 = 'srv1') and k2 is not null and k3 is not null and v2 is not null and v3 is not null and k1 is not null)
+                                                      TableScan [TS_3] (rows=85/85 width=534)
+                                                        default@sr,sr,Tbl:COMPLETE,Col:COMPLETE,Output:["k1","v1","k2","v2","k3","v3"]
 
 PREHOOK: query: SELECT x.key, z.value, y.value
 FROM src1 x JOIN src y ON (x.key = y.key) 
@@ -3426,11 +3424,15 @@ STAGE PLANS:
                       expressions: value (type: string)
                       outputColumnNames: _col1
                       Statistics: Num rows: 25/25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
-                      Reduce Output Operator
-                        key expressions: _col1 (type: string)
-                        sort order: +
-                        Map-reduce partition columns: _col1 (type: string)
+                      Select Operator
+                        expressions: _col1 (type: string)
+                        outputColumnNames: _col1
                         Statistics: Num rows: 525/525 Data size: 5503 Basic stats: COMPLETE Column stats: NONE
+                        Reduce Output Operator
+                          key expressions: _col1 (type: string)
+                          sort order: +
+                          Map-reduce partition columns: _col1 (type: string)
+                          Statistics: Num rows: 525/525 Data size: 5503 Basic stats: COMPLETE Column stats: NONE
         Map 11 
             Map Operator Tree:
                 TableScan
@@ -3443,11 +3445,15 @@ STAGE PLANS:
                       expressions: value (type: string)
                       outputColumnNames: _col1
                       Statistics: Num rows: 500/500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                      Reduce Output Operator
-                        key expressions: _col1 (type: string)
-                        sort order: +
-                        Map-reduce partition columns: _col1 (type: string)
+                      Select Operator
+                        expressions: _col1 (type: string)
+                        outputColumnNames: _col1
                         Statistics: Num rows: 1025/1025 Data size: 10815 Basic stats: COMPLETE Column stats: NONE
+                        Reduce Output Operator
+                          key expressions: _col1 (type: string)
+                          sort order: +
+                          Map-reduce partition columns: _col1 (type: string)
+                          Statistics: Num rows: 1025/1025 Data size: 10815 Basic stats: COMPLETE Column stats: NONE
         Map 12 
             Map Operator Tree:
                 TableScan
@@ -3460,11 +3466,15 @@ STAGE PLANS:
                       expressions: value (type: string)
                       outputColumnNames: _col1
                       Statistics: Num rows: 500/500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                      Reduce Output Operator
-                        key expressions: _col1 (type: string)
-                        sort order: +
-                        Map-reduce partition columns: _col1 (type: string)
+                      Select Operator
+                        expressions: _col1 (type: string)
+                        outputColumnNames: _col1
                         Statistics: Num rows: 1025/1025 Data size: 10815 Basic stats: COMPLETE Column stats: NONE
+                        Reduce Output Operator
+                          key expressions: _col1 (type: string)
+                          sort order: +
+                          Map-reduce partition columns: _col1 (type: string)
+                          Statistics: Num rows: 1025/1025 Data size: 10815 Basic stats: COMPLETE Column stats: NONE
         Map 13 
             Map Operator Tree:
                 TableScan
@@ -3505,45 +3515,49 @@ STAGE PLANS:
                       expressions: value (type: string)
                       outputColumnNames: _col1
                       Statistics: Num rows: 25/25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
-                      Map Join Operator
-                        condition map:
-                             Inner Join 0 to 1
-                        keys:
-                          0 _col1 (type: string)
-                          1 _col1 (type: string)
-                        outputColumnNames: _col0, _col6
-                        input vertices:
-                          0 Map 20
-                        Statistics: Num rows: 1677/820 Data size: 17739 Basic stats: COMPLETE Column stats: NONE
-                        HybridGraceHashJoin: true
-                        Select Operator
-                          expressions: _col0 (type: string), _col6 (type: string)
-                          outputColumnNames: _col0, _col1
+                      Select Operator
+                        expressions: _col1 (type: string)
+                        outputColumnNames: _col1
+                        Statistics: Num rows: 1525/1525 Data size: 16127 Basic stats: COMPLETE Column stats: NONE
+                        Map Join Operator
+                          condition map:
+                               Inner Join 0 to 1
+                          keys:
+                            0 _col1 (type: string)
+                            1 _col1 (type: string)
+                          outputColumnNames: _col0, _col6
+                          input vertices:
+                            0 Map 20
                           Statistics: Num rows: 1677/820 Data size: 17739 Basic stats: COMPLETE Column stats: NONE
-                          File Output Operator
-                            compressed: false
-                            Statistics: Num rows: 3409/6411 Data size: 36062 Basic stats: COMPLETE Column stats: NONE
-                            table:
-                                input format: org.apache.hadoop.mapred.TextInputFormat
-                                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                                name: default.a
-                          File Output Operator
-                            compressed: false
-                            Statistics: Num rows: 3409/6411 Data size: 36062 Basic stats: COMPLETE Column stats: NONE
-                            table:
-                                input format: org.apache.hadoop.mapred.TextInputFormat
-                                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                                name: default.b
-                          File Output Operator
-                            compressed: false
-                            Statistics: Num rows: 3409/6411 Data size: 36062 Basic stats: COMPLETE Column stats: NONE
-                            table:
-                                input format: org.apache.hadoop.mapred.TextInputFormat
-                                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                                name: default.c
+                          HybridGraceHashJoin: true
+                          Select Operator
+                            expressions: _col0 (type: string), _col6 (type: string)
+                            outputColumnNames: _col0, _col1
+                            Statistics: Num rows: 1677/820 Data size: 17739 Basic stats: COMPLETE Column stats: NONE
+                            File Output Operator
+                              compressed: false
+                              Statistics: Num rows: 3409/6411 Data size: 36062 Basic stats: COMPLETE Column stats: NONE
+                              table:
+                                  input format: org.apache.hadoop.mapred.TextInputFormat
+                                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                                  name: default.a
+                            File Output Operator
+                              compressed: false
+                              Statistics: Num rows: 3409/6411 Data size: 36062 Basic stats: COMPLETE Column stats: NONE
+                              table:
+                                  input format: org.apache.hadoop.mapred.TextInputFormat
+                                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                                  name: default.b
+                            File Output Operator
+                              compressed: false
+                              Statistics: Num rows: 3409/6411 Data size: 36062 Basic stats: COMPLETE Column stats: NONE
+                              table:
+                                  input format: org.apache.hadoop.mapred.TextInputFormat
+                                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                                  name: default.c
         Map 17 
             Map Operator Tree:
                 TableScan
@@ -3556,45 +3570,49 @@ STAGE PLANS:
                       expressions: value (type: string)
                       outputColumnNames: _col1
                       Statistics: Num rows: 500/500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                      Map Join Operator
-                        condition map:
-                             Inner Join 0 to 1
-                        keys:
-                          0 _col1 (type: string)
-                          1 _col1 (type: string)
-                        outputColumnNames: _col0, _col6
-                        input vertices:
-                          0 Map 20
-                        Statistics: Num rows: 1677/820 Data size: 17739 Basic stats: COMPLETE Column stats: NONE
-                        HybridGraceHashJoin: true
-                        Select Operator
-                          expressions: _col0 (type: string), _col6 (type: string)
-                          outputColumnNames: _col0, _col1
+                      Select Operator
+                        expressions: _col1 (type: string)
+                        outputColumnNames: _col1
+                        Statistics: Num rows: 1525/1525 Data size: 16127 Basic stats: COMPLETE Column stats: NONE
+                        Map Join Operator
+                          condition map:
+                               Inner Join 0 to 1
+                          keys:
+                            0 _col1 (type: string)
+                            1 _col1 (type: string)
+                          outputColumnNames: _col0, _col6
+                          input vertices:
+                            0 Map 20
                           Statistics: Num rows: 1677/820 Data size: 17739 Basic stats: COMPLETE Column stats: NONE
-                          File Output Operator
-                            compressed: false
-                            Statistics: Num rows: 3409/6411 Data size: 36062 Basic stats: COMPLETE Column stats: NONE
-                            table:
-                                input format: org.apache.hadoop.mapred.TextInputFormat
-                                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                                name: default.a
-                          File Output Operator
-                            compressed: false
-                            Statistics: Num rows: 3409/6411 Data size: 36062 Basic stats: COMPLETE Column stats: NONE
-                            table:
-                                input format: org.apache.hadoop.mapred.TextInputFormat
-                                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                                name: default.b
-                          File Output Operator
-                            compressed: false
-                            Statistics: Num rows: 3409/6411 Data size: 36062 Basic stats: COMPLETE Column stats: NONE
-                            table:
-                                input format: org.apache.hadoop.mapred.TextInputFormat
-                                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                                name: default.c
+                          HybridGraceHashJoin: true
+                          Select Operator
+                            expressions: _col0 (type: string), _col6 (type: string)
+                            outputColumnNames: _col0, _col1
+                            Statistics: Num rows: 1677/820 Data size: 17739 Basic stats: COMPLETE Column stats: NONE
+                            File Output Operator
+                              compressed: false
+                              Statistics: Num rows: 3409/6411 Data size: 36062 Basic stats: COMPLETE Column stats: NONE
+                              table:
+                                  input format: org.apache.hadoop.mapred.TextInputFormat
+                                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                                  name: default.a
+                            File Output Operator
+                              compressed: false
+                              Statistics: Num rows: 3409/6411 Data size: 36062 Basic stats: COMPLETE Column stats: NONE
+                              table:
+                                  input format: org.apache.hadoop.mapred.TextInputFormat
+                                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                                  name: default.b
+                            File Output Operator
+                              compressed: false
+                              Statistics: Num rows: 3409/6411 Data size: 36062 Basic stats: COMPLETE Column stats: NONE
+                              table:
+                                  input format: org.apache.hadoop.mapred.TextInputFormat
+                                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                                  name: default.c
         Map 18 
             Map Operator Tree:
                 TableScan
@@ -3607,45 +3625,49 @@ STAGE PLANS:
                       expressions: value (type: string)
                       outputColumnNames: _col1
                       Statistics: Num rows: 500/500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                      Map Join Operator
-                        condition map:
-                             Inner Join 0 to 1
-                        keys:
-                          0 _col1 (type: string)
-                          1 _col1 (type: string)
-                        outputColumnNames: _col0, _col6
-                        input vertices:
-                          0 Map 20
-                        Statistics: Num rows: 1677/820 Data size: 17739 Basic stats: COMPLETE Column stats: NONE
-                        HybridGraceHashJoin: true
-                        Select Operator
-                          expressions: _col0 (type: string), _col6 (type: string)
-                          outputColumnNames: _col0, _col1
+                      Select Operator
+                        expressions: _col1 (type: string)
+                        outputColumnNames: _col1
+                        Statistics: Num rows: 1525/1525 Data size: 16127 Basic stats: COMPLETE Column stats: NONE
+                        Map Join Operator
+                          condition map:
+                               Inner Join 0 to 1
+                          keys:
+                            0 _col1 (type: string)
+                            1 _col1 (type: string)
+                          outputColumnNames: _col0, _col6
+                          input vertices:
+                            0 Map 20
                           Statistics: Num rows: 1677/820 Data size: 17739 Basic stats: COMPLETE Column stats: NONE
-                          File Output Operator
-                            compressed: false
-                            Statistics: Num rows: 3409/6411 Data size: 36062 Basic stats: COMPLETE Column stats: NONE
-                            table:
-                                input format: org.apache.hadoop.mapred.TextInputFormat
-                                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                                name: default.a
-                          File Output Operator
-                            compressed: false
-                            Statistics: Num rows: 3409/6411 Data size: 36062 Basic stats: COMPLETE Column stats: NONE
-                            table:
-                                input format: org.apache.hadoop.mapred.TextInputFormat
-                                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                                name: default.b
-                          File Output Operator
-                            compressed: false
-                            Statistics: Num rows: 3409/6411 Data size: 36062 Basic stats: COMPLETE Column stats: NONE
-                            table:
-                                input format: org.apache.hadoop.mapred.TextInputFormat
-                                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                                name: default.c
+                          HybridGraceHashJoin: true
+                          Select Operator
+                            expressions: _col0 (type: string), _col6 (type: string)
+                            outputColumnNames: _col0, _col1
+                            Statistics: Num rows: 1677/820 Data size: 17739 Basic stats: COMPLETE Column stats: NONE
+                            File Output Operator
+                              compressed: false
+                              Statistics: Num rows: 3409/6411 Data size: 36062 Basic stats: COMPLETE Column stats: NONE
+                              table:
+                                  input format: org.apache.hadoop.mapred.TextInputFormat
+                                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                                  name: default.a
+                            File Output Operator
+                              compressed: false
+                              Statistics: Num rows: 3409/6411 Data size: 36062 Basic stats: COMPLETE Column stats: NONE
+                              table:
+                                  input format: org.apache.hadoop.mapred.TextInputFormat
+                                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                                  name: default.b
+                            File Output Operator
+                              compressed: false
+                              Statistics: Num rows: 3409/6411 Data size: 36062 Basic stats: COMPLETE Column stats: NONE
+                              table:
+                                  input format: org.apache.hadoop.mapred.TextInputFormat
+                                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                                  name: default.c
         Map 19 
             Map Operator Tree:
                 TableScan
@@ -3658,45 +3680,49 @@ STAGE PLANS:
                       expressions: value (type: string)
                       outputColumnNames: _col1
                       Statistics: Num rows: 500/500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                      Map Join Operator
-                        condition map:
-                             Inner Join 0 to 1
-                        keys:
-                          0 _col1 (type: string)
-                          1 _col1 (type: string)
-                        outputColumnNames: _col0, _col6
-                        input vertices:
-                          0 Map 20
-                        Statistics: Num rows: 1677/820 Data size: 17739 Basic stats: COMPLETE Column stats: NONE
-                        HybridGraceHashJoin: true
-                        Select Operator
-                          expressions: _col0 (type: string), _col6 (type: string)
-                          outputColumnNames: _col0, _col1
+                      Select Operator
+                        expressions: _col1 (type: string)
+                        outputColumnNames: _col1
+                        Statistics: Num rows: 1525/1525 Data size: 16127 Basic stats: COMPLETE Column stats: NONE
+                        Map Join Operator
+                          condition map:
+                               Inner Join 0 to 1
+                          keys:
+                            0 _col1 (type: string)
+                            1 _col1 (type: string)
+                          outputColumnNames: _col0, _col6
+                          input vertices:
+                            0 Map 20
                           Statistics: Num rows: 1677/820 Data size: 17739 Basic stats: COMPLETE Column stats: NONE
-                          File Output Operator
-                            compressed: false
-                            Statistics: Num rows: 3409/6411 Data size: 36062 Basic stats: COMPLETE Column stats: NONE
-                            table:
-                                input format: org.apache.hadoop.mapred.TextInputFormat
-                                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                                name: default.a
-                          File Output Operator
-                            compressed: false
-                            Statistics: Num rows: 3409/6411 Data size: 36062 Basic stats: COMPLETE Column stats: NONE
-                            table:
-                                input format: org.apache.hadoop.mapred.TextInputFormat
-                                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                                name: default.b
-                          File Output Operator
-                            compressed: false
-                            Statistics: Num rows: 3409/6411 Data size: 36062 Basic stats: COMPLETE Column stats: NONE
-                            table:
-                                input format: org.apache.hadoop.mapred.TextInputFormat
-                                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                                name: default.c
+                          HybridGraceHashJoin: true
+                          Select Operator
+                            expressions: _col0 (type: string), _col6 (type: string)
+                            outputColumnNames: _col0, _col1
+                            Statistics: Num rows: 1677/820 Data size: 17739 Basic stats: COMPLETE Column stats: NONE
+                            File Output Operator
+                              compressed: false
+                              Statistics: Num rows: 3409/6411 Data size: 36062 Basic stats: COMPLETE Column stats: NONE
+                              table:
+                                  input format: org.apache.hadoop.mapred.TextInputFormat
+                                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                                  name: default.a
+                            File Output Operator
+                              compressed: false
+                              Statistics: Num rows: 3409/6411 Data size: 36062 Basic stats: COMPLETE Column stats: NONE
+                              table:
+                                  input format: org.apache.hadoop.mapred.TextInputFormat
+                                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                                  name: default.b
+                            File Output Operator
+                              compressed: false
+                              Statistics: Num rows: 3409/6411 Data size: 36062 Basic stats: COMPLETE Column stats: NONE
+                              table:
+                                  input format: org.apache.hadoop.mapred.TextInputFormat
+                                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                                  name: default.c
         Map 20 
             Map Operator Tree:
                 TableScan
@@ -3766,11 +3792,15 @@ STAGE PLANS:
                       expressions: value (type: string)
                       outputColumnNames: _col1
                       Statistics: Num rows: 500/500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                      Reduce Output Operator
-                        key expressions: _col1 (type: string)
-                        sort order: +
-                        Map-reduce partition columns: _col1 (type: string)
+                      Select Operator
+                        expressions: _col1 (type: string)
+                        outputColumnNames: _col1
                         Statistics: Num rows: 525/525 Data size: 5503 Basic stats: COMPLETE Column stats: NONE
+                        Reduce Output Operator
+                          key expressions: _col1 (type: string)
+                          sort order: +
+                          Map-reduce partition columns: _col1 (type: string)
+                          Statistics: Num rows: 525/525 Data size: 5503 Basic stats: COMPLETE Column stats: NONE
         Map 6 
             Map Operator Tree:
                 TableScan
@@ -3822,11 +3852,15 @@ STAGE PLANS:
                       expressions: value (type: string)
                       outputColumnNames: _col1
                       Statistics: Num rows: 25/25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
-                      Reduce Output Operator
-                        key expressions: _col1 (type: string)
-                        sort order: +
-                        Map-reduce partition columns: _col1 (type: string)
+                      Select Operator
+                        expressions: _col1 (type: string)
+                        outputColumnNames: _col1
                         Statistics: Num rows: 1025/1025 Data size: 10815 Basic stats: COMPLETE Column stats: NONE
+                        Reduce Output Operator
+                          key expressions: _col1 (type: string)
+                          sort order: +
+                          Map-reduce partition columns: _col1 (type: string)
+                          Statistics: Num rows: 1025/1025 Data size: 10815 Basic stats: COMPLETE Column stats: NONE
         Reducer 10 
             Reduce Operator Tree:
               Merge Join Operator
@@ -4632,15 +4666,15 @@ Stage-4
           Dependency Collection{}
             Stage-2
               Reducer 4
-              File Output Operator [FS_18]
+              File Output Operator [FS_20]
                 table:{"name:":"default.dest1"}
-                Group By Operator [GBY_16] (rows=1/310 width=96)
+                Group By Operator [GBY_18] (rows=1/310 width=96)
                   Output:["_col0","_col1"],aggregations:["count(DISTINCT KEY._col1:0._col0)"],keys:KEY._col0
                 <-Union 3 [SIMPLE_EDGE]
                   <-Map 6 [CONTAINS]
-                    Reduce Output Operator [RS_15]
+                    Reduce Output Operator [RS_17]
                       PartitionCols:_col0
-                      Group By Operator [GBY_14] (rows=1/619 width=280)
+                      Group By Operator [GBY_16] (rows=1/619 width=280)
                         Output:["_col0","_col1","_col2"],aggregations:["count(DISTINCT substr(_col1, 5))"],keys:_col0, substr(_col1, 5)
                         Select Operator [SEL_9] (rows=501/501 width=272)
                           Output:["_col0","_col1"]
@@ -4648,29 +4682,29 @@ Stage-4
                             Output:["_col0","_col1"]
                             TableScan [TS_6] (rows=500/500 width=10)
                               Output:["key","value"]
-                    Reduce Output Operator [RS_21]
+                    Reduce Output Operator [RS_23]
                       PartitionCols:_col0, _col1
-                      Group By Operator [GBY_20] (rows=1/619 width=464)
+                      Group By Operator [GBY_22] (rows=1/619 width=464)
                         Output:["_col0","_col1","_col2","_col3"],aggregations:["count(DISTINCT substr(_col1, 5))"],keys:_col0, _col1, substr(_col1, 5)
                          Please refer to the previous Select Operator [SEL_9]
                   <-Map 7 [CONTAINS]
-                    Reduce Output Operator [RS_15]
+                    Reduce Output Operator [RS_17]
                       PartitionCols:_col0
-                      Group By Operator [GBY_14] (rows=1/619 width=280)
+                      Group By Operator [GBY_16] (rows=1/619 width=280)
                         Output:["_col0","_col1","_col2"],aggregations:["count(DISTINCT substr(_col1, 5))"],keys:_col0, substr(_col1, 5)
-                        Select Operator [SEL_11] (rows=500/500 width=10)
+                        Select Operator [SEL_12] (rows=500/500 width=10)
                           Output:["_col0","_col1"]
-                          TableScan [TS_10] (rows=500/500 width=10)
+                          TableScan [TS_11] (rows=500/500 width=10)
                             Output:["key","value"]
-                    Reduce Output Operator [RS_21]
+                    Reduce Output Operator [RS_23]
                       PartitionCols:_col0, _col1
-                      Group By Operator [GBY_20] (rows=1/619 width=464)
+                      Group By Operator [GBY_22] (rows=1/619 width=464)
                         Output:["_col0","_col1","_col2","_col3"],aggregations:["count(DISTINCT substr(_col1, 5))"],keys:_col0, _col1, substr(_col1, 5)
-                         Please refer to the previous Select Operator [SEL_11]
+                         Please refer to the previous Select Operator [SEL_12]
                   <-Reducer 2 [CONTAINS]
-                    Reduce Output Operator [RS_15]
+                    Reduce Output Operator [RS_17]
                       PartitionCols:_col0
-                      Group By Operator [GBY_14] (rows=1/619 width=280)
+                      Group By Operator [GBY_16] (rows=1/619 width=280)
                         Output:["_col0","_col1","_col2"],aggregations:["count(DISTINCT substr(_col1, 5))"],keys:_col0, substr(_col1, 5)
                         Select Operator [SEL_9] (rows=501/501 width=272)
                           Output:["_col0","_col1"]
@@ -4685,15 +4719,15 @@ Stage-4
                                   Select Operator [SEL_1] (rows=500/500 width=10)
                                     TableScan [TS_0] (rows=500/500 width=10)
                                       default@src,s1,Tbl:COMPLETE,Col:COMPLETE
-                    Reduce Output Operator [RS_21]
+                    Reduce Output Operator [RS_23]
                       PartitionCols:_col0, _col1
-                      Group By Operator [GBY_20] (rows=1/619 width=464)
+                      Group By Operator [GBY_22] (rows=1/619 width=464)
                         Output:["_col0","_col1","_col2","_col3"],aggregations:["count(DISTINCT substr(_col1, 5))"],keys:_col0, _col1, substr(_col1, 5)
                          Please refer to the previous Select Operator [SEL_9]
               Reducer 5
-              File Output Operator [FS_24]
+              File Output Operator [FS_26]
                 table:{"name:":"default.dest2"}
-                Group By Operator [GBY_22] (rows=1/310 width=280)
+                Group By Operator [GBY_24] (rows=1/310 width=280)
                   Output:["_col0","_col1","_col2"],aggregations:["count(DISTINCT KEY._col2:0._col0)"],keys:KEY._col0, KEY._col1
                 <- Please refer to the previous Union 3 [SIMPLE_EDGE]
 Stage-5
@@ -4764,15 +4798,15 @@ Stage-4
           Dependency Collection{}
             Stage-2
               Reducer 4
-              File Output Operator [FS_14]
+              File Output Operator [FS_15]
                 table:{"name:":"default.dest1"}
-                Group By Operator [GBY_12] (rows=1/310 width=96)
+                Group By Operator [GBY_13] (rows=1/310 width=96)
                   Output:["_col0","_col1"],aggregations:["count(DISTINCT KEY._col1:0._col0)"],keys:KEY._col0
                 <-Union 3 [SIMPLE_EDGE]
                   <-Map 6 [CONTAINS]
-                    Reduce Output Operator [RS_11]
+                    Reduce Output Operator [RS_12]
                       PartitionCols:_col0
-                      Group By Operator [GBY_10] (rows=1/310 width=280)
+                      Group By Operator [GBY_11] (rows=1/310 width=280)
                         Output:["_col0","_col1","_col2"],aggregations:["count(DISTINCT substr(_col1, 5))"],keys:_col0, substr(_col1, 5)
                         Select Operator [SEL_9] (rows=501/501 width=11)
                           Output:["_col0","_col1"]
@@ -4780,17 +4814,15 @@ Stage-4
                             Output:["_col0","_col1"]
                             TableScan [TS_6] (rows=500/500 width=10)
                               Output:["key","value"]
-                    Reduce Output Operator [RS_17]
+                    Reduce Output Operator [RS_18]
                       PartitionCols:_col0, _col1
-                      Group By Operator [GBY_16] (rows=1/310 width=464)
+                      Group By Operator [GBY_17] (rows=1/310 width=464)
                         Output:["_col0","_col1","_col2","_col3"],aggregations:["count(DISTINCT substr(_col1, 5))"],keys:_col0, _col1, substr(_col1, 5)
-                        Select Operator [SEL_15] (rows=501/501 width=11)
-                          Output:["_col0","_col1"]
-                           Please refer to the previous Select Operator [SEL_7]
+                         Please refer to the previous Select Operator [SEL_9]
                   <-Reducer 2 [CONTAINS]
-                    Reduce Output Operator [RS_11]
+                    Reduce Output Operator [RS_12]
                       PartitionCols:_col0
-                      Group By Operator [GBY_10] (rows=1/310 width=280)
+                      Group By Operator [GBY_11] (rows=1/310 width=280)
                         Output:["_col0","_col1","_col2"],aggregations:["count(DISTINCT substr(_col1, 5))"],keys:_col0, substr(_col1, 5)
                         Select Operator [SEL_9] (rows=501/501 width=11)
                           Output:["_col0","_col1"]
@@ -4805,17 +4837,15 @@ Stage-4
                                   Select Operator [SEL_1] (rows=500/500 width=10)
                                     TableScan [TS_0] (rows=500/500 width=10)
                                       default@src,s1,Tbl:COMPLETE,Col:COMPLETE
-                    Reduce Output Operator [RS_17]
+                    Reduce Output Operator [RS_18]
                       PartitionCols:_col0, _col1
-                      Group By Operator [GBY_16] (rows=1/310 width=464)
+                      Group By Operator [GBY_17] (rows=1/310 width=464)
                         Output:["_col0","_col1","_col2","_col3"],aggregations:["count(DISTINCT substr(_col1, 5))"],keys:_col0, _col1, substr(_col1, 5)
-                        Select Operator [SEL_15] (rows=501/501 width=11)
-                          Output:["_col0","_col1"]
-                           Please refer to the previous Select Operator [SEL_5]
+                         Please refer to the previous Select Operator [SEL_9]
               Reducer 5
-              File Output Operator [FS_20]
+              File Output Operator [FS_21]
                 table:{"name:":"default.dest2"}
-                Group By Operator [GBY_18] (rows=1/310 width=280)
+                Group By Operator [GBY_19] (rows=1/310 width=280)
                   Output:["_col0","_col1","_col2"],aggregations:["count(DISTINCT KEY._col2:0._col0)"],keys:KEY._col0, KEY._col1
                 <- Please refer to the previous Union 3 [SIMPLE_EDGE]
 Stage-5

http://git-wip-us.apache.org/repos/asf/hive/blob/2435e702/ql/src/test/results/clientpositive/union_offcbo.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/union_offcbo.q.out b/ql/src/test/results/clientpositive/union_offcbo.q.out
index 2c4c0de..cb6a4a5 100644
--- a/ql/src/test/results/clientpositive/union_offcbo.q.out
+++ b/ql/src/test/results/clientpositive/union_offcbo.q.out
@@ -985,37 +985,45 @@ STAGE PLANS:
           TableScan
             Union
               Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE
-              Filter Operator
-                predicate: (_col7 <> 'RET') (type: boolean)
-                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
-                Select Operator
-                  expressions: _col0 (type: bigint), _col1 (type: string), _col2 (type: bigint), CASE WHEN ((_col7 = 'A_INS')) THEN ('2015-11-21') ELSE ('2015-11-20') END (type: string), _col4 (type: string), _col5 (type: string), _col6 (type: string), _col7 (type: string)
-                  outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
+              Select Operator
+                expressions: _col0 (type: bigint), _col1 (type: string), _col2 (type: bigint), _col4 (type: string), _col5 (type: string), _col6 (type: string), _col7 (type: string)
+                outputColumnNames: _col0, _col1, _col2, _col4, _col5, _col6, _col7
+                Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                Filter Operator
+                  predicate: (_col7 <> 'RET') (type: boolean)
                   Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
-                  File Output Operator
-                    compressed: false
+                  Select Operator
+                    expressions: _col0 (type: bigint), _col1 (type: string), _col2 (type: bigint), CASE WHEN ((_col7 = 'A_INS')) THEN ('2015-11-21') ELSE ('2015-11-20') END (type: string), _col4 (type: string), _col5 (type: string), _col6 (type: string), _col7 (type: string)
+                    outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
-                    table:
-                        input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                        output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                    File Output Operator
+                      compressed: false
+                      Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                      table:
+                          input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                          output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                          serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
           TableScan
             Union
               Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE
-              Filter Operator
-                predicate: (_col7 <> 'RET') (type: boolean)
-                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
-                Select Operator
-                  expressions: _col0 (type: bigint), _col1 (type: string), _col2 (type: bigint), CASE WHEN ((_col7 = 'A_INS')) THEN ('2015-11-21') ELSE ('2015-11-20') END (type: string), _col4 (type: string), _col5 (type: string), _col6 (type: string), _col7 (type: string)
-                  outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
+              Select Operator
+                expressions: _col0 (type: bigint), _col1 (type: string), _col2 (type: bigint), _col4 (type: string), _col5 (type: string), _col6 (type: string), _col7 (type: string)
+                outputColumnNames: _col0, _col1, _col2, _col4, _col5, _col6, _col7
+                Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                Filter Operator
+                  predicate: (_col7 <> 'RET') (type: boolean)
                   Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
-                  File Output Operator
-                    compressed: false
+                  Select Operator
+                    expressions: _col0 (type: bigint), _col1 (type: string), _col2 (type: bigint), CASE WHEN ((_col7 = 'A_INS')) THEN ('2015-11-21') ELSE ('2015-11-20') END (type: string), _col4 (type: string), _col5 (type: string), _col6 (type: string), _col7 (type: string)
+                    outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
-                    table:
-                        input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                        output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                    File Output Operator
+                      compressed: false
+                      Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                      table:
+                          input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                          output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                          serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
 
   Stage: Stage-4
     Map Reduce
@@ -1328,37 +1336,45 @@ STAGE PLANS:
           TableScan
             Union
               Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE
-              Filter Operator
-                predicate: (_col7 <> 'RET') (type: boolean)
-                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
-                Select Operator
-                  expressions: _col0 (type: bigint), _col1 (type: string), _col2 (type: bigint), CASE WHEN ((_col7 = 'A_INS')) THEN ('2015-11-21') ELSE ('2015-11-20') END (type: string), _col4 (type: string), _col5 (type: string), _col6 (type: string), _col7 (type: string)
-                  outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
+              Select Operator
+                expressions: _col0 (type: bigint), _col1 (type: string), _col2 (type: bigint), _col4 (type: string), _col5 (type: string), _col6 (type: string), _col7 (type: string)
+                outputColumnNames: _col0, _col1, _col2, _col4, _col5, _col6, _col7
+                Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                Filter Operator
+                  predicate: (_col7 <> 'RET') (type: boolean)
                   Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
-                  File Output Operator
-                    compressed: false
+                  Select Operator
+                    expressions: _col0 (type: bigint), _col1 (type: string), _col2 (type: bigint), CASE WHEN ((_col7 = 'A_INS')) THEN ('2015-11-21') ELSE ('2015-11-20') END (type: string), _col4 (type: string), _col5 (type: string), _col6 (type: string), _col7 (type: string)
+                    outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
-                    table:
-                        input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                        output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                    File Output Operator
+                      compressed: false
+                      Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                      table:
+                          input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                          output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                          serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
           TableScan
             Union
               Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE
-              Filter Operator
-                predicate: (_col7 <> 'RET') (type: boolean)
-                Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
-                Select Operator
-                  expressions: _col0 (type: bigint), _col1 (type: string), _col2 (type: bigint), CASE WHEN ((_col7 = 'A_INS')) THEN ('2015-11-21') ELSE ('2015-11-20') END (type: string), _col4 (type: string), _col5 (type: string), _col6 (type: string), _col7 (type: string)
-                  outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
+              Select Operator
+                expressions: _col0 (type: bigint), _col1 (type: string), _col2 (type: bigint), _col4 (type: string), _col5 (type: string), _col6 (type: string), _col7 (type: string)
+                outputColumnNames: _col0, _col1, _col2, _col4, _col5, _col6, _col7
+                Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                Filter Operator
+                  predicate: (_col7 <> 'RET') (type: boolean)
                   Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
-                  File Output Operator
-                    compressed: false
+                  Select Operator
+                    expressions: _col0 (type: bigint), _col1 (type: string), _col2 (type: bigint), CASE WHEN ((_col7 = 'A_INS')) THEN ('2015-11-21') ELSE ('2015-11-20') END (type: string), _col4 (type: string), _col5 (type: string), _col6 (type: string), _col7 (type: string)
+                    outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
                     Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
-                    table:
-                        input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                        output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                    File Output Operator
+                      compressed: false
+                      Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+                      table:
+                          input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                          output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                          serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
 
   Stage: Stage-4
     Map Reduce

http://git-wip-us.apache.org/repos/asf/hive/blob/2435e702/ql/src/test/results/clientpositive/union_paren.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/union_paren.q.out b/ql/src/test/results/clientpositive/union_paren.q.out
new file mode 100644
index 0000000..2c96092
--- /dev/null
+++ b/ql/src/test/results/clientpositive/union_paren.q.out
@@ -0,0 +1,260 @@
+PREHOOK: query: explain select * from src union all select * from src
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select * from src union all select * from src
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: src
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+              Union
+                Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+          TableScan
+            alias: src
+            Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: string), value (type: string)
+              outputColumnNames: _col0, _col1
+              Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+              Union
+                Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: create table t1(c int)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@t1
+POSTHOOK: query: create table t1(c int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@t1
+PREHOOK: query: insert into t1 values (1),(1),(2)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__1
+PREHOOK: Output: default@t1
+POSTHOOK: query: insert into t1 values (1),(1),(2)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__1
+POSTHOOK: Output: default@t1
+POSTHOOK: Lineage: t1.c EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+PREHOOK: query: create table t2(c int)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@t2
+POSTHOOK: query: create table t2(c int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@t2
+PREHOOK: query: insert into t2 values (2),(1),(2)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__2
+PREHOOK: Output: default@t2
+POSTHOOK: query: insert into t2 values (2),(1),(2)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__2
+POSTHOOK: Output: default@t2
+POSTHOOK: Lineage: t2.c EXPRESSION [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+PREHOOK: query: create table t3(c int)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@t3
+POSTHOOK: query: create table t3(c int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@t3
+PREHOOK: query: insert into t3 values (2),(3),(2)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__3
+PREHOOK: Output: default@t3
+POSTHOOK: query: insert into t3 values (2),(3),(2)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__3
+POSTHOOK: Output: default@t3
+POSTHOOK: Lineage: t3.c EXPRESSION [(values__tmp__table__3)values__tmp__table__3.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+PREHOOK: query: (select * from t1) union all select * from t2 union select * from t3 order by c
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1
+PREHOOK: Input: default@t2
+PREHOOK: Input: default@t3
+#### A masked pattern was here ####
+POSTHOOK: query: (select * from t1) union all select * from t2 union select * from t3 order by c
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1
+POSTHOOK: Input: default@t2
+POSTHOOK: Input: default@t3
+#### A masked pattern was here ####
+1
+2
+3
+PREHOOK: query: (select * from t1) union all (select * from t2 union select * from t3) order by c
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1
+PREHOOK: Input: default@t2
+PREHOOK: Input: default@t3
+#### A masked pattern was here ####
+POSTHOOK: query: (select * from t1) union all (select * from t2 union select * from t3) order by c
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1
+POSTHOOK: Input: default@t2
+POSTHOOK: Input: default@t3
+#### A masked pattern was here ####
+1
+1
+1
+2
+2
+3
+PREHOOK: query: (select * from src order by key limit 1)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: (select * from src order by key limit 1)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+0	val_0
+PREHOOK: query: (select * from src) union all select * from src order by key limit 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: (select * from src) union all select * from src order by key limit 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+0	val_0
+PREHOOK: query: (select * from src limit 1) union all select * from src order by key limit 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: (select * from src limit 1) union all select * from src order by key limit 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+0	val_0
+PREHOOK: query: ((select * from src)) union all select * from src order by key limit 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: ((select * from src)) union all select * from src order by key limit 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+0	val_0
+PREHOOK: query: select * from src union all ((select * from src)) order by key limit 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: select * from src union all ((select * from src)) order by key limit 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+0	val_0
+PREHOOK: query: select * from src union all ((select * from src limit 1)) order by key limit 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: select * from src union all ((select * from src limit 1)) order by key limit 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+0	val_0
+PREHOOK: query: select * from src union all (select * from src) order by key limit 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: select * from src union all (select * from src) order by key limit 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+0	val_0
+PREHOOK: query: (select * from src order by key) union all (select * from src) order by key limit 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: (select * from src order by key) union all (select * from src) order by key limit 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+0	val_0
+PREHOOK: query: (select * from src order by key) union all (select * from src limit 1) order by key limit 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: (select * from src order by key) union all (select * from src limit 1) order by key limit 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+0	val_0
+PREHOOK: query: select count(*) from (select key from src union select key from src)cool_cust
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from (select key from src union select key from src)cool_cust
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+309
+PREHOOK: query: --similar tpcds q14
+
+with  cross_items as
+ (select key, k
+ from src,
+ (select iss.key k
+ from src iss
+ union all
+ select ics.key k
+ from src ics
+ ) x
+ where key = k
+)
+select * from cross_items order by key limit 1
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: --similar tpcds q14
+
+with  cross_items as
+ (select key, k
+ from src,
+ (select iss.key k
+ from src iss
+ union all
+ select ics.key k
+ from src ics
+ ) x
+ where key = k
+)
+select * from cross_items order by key limit 1
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+0	0

http://git-wip-us.apache.org/repos/asf/hive/blob/2435e702/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 9763679..d346010 100644
--- a/ql/src/test/results/clientpositive/union_ppr.q.out
+++ b/ql/src/test/results/clientpositive/union_ppr.q.out
@@ -174,8 +174,8 @@ STAGE PLANS:
               name: default.srcpart
             name: default.srcpart
       Truncated Path -> Alias:
-        /srcpart/ds=2008-04-08/hr=11 [null-subquery1:a-subquery1:x, null-subquery2:a-subquery2:y]
-        /srcpart/ds=2008-04-08/hr=12 [null-subquery1:a-subquery1:x, null-subquery2:a-subquery2:y]
+        /srcpart/ds=2008-04-08/hr=11 [a-subquery1:_u1-subquery1:x, a-subquery2:_u1-subquery2:y]
+        /srcpart/ds=2008-04-08/hr=12 [a-subquery1:_u1-subquery1:x, a-subquery2:_u1-subquery2:y]
       Needs Tagging: false
       Reduce Operator Tree:
         Select Operator


[18/22] hive git commit: Revert "HIVE-14803: S3: Stats gathering for insert queries can be expensive for partitioned dataset (Rajesh Balamohan via Pengcheng Xiong)"

Posted by se...@apache.org.
Revert "HIVE-14803: S3: Stats gathering for insert queries can be expensive for partitioned dataset (Rajesh Balamohan via Pengcheng Xiong)"

This reverts commit c52c17b227671ad1cb44b1fe24134c7ccb4a06e1.


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

Branch: refs/heads/hive-14535
Commit: 17afd9bb90ed8b073ac23c8c3162da9c877df90b
Parents: c52c17b
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue Oct 11 13:55:01 2016 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Tue Oct 11 13:55:01 2016 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hive/ql/exec/StatsTask.java   | 134 +++++++------------
 1 file changed, 52 insertions(+), 82 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/17afd9bb/ql/src/java/org/apache/hadoop/hive/ql/exec/StatsTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/StatsTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/StatsTask.java
index 51bafc4..9e528b5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/StatsTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/StatsTask.java
@@ -24,13 +24,7 @@ import java.util.ArrayList;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
 
-import com.google.common.collect.Lists;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.fs.FileStatus;
@@ -149,10 +143,10 @@ public class StatsTask extends Task<StatsWork> implements Serializable {
     StatsAggregator statsAggregator = null;
     int ret = 0;
     StatsCollectionContext scc = null;
-    final EnvironmentContext environmentContext = new EnvironmentContext();;
+    EnvironmentContext environmentContext = null;
     try {
       // Stats setup:
-      final Warehouse wh = new Warehouse(conf);
+      Warehouse wh = new Warehouse(conf);
       if (!getWork().getNoStatsAggregator() && !getWork().isNoScanAnalyzeCommand()) {
         try {
           scc = getContext();
@@ -166,8 +160,9 @@ public class StatsTask extends Task<StatsWork> implements Serializable {
       }
 
       List<Partition> partitions = getPartitionsList(db);
-      final boolean atomic = HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_STATS_ATOMIC);
-      final String tableFullName = table.getDbName() + "." + table.getTableName();
+      boolean atomic = HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_STATS_ATOMIC);
+
+      String tableFullName = table.getDbName() + "." + table.getTableName();
 
       if (partitions == null) {
         org.apache.hadoop.hive.metastore.api.Table tTable = table.getTTable();
@@ -205,6 +200,7 @@ public class StatsTask extends Task<StatsWork> implements Serializable {
           }
           // write table stats to metastore
           if (!getWork().getNoStatsAggregator()) {
+            environmentContext = new EnvironmentContext();
             environmentContext.putToProperties(StatsSetupConst.STATS_GENERATED,
                 StatsSetupConst.TASK);
           }
@@ -216,87 +212,61 @@ public class StatsTask extends Task<StatsWork> implements Serializable {
         }
         LOG.info("Table " + tableFullName + " stats: [" + toString(parameters) + ']');
       } else {
-        int poolSize = conf.getInt(ConfVars.HIVE_MOVE_FILES_THREAD_COUNT.varname, 1);
-        final ExecutorService pool = Executors.newFixedThreadPool(poolSize,
-            new ThreadFactoryBuilder()
-                .setDaemon(true)
-                .setNameFormat("stats-updater-thread-%d")
-                .build());
-        final List<Future<Void>> futures = Lists.newLinkedList();
-        final StatsAggregator statsAgg = statsAggregator;
-
         // Partitioned table:
         // Need to get the old stats of the partition
         // and update the table stats based on the old and new stats.
-        final List<Partition> updates = new ArrayList<Partition>();
-        try {
-          for (final Partition partn : partitions) {
-            futures.add(pool.submit(new Callable<Void>() {
-              @Override
-              public Void call() throws Exception {
-                //
-                // get the old partition stats
-                //
-                org.apache.hadoop.hive.metastore.api.Partition tPart = partn.getTPartition();
-                Map<String, String> parameters = tPart.getParameters();
-                if (work.getTableSpecs() == null && AcidUtils.isAcidTable(table)) {
-                  StatsSetupConst.setBasicStatsState(parameters, StatsSetupConst.FALSE);
-                } else if (work.getTableSpecs() != null
-                    || (work.getLoadTableDesc() != null && work.getLoadTableDesc().getReplace())
-                    || (work.getLoadFileDesc() != null && !work.getLoadFileDesc()
-                    .getDestinationCreateTable().isEmpty())) {
-                  StatsSetupConst.setBasicStatsState(parameters, StatsSetupConst.TRUE);
-                }
-                if (!existStats(parameters) && atomic) {
-                  return null;
-                }
-
-                // The collectable stats for the aggregator needs to be cleared.
-                // For eg. if a file is being loaded, the old number of rows are not valid
-                if (work.isClearAggregatorStats()) {
-                  // we choose to keep the invalid stats and only change the setting.
-                  StatsSetupConst.setBasicStatsState(parameters, StatsSetupConst.FALSE);
-                }
-
-                updateQuickStats(wh, parameters, tPart.getSd());
-                if (StatsSetupConst.areBasicStatsUptoDate(parameters)) {
-                  if (statsAgg != null) {
-                    String prefix = getAggregationPrefix(table, partn);
-                    updateStats(statsAgg, parameters, prefix, atomic);
-                  }
-                  if (!getWork().getNoStatsAggregator()) {
-                    environmentContext.putToProperties(StatsSetupConst.STATS_GENERATED,
-                        StatsSetupConst.TASK);
-                  }
-                }
-                updates.add(new Partition(table, tPart));
-
-                if (conf.getBoolVar(ConfVars.TEZ_EXEC_SUMMARY)) {
-                  console.printInfo("Partition " + tableFullName + partn.getSpec() +
-                      " stats: [" + StatsTask.toString(parameters) + ']');
-                }
-                LOG.info("Partition " + tableFullName + partn.getSpec() +
-                    " stats: [" + StatsTask.toString(parameters) + ']');
-                return null;
-              }
-            }));
+        List<Partition> updates = new ArrayList<Partition>();
+        for (Partition partn : partitions) {
+          //
+          // get the old partition stats
+          //
+          org.apache.hadoop.hive.metastore.api.Partition tPart = partn.getTPartition();
+          Map<String, String> parameters = tPart.getParameters();
+          if (work.getTableSpecs() == null && AcidUtils.isAcidTable(table)) {
+            StatsSetupConst.setBasicStatsState(parameters, StatsSetupConst.FALSE);
+          } else if (work.getTableSpecs() != null
+              || (work.getLoadTableDesc() != null && work.getLoadTableDesc().getReplace())
+              || (work.getLoadFileDesc() != null && !work.getLoadFileDesc()
+                  .getDestinationCreateTable().isEmpty())) {
+            StatsSetupConst.setBasicStatsState(parameters, StatsSetupConst.TRUE);
+          }
+          if (!existStats(parameters) && atomic) {
+            continue;
           }
-          pool.shutdown();
 
-          for (Future future : futures) {
-            future.get();
+          // The collectable stats for the aggregator needs to be cleared.
+          // For eg. if a file is being loaded, the old number of rows are not valid
+          if (work.isClearAggregatorStats()) {
+            // we choose to keep the invalid stats and only change the setting.
+            StatsSetupConst.setBasicStatsState(parameters, StatsSetupConst.FALSE);
           }
-        } catch (InterruptedException e) {
-          LOG.debug("Cancelling " + futures.size() + " for partition stats update");
-          //cancel other futures
-          for (Future future : futures) {
-            future.cancel(true);
+
+          updateQuickStats(wh, parameters, tPart.getSd());
+          if (StatsSetupConst.areBasicStatsUptoDate(parameters)) {
+            if (statsAggregator != null) {
+              String prefix = getAggregationPrefix(table, partn);
+              updateStats(statsAggregator, parameters, prefix, atomic);
+            }
+            if (!getWork().getNoStatsAggregator()) {
+              environmentContext = new EnvironmentContext();
+              environmentContext.putToProperties(StatsSetupConst.STATS_GENERATED,
+                  StatsSetupConst.TASK);
+            }
           }
+          updates.add(new Partition(table, tPart));
+
+          if (conf.getBoolVar(ConfVars.TEZ_EXEC_SUMMARY)) {
+            console.printInfo("Partition " + tableFullName + partn.getSpec() +
+            " stats: [" + toString(parameters) + ']');
+          }
+          LOG.info("Partition " + tableFullName + partn.getSpec() +
+              " stats: [" + toString(parameters) + ']');
         }
         if (!updates.isEmpty()) {
-            db.alterPartitions(tableFullName, updates, environmentContext);
-          }
+          db.alterPartitions(tableFullName, updates, environmentContext);
         }
+      }
+
     } catch (Exception e) {
       console.printInfo("[Warning] could not update stats.",
           "Failed with exception " + e.getMessage() + "\n"
@@ -404,7 +374,7 @@ public class StatsTask extends Task<StatsWork> implements Serializable {
     MetaStoreUtils.populateQuickStats(partfileStatus, parameters);
   }
 
-  static String toString(Map<String, String> parameters) {
+  private String toString(Map<String, String> parameters) {
     StringBuilder builder = new StringBuilder();
     for (String statType : StatsSetupConst.supportedStats) {
       String value = parameters.get(statType);


[12/22] hive git commit: HIVE-14861: Support precedence for set operator using parentheses (Pengcheng Xiong, reviewed by Ashutosh Chauhan)

Posted by se...@apache.org.
HIVE-14861: Support precedence for set operator using parentheses (Pengcheng Xiong, 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/2435e702
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/2435e702
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/2435e702

Branch: refs/heads/hive-14535
Commit: 2435e702dab5bd50c6605af15e229f678b8f545d
Parents: 15039c0
Author: Pengcheng Xiong <px...@apache.org>
Authored: Fri Oct 7 13:53:36 2016 -0700
Committer: Pengcheng Xiong <px...@apache.org>
Committed: Fri Oct 7 13:53:36 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hive/ql/parse/FromClauseParser.g     |   2 +-
 .../apache/hadoop/hive/ql/parse/HiveParser.g    | 104 +--
 .../hadoop/hive/ql/parse/IdentifiersParser.g    |   2 +-
 .../ptf_negative_AmbiguousWindowDefn.q          |   6 +-
 .../clientnegative/ptf_negative_NoWindowDefn.q  |   4 +-
 .../clientnegative/windowing_after_orderby.q    |   7 +
 .../test/queries/clientpositive/union_paren.q   |  54 ++
 .../create_or_replace_view6.q.out               |   2 +-
 .../results/clientnegative/unionClusterBy.q.out |   2 +-
 .../clientnegative/unionDistributeBy.q.out      |   2 +-
 .../results/clientnegative/unionLimit.q.out     |   2 +-
 .../results/clientnegative/unionOrderBy.q.out   |   2 +-
 .../results/clientnegative/unionSortBy.q.out    |   2 +-
 .../windowing_after_orderby.q.out               |   9 +
 .../clientpositive/cbo_rp_lineage2.q.out        |   2 +-
 .../results/clientpositive/complex_alias.q.out  |   4 +-
 .../clientpositive/constant_prop_1.q.out        |  12 +-
 .../results/clientpositive/input_part7.q.out    |   4 +-
 .../clientpositive/llap/explainuser_2.q.out     | 442 ++++++------
 .../clientpositive/optimize_nullscan.q.out      |   8 +-
 .../clientpositive/tez/explainanalyze_2.q.out   | 704 ++++++++++---------
 .../results/clientpositive/union_offcbo.q.out   | 112 +--
 .../results/clientpositive/union_paren.q.out    | 260 +++++++
 .../test/results/clientpositive/union_ppr.q.out |   4 +-
 24 files changed, 1083 insertions(+), 669 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/2435e702/ql/src/java/org/apache/hadoop/hive/ql/parse/FromClauseParser.g
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/FromClauseParser.g b/ql/src/java/org/apache/hadoop/hive/ql/parse/FromClauseParser.g
index ad9abce..bf35d60 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/FromClauseParser.g
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/FromClauseParser.g
@@ -222,7 +222,7 @@ subQuerySource
 @init { gParent.pushMsg("subquery source", state); }
 @after { gParent.popMsg(state); }
     :
-    LPAREN queryStatementExpression[false] RPAREN KW_AS? identifier -> ^(TOK_SUBQUERY queryStatementExpression identifier)
+    LPAREN queryStatementExpression RPAREN KW_AS? identifier -> ^(TOK_SUBQUERY queryStatementExpression identifier)
     ;
 
 //---------------------- Rules for parsing PTF clauses -----------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/2435e702/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 bf78545..bef3acf 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
@@ -717,7 +717,7 @@ explainStatement
 	: KW_EXPLAIN (
 	    explainOption* execStatement -> ^(TOK_EXPLAIN execStatement explainOption*)
         |
-        KW_REWRITE queryStatementExpression[true] -> ^(TOK_EXPLAIN_SQ_REWRITE queryStatementExpression))
+        KW_REWRITE queryStatementExpression -> ^(TOK_EXPLAIN_SQ_REWRITE queryStatementExpression))
 	;
 
 explainOption
@@ -729,7 +729,7 @@ explainOption
 execStatement
 @init { pushMsg("statement", state); }
 @after { popMsg(state); }
-    : queryStatementExpression[true]
+    : queryStatementExpression
     | loadStatement
     | exportStatement
     | importStatement
@@ -2310,14 +2310,14 @@ setOperator
     | KW_UNION KW_DISTINCT? -> ^(TOK_UNIONDISTINCT)
     ;
 
-queryStatementExpression[boolean topLevel]
+queryStatementExpression
     :
     /* Would be nice to do this as a gated semantic perdicate
        But the predicate gets pushed as a lookahead decision.
        Calling rule doesnot know about topLevel
     */
-    (w=withClause {topLevel}?)?
-    queryStatementExpressionBody[topLevel] {
+    (w=withClause)?
+    queryStatementExpressionBody {
       if ($w.tree != null) {
       $queryStatementExpressionBody.tree.insertChild(0, $w.tree);
       }
@@ -2325,10 +2325,10 @@ queryStatementExpression[boolean topLevel]
     ->  queryStatementExpressionBody
     ;
 
-queryStatementExpressionBody[boolean topLevel]
+queryStatementExpressionBody
     :
-    fromStatement[topLevel]
-    | regularBody[topLevel]
+    fromStatement
+    | regularBody
     ;
 
 withClause
@@ -2338,16 +2338,16 @@ withClause
 
 cteStatement
    :
-   identifier KW_AS LPAREN queryStatementExpression[false] RPAREN
+   identifier KW_AS LPAREN queryStatementExpression RPAREN
    -> ^(TOK_SUBQUERY queryStatementExpression identifier)
 ;
 
-fromStatement[boolean topLevel]
+fromStatement
 : (singleFromStatement  -> singleFromStatement)
 	(u=setOperator r=singleFromStatement
 	  -> ^($u {$fromStatement.tree} $r)
 	)*
-	 -> {u != null && topLevel}? ^(TOK_QUERY
+	 -> {u != null}? ^(TOK_QUERY
 	       ^(TOK_FROM
 	         ^(TOK_SUBQUERY
 	           {$fromStatement.tree}
@@ -2376,11 +2376,11 @@ The valuesClause rule below ensures that the parse tree for
 very similar to the tree for "insert into table FOO select a,b from BAR".  Since virtual table name
 is implicit, it's represented as TOK_ANONYMOUS.
 */
-regularBody[boolean topLevel]
+regularBody
    :
    i=insertClause
    (
-   s=selectStatement[topLevel]
+   s=selectStatement
      {$s.tree.getFirstChildWithType(TOK_INSERT).replaceChildren(0, 0, $i.tree);} -> {$s.tree}
      |
      valuesClause
@@ -2392,38 +2392,63 @@ regularBody[boolean topLevel]
           )
    )
    |
-   selectStatement[topLevel]
+   selectStatement
    ;
 
-selectStatement[boolean topLevel]
+atomSelectStatement
    :
-   (
    s=selectClause
    f=fromClause?
    w=whereClause?
    g=groupByClause?
    h=havingClause?
+   win=window_clause?
+   -> ^(TOK_QUERY $f? ^(TOK_INSERT ^(TOK_DESTINATION ^(TOK_DIR TOK_TMP_FILE))
+                     $s $w? $g? $h? $win?))
+   |
+   LPAREN! selectStatement RPAREN!
+   ;
+
+selectStatement
+   :
+   a=atomSelectStatement
+   set=setOpSelectStatement[$atomSelectStatement.tree]?
    o=orderByClause?
    c=clusterByClause?
    d=distributeByClause?
    sort=sortByClause?
-   win=window_clause?
    l=limitClause?
-   -> ^(TOK_QUERY $f? ^(TOK_INSERT ^(TOK_DESTINATION ^(TOK_DIR TOK_TMP_FILE))
-                     $s $w? $g? $h? $o? $c?
-                     $d? $sort? $win? $l?))
-   )
-   (set=setOpSelectStatement[$selectStatement.tree, topLevel])?
+   {
+   if(set == null){
+   $a.tree.getFirstChildWithType(TOK_INSERT).addChild($o.tree);
+   $a.tree.getFirstChildWithType(TOK_INSERT).addChild($c.tree);
+   $a.tree.getFirstChildWithType(TOK_INSERT).addChild($d.tree);
+   $a.tree.getFirstChildWithType(TOK_INSERT).addChild($sort.tree);
+   $a.tree.getFirstChildWithType(TOK_INSERT).addChild($l.tree);
+   }
+   }
    -> {set == null}?
-      {$selectStatement.tree}
+      {$a.tree}
    -> {o==null && c==null && d==null && sort==null && l==null}?
       {$set.tree}
-   -> {throwSetOpException()}
+   -> ^(TOK_QUERY
+          ^(TOK_FROM
+            ^(TOK_SUBQUERY
+              {$set.tree}
+              {adaptor.create(Identifier, generateUnionAlias())}
+             )
+          )
+          ^(TOK_INSERT
+             ^(TOK_DESTINATION ^(TOK_DIR TOK_TMP_FILE))
+             ^(TOK_SELECT ^(TOK_SELEXPR TOK_ALLCOLREF))
+             $o? $c? $d? $sort? $l?
+          )
+      )
    ;
 
-setOpSelectStatement[CommonTree t, boolean topLevel]
+setOpSelectStatement[CommonTree t]
    :
-   (u=setOperator b=simpleSelectStatement
+   (u=setOperator b=atomSelectStatement
    -> {$setOpSelectStatement.tree != null && u.tree.getType()==HiveParser.TOK_UNIONDISTINCT}?
       ^(TOK_QUERY
           ^(TOK_FROM
@@ -2454,15 +2479,8 @@ setOpSelectStatement[CommonTree t, boolean topLevel]
        )
    -> ^(TOK_UNIONALL {$t} $b)
    )+
-   o=orderByClause?
-   c=clusterByClause?
-   d=distributeByClause?
-   sort=sortByClause?
-   win=window_clause?
-   l=limitClause?
-   -> {o==null && c==null && d==null && sort==null && win==null && l==null && !topLevel}?
-      {$setOpSelectStatement.tree}
-   -> ^(TOK_QUERY
+   -> {$setOpSelectStatement.tree.getChild(0).getType()==HiveParser.TOK_UNIONALL}?
+      ^(TOK_QUERY
           ^(TOK_FROM
             ^(TOK_SUBQUERY
               {$setOpSelectStatement.tree}
@@ -2472,27 +2490,15 @@ setOpSelectStatement[CommonTree t, boolean topLevel]
           ^(TOK_INSERT
              ^(TOK_DESTINATION ^(TOK_DIR TOK_TMP_FILE))
              ^(TOK_SELECT ^(TOK_SELEXPR TOK_ALLCOLREF))
-             $o? $c? $d? $sort? $win? $l?
           )
        )
-   ;
-
-simpleSelectStatement
-   :
-   selectClause
-   fromClause?
-   whereClause?
-   groupByClause?
-   havingClause?
-   ((window_clause) => window_clause)?
-   -> ^(TOK_QUERY fromClause? ^(TOK_INSERT ^(TOK_DESTINATION ^(TOK_DIR TOK_TMP_FILE))
-                     selectClause whereClause? groupByClause? havingClause? window_clause?))
+   -> {$setOpSelectStatement.tree}
    ;
 
 selectStatementWithCTE
     :
     (w=withClause)?
-    selectStatement[true] {
+    selectStatement {
       if ($w.tree != null) {
       $selectStatement.tree.insertChild(0, $w.tree);
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/2435e702/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 4a44173..6ae731f 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
@@ -476,7 +476,7 @@ precedenceEqualOperator
 
 subQueryExpression 
     : 
-    LPAREN! selectStatement[true] RPAREN!     
+    LPAREN! selectStatement RPAREN!
     ;
 
 precedenceEqualExpression

http://git-wip-us.apache.org/repos/asf/hive/blob/2435e702/ql/src/test/queries/clientnegative/ptf_negative_AmbiguousWindowDefn.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/ptf_negative_AmbiguousWindowDefn.q b/ql/src/test/queries/clientnegative/ptf_negative_AmbiguousWindowDefn.q
index b2465fd..5128576 100644
--- a/ql/src/test/queries/clientnegative/ptf_negative_AmbiguousWindowDefn.q
+++ b/ql/src/test/queries/clientnegative/ptf_negative_AmbiguousWindowDefn.q
@@ -4,9 +4,9 @@ sum(p_size) over (w1) as s1,
 sum(p_size) over (w2) as s2,
 sum(p_size) over (w3) as s3
 from part 
-distribute by p_mfgr 
-sort by p_mfgr 
 window w1 as (rows between 2 preceding and 2 following), 
        w2 as (rows between unbounded preceding and current row), 
-       w3 as w3;
+       w3 as w3
+distribute by p_mfgr 
+sort by p_mfgr;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/2435e702/ql/src/test/queries/clientnegative/ptf_negative_NoWindowDefn.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/ptf_negative_NoWindowDefn.q b/ql/src/test/queries/clientnegative/ptf_negative_NoWindowDefn.q
index 8defb3a..99932cd 100644
--- a/ql/src/test/queries/clientnegative/ptf_negative_NoWindowDefn.q
+++ b/ql/src/test/queries/clientnegative/ptf_negative_NoWindowDefn.q
@@ -3,7 +3,7 @@ select p_mfgr, p_name, p_size,
 sum(p_size) over (w1) as s1,
 sum(p_size) over (w2) as s2
 from part
+window w1 as (rows between 2 preceding and 2 following)
 distribute by p_mfgr
-sort by p_mfgr
-window w1 as (rows between 2 preceding and 2 following);
+sort by p_mfgr;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/2435e702/ql/src/test/queries/clientnegative/windowing_after_orderby.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/windowing_after_orderby.q b/ql/src/test/queries/clientnegative/windowing_after_orderby.q
new file mode 100644
index 0000000..b9acb34
--- /dev/null
+++ b/ql/src/test/queries/clientnegative/windowing_after_orderby.q
@@ -0,0 +1,7 @@
+create table empsalary (depname varchar(10), salary int);
+
+SELECT sum(salary) OVER w as s , avg(salary) OVER w as a
+  FROM empsalary
+  order by s
+  WINDOW w AS (PARTITION BY depname ORDER BY salary DESC);
+

http://git-wip-us.apache.org/repos/asf/hive/blob/2435e702/ql/src/test/queries/clientpositive/union_paren.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/union_paren.q b/ql/src/test/queries/clientpositive/union_paren.q
new file mode 100644
index 0000000..0b38b68
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/union_paren.q
@@ -0,0 +1,54 @@
+set hive.mapred.mode=nonstrict;
+
+explain select * from src union all select * from src;
+
+create table t1(c int);
+
+insert into t1 values (1),(1),(2);
+
+create table t2(c int);
+
+insert into t2 values (2),(1),(2);
+
+create table t3(c int);
+
+insert into t3 values (2),(3),(2);
+
+(select * from t1) union all select * from t2 union select * from t3 order by c;
+
+(select * from t1) union all (select * from t2 union select * from t3) order by c;
+
+(select * from src order by key limit 1);
+
+(select * from src) union all select * from src order by key limit 1;
+
+(select * from src limit 1) union all select * from src order by key limit 1;
+
+((select * from src)) union all select * from src order by key limit 1;
+
+select * from src union all ((select * from src)) order by key limit 1;
+
+select * from src union all ((select * from src limit 1)) order by key limit 1;
+
+select * from src union all (select * from src) order by key limit 1;
+
+(select * from src order by key) union all (select * from src) order by key limit 1;
+
+(select * from src order by key) union all (select * from src limit 1) order by key limit 1;
+
+select count(*) from (select key from src union select key from src)cool_cust;
+
+--similar tpcds q14
+
+with  cross_items as
+ (select key, k
+ from src,
+ (select iss.key k
+ from src iss
+ union all
+ select ics.key k
+ from src ics
+ ) x
+ where key = k
+)
+select * from cross_items order by key limit 1;

http://git-wip-us.apache.org/repos/asf/hive/blob/2435e702/ql/src/test/results/clientnegative/create_or_replace_view6.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/create_or_replace_view6.q.out b/ql/src/test/results/clientnegative/create_or_replace_view6.q.out
index b919e72..6f571dd 100644
--- a/ql/src/test/results/clientnegative/create_or_replace_view6.q.out
+++ b/ql/src/test/results/clientnegative/create_or_replace_view6.q.out
@@ -16,4 +16,4 @@ POSTHOOK: type: CREATEVIEW
 POSTHOOK: Input: default@srcpart
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@v
-FAILED: ParseException line 2:52 cannot recognize input near 'blah' '<EOF>' '<EOF>' in select clause
+FAILED: ParseException line 2:52 cannot recognize input near 'blah' '<EOF>' '<EOF>' in create view statement

http://git-wip-us.apache.org/repos/asf/hive/blob/2435e702/ql/src/test/results/clientnegative/unionClusterBy.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/unionClusterBy.q.out b/ql/src/test/results/clientnegative/unionClusterBy.q.out
index 4705f3b..7f01413 100644
--- a/ql/src/test/results/clientnegative/unionClusterBy.q.out
+++ b/ql/src/test/results/clientnegative/unionClusterBy.q.out
@@ -1 +1 @@
-FAILED: ParseException line 6:19 Failed to recognize predicate '<EOF>'. Failed rule: 'orderByClause clusterByClause distributeByClause sortByClause limitClause can only be applied to the whole union.' in statement
+FAILED: ParseException line 5:0 missing EOF at 'union' near 'key'

http://git-wip-us.apache.org/repos/asf/hive/blob/2435e702/ql/src/test/results/clientnegative/unionDistributeBy.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/unionDistributeBy.q.out b/ql/src/test/results/clientnegative/unionDistributeBy.q.out
index 4705f3b..7f01413 100644
--- a/ql/src/test/results/clientnegative/unionDistributeBy.q.out
+++ b/ql/src/test/results/clientnegative/unionDistributeBy.q.out
@@ -1 +1 @@
-FAILED: ParseException line 6:19 Failed to recognize predicate '<EOF>'. Failed rule: 'orderByClause clusterByClause distributeByClause sortByClause limitClause can only be applied to the whole union.' in statement
+FAILED: ParseException line 5:0 missing EOF at 'union' near 'key'

http://git-wip-us.apache.org/repos/asf/hive/blob/2435e702/ql/src/test/results/clientnegative/unionLimit.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/unionLimit.q.out b/ql/src/test/results/clientnegative/unionLimit.q.out
index 4705f3b..e8463b5 100644
--- a/ql/src/test/results/clientnegative/unionLimit.q.out
+++ b/ql/src/test/results/clientnegative/unionLimit.q.out
@@ -1 +1 @@
-FAILED: ParseException line 6:19 Failed to recognize predicate '<EOF>'. Failed rule: 'orderByClause clusterByClause distributeByClause sortByClause limitClause can only be applied to the whole union.' in statement
+FAILED: ParseException line 5:0 missing EOF at 'union' near '1'

http://git-wip-us.apache.org/repos/asf/hive/blob/2435e702/ql/src/test/results/clientnegative/unionOrderBy.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/unionOrderBy.q.out b/ql/src/test/results/clientnegative/unionOrderBy.q.out
index 4705f3b..7f01413 100644
--- a/ql/src/test/results/clientnegative/unionOrderBy.q.out
+++ b/ql/src/test/results/clientnegative/unionOrderBy.q.out
@@ -1 +1 @@
-FAILED: ParseException line 6:19 Failed to recognize predicate '<EOF>'. Failed rule: 'orderByClause clusterByClause distributeByClause sortByClause limitClause can only be applied to the whole union.' in statement
+FAILED: ParseException line 5:0 missing EOF at 'union' near 'key'

http://git-wip-us.apache.org/repos/asf/hive/blob/2435e702/ql/src/test/results/clientnegative/unionSortBy.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/unionSortBy.q.out b/ql/src/test/results/clientnegative/unionSortBy.q.out
index 4705f3b..7f01413 100644
--- a/ql/src/test/results/clientnegative/unionSortBy.q.out
+++ b/ql/src/test/results/clientnegative/unionSortBy.q.out
@@ -1 +1 @@
-FAILED: ParseException line 6:19 Failed to recognize predicate '<EOF>'. Failed rule: 'orderByClause clusterByClause distributeByClause sortByClause limitClause can only be applied to the whole union.' in statement
+FAILED: ParseException line 5:0 missing EOF at 'union' near 'key'

http://git-wip-us.apache.org/repos/asf/hive/blob/2435e702/ql/src/test/results/clientnegative/windowing_after_orderby.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/windowing_after_orderby.q.out b/ql/src/test/results/clientnegative/windowing_after_orderby.q.out
new file mode 100644
index 0000000..03325bc
--- /dev/null
+++ b/ql/src/test/results/clientnegative/windowing_after_orderby.q.out
@@ -0,0 +1,9 @@
+PREHOOK: query: create table empsalary (depname varchar(10), salary int)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@empsalary
+POSTHOOK: query: create table empsalary (depname varchar(10), salary int)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@empsalary
+FAILED: ParseException line 6:2 missing EOF at 'WINDOW' near 's'

http://git-wip-us.apache.org/repos/asf/hive/blob/2435e702/ql/src/test/results/clientpositive/cbo_rp_lineage2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/cbo_rp_lineage2.q.out b/ql/src/test/results/clientpositive/cbo_rp_lineage2.q.out
index 9e0613f..e5df0ee 100644
--- a/ql/src/test/results/clientpositive/cbo_rp_lineage2.q.out
+++ b/ql/src/test/results/clientpositive/cbo_rp_lineage2.q.out
@@ -552,7 +552,7 @@ PREHOOK: type: QUERY
 PREHOOK: Input: default@src
 PREHOOK: Input: default@src1
 PREHOOK: Output: default@dest_l1
-{"version":"1.0","engine":"mr","database":"default","hash":"60b589744e2527dd235a6c8168d6a653","queryText":"INSERT OVERWRITE TABLE dest_l1\nSELECT j.*\nFROM (SELECT t1.key, p1.value\n      FROM src1 t1\n      LEFT OUTER JOIN src p1\n      ON (t1.key = p1.key)\n      UNION ALL\n      SELECT t2.key, p2.value\n      FROM src1 t2\n      LEFT OUTER JOIN src p2\n      ON (t2.key = p2.key)) j","edges":[{"sources":[2],"targets":[0],"expression":"UDFToInteger(key)","edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"expression":"value","edgeType":"PROJECTION"},{"sources":[4,2],"targets":[0,1],"expression":"(null-subquery1:j-subquery1:p1.key = null-subquery1:j-subquery1:t1.key)","edgeType":"PREDICATE"},{"sources":[4,2],"targets":[0,1],"expression":"(null-subquery2:j-subquery2:p2.key = null-subquery2:j-subquery2:t2.key)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"default.dest_l1.key"},{"id":1,"vertexType":"COLUMN","vertexId":"default.dest_l1.value"},{"id":
 2,"vertexType":"COLUMN","vertexId":"default.src1.key"},{"id":3,"vertexType":"COLUMN","vertexId":"default.src.value"},{"id":4,"vertexType":"COLUMN","vertexId":"default.src.key"}]}
+{"version":"1.0","engine":"mr","database":"default","hash":"60b589744e2527dd235a6c8168d6a653","queryText":"INSERT OVERWRITE TABLE dest_l1\nSELECT j.*\nFROM (SELECT t1.key, p1.value\n      FROM src1 t1\n      LEFT OUTER JOIN src p1\n      ON (t1.key = p1.key)\n      UNION ALL\n      SELECT t2.key, p2.value\n      FROM src1 t2\n      LEFT OUTER JOIN src p2\n      ON (t2.key = p2.key)) j","edges":[{"sources":[2],"targets":[0],"expression":"UDFToInteger(key)","edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"expression":"value","edgeType":"PROJECTION"},{"sources":[4,2],"targets":[0,1],"expression":"(j-subquery1:_u1-subquery1:p1.key = j-subquery1:_u1-subquery1:t1.key)","edgeType":"PREDICATE"},{"sources":[4,2],"targets":[0,1],"expression":"(j-subquery2:_u1-subquery2:p2.key = j-subquery2:_u1-subquery2:t2.key)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"default.dest_l1.key"},{"id":1,"vertexType":"COLUMN","vertexId":"default.dest_l1.value"},{"id":2,"v
 ertexType":"COLUMN","vertexId":"default.src1.key"},{"id":3,"vertexType":"COLUMN","vertexId":"default.src.value"},{"id":4,"vertexType":"COLUMN","vertexId":"default.src.key"}]}
 PREHOOK: query: drop table if exists emp
 PREHOOK: type: DROPTABLE
 PREHOOK: query: drop table if exists dept

http://git-wip-us.apache.org/repos/asf/hive/blob/2435e702/ql/src/test/results/clientpositive/complex_alias.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/complex_alias.q.out b/ql/src/test/results/clientpositive/complex_alias.q.out
index 4cb6c83..361bc8d 100644
--- a/ql/src/test/results/clientpositive/complex_alias.q.out
+++ b/ql/src/test/results/clientpositive/complex_alias.q.out
@@ -17,7 +17,7 @@ POSTHOOK: Output: default@agg1
 POSTHOOK: Lineage: agg1.col0 EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ]
 POSTHOOK: Lineage: agg1.col1 SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
 POSTHOOK: Lineage: agg1.col2 EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ]
-Warning: Shuffle Join JOIN[19][tables = [single_use_subq12, single_use_subq11]] in Stage 'Stage-2:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[20][tables = [single_use_subq12, single_use_subq11]] in Stage 'Stage-2:MAPRED' is a cross product
 PREHOOK: query: EXPLAIN
 SELECT single_use_subq11.a1 AS a1,
        single_use_subq11.a2 AS a2
@@ -218,7 +218,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join JOIN[19][tables = [single_use_subq12, single_use_subq11]] in Stage 'Stage-2:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[20][tables = [single_use_subq12, single_use_subq11]] in Stage 'Stage-2:MAPRED' is a cross product
 PREHOOK: query: SELECT single_use_subq11.a1 AS a1,
        single_use_subq11.a2 AS a2
 FROM   (SELECT Sum(agg1.col2) AS a1

http://git-wip-us.apache.org/repos/asf/hive/blob/2435e702/ql/src/test/results/clientpositive/constant_prop_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/constant_prop_1.q.out b/ql/src/test/results/clientpositive/constant_prop_1.q.out
index 8695e1d..aaa1dac 100644
--- a/ql/src/test/results/clientpositive/constant_prop_1.q.out
+++ b/ql/src/test/results/clientpositive/constant_prop_1.q.out
@@ -65,7 +65,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join JOIN[13][tables = [sub, b]] in Stage 'Stage-2:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[14][tables = [sub, b]] in Stage 'Stage-2:MAPRED' is a cross product
 PREHOOK: query: explain
 select a, key, value from
 (
@@ -102,10 +102,10 @@ STAGE PLANS:
                   Statistics: Num rows: 1000 Data size: 4000 Basic stats: COMPLETE Column stats: COMPLETE
                   Limit
                     Number of rows: 1
-                    Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
                       sort order: 
-                      Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                       TopN Hash Memory Usage: 0.1
           TableScan
             alias: src
@@ -118,10 +118,10 @@ STAGE PLANS:
                   Statistics: Num rows: 1000 Data size: 4000 Basic stats: COMPLETE Column stats: COMPLETE
                   Limit
                     Number of rows: 1
-                    Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
                       sort order: 
-                      Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                       TopN Hash Memory Usage: 0.1
       Reduce Operator Tree:
         Limit
@@ -241,7 +241,7 @@ STAGE PLANS:
       Processor Tree:
         ListSink
 
-Warning: Shuffle Join JOIN[13][tables = [sub, b]] in Stage 'Stage-2:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[14][tables = [sub, b]] in Stage 'Stage-2:MAPRED' is a cross product
 PREHOOK: query: explain
 select a, key, value from
 (

http://git-wip-us.apache.org/repos/asf/hive/blob/2435e702/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 459e384..3543d0e 100644
--- a/ql/src/test/results/clientpositive/input_part7.q.out
+++ b/ql/src/test/results/clientpositive/input_part7.q.out
@@ -168,8 +168,8 @@ STAGE PLANS:
               name: default.srcpart
             name: default.srcpart
       Truncated Path -> Alias:
-        /srcpart/ds=2008-04-08/hr=11 [null-subquery1:a-subquery1:x, null-subquery2:a-subquery2:y]
-        /srcpart/ds=2008-04-08/hr=12 [null-subquery1:a-subquery1:x, null-subquery2:a-subquery2:y]
+        /srcpart/ds=2008-04-08/hr=11 [a-subquery1:_u1-subquery1:x, a-subquery2:_u1-subquery2:y]
+        /srcpart/ds=2008-04-08/hr=12 [a-subquery1:_u1-subquery1:x, a-subquery2:_u1-subquery2:y]
       Needs Tagging: false
       Reduce Operator Tree:
         Select Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/2435e702/ql/src/test/results/clientpositive/llap/explainuser_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/explainuser_2.q.out b/ql/src/test/results/clientpositive/llap/explainuser_2.q.out
index 840b757..931f1a2 100644
--- a/ql/src/test/results/clientpositive/llap/explainuser_2.q.out
+++ b/ql/src/test/results/clientpositive/llap/explainuser_2.q.out
@@ -2966,11 +2966,15 @@ STAGE PLANS:
                       expressions: value (type: string)
                       outputColumnNames: _col1
                       Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
-                      Reduce Output Operator
-                        key expressions: _col1 (type: string)
-                        sort order: +
-                        Map-reduce partition columns: _col1 (type: string)
+                      Select Operator
+                        expressions: _col1 (type: string)
+                        outputColumnNames: _col1
                         Statistics: Num rows: 525 Data size: 5503 Basic stats: COMPLETE Column stats: NONE
+                        Reduce Output Operator
+                          key expressions: _col1 (type: string)
+                          sort order: +
+                          Map-reduce partition columns: _col1 (type: string)
+                          Statistics: Num rows: 525 Data size: 5503 Basic stats: COMPLETE Column stats: NONE
             Execution mode: llap
             LLAP IO: no inputs
         Map 11 
@@ -2985,11 +2989,15 @@ STAGE PLANS:
                       expressions: value (type: string)
                       outputColumnNames: _col1
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                      Reduce Output Operator
-                        key expressions: _col1 (type: string)
-                        sort order: +
-                        Map-reduce partition columns: _col1 (type: string)
+                      Select Operator
+                        expressions: _col1 (type: string)
+                        outputColumnNames: _col1
                         Statistics: Num rows: 1025 Data size: 10815 Basic stats: COMPLETE Column stats: NONE
+                        Reduce Output Operator
+                          key expressions: _col1 (type: string)
+                          sort order: +
+                          Map-reduce partition columns: _col1 (type: string)
+                          Statistics: Num rows: 1025 Data size: 10815 Basic stats: COMPLETE Column stats: NONE
             Execution mode: llap
             LLAP IO: no inputs
         Map 12 
@@ -3004,11 +3012,15 @@ STAGE PLANS:
                       expressions: value (type: string)
                       outputColumnNames: _col1
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                      Reduce Output Operator
-                        key expressions: _col1 (type: string)
-                        sort order: +
-                        Map-reduce partition columns: _col1 (type: string)
+                      Select Operator
+                        expressions: _col1 (type: string)
+                        outputColumnNames: _col1
                         Statistics: Num rows: 1025 Data size: 10815 Basic stats: COMPLETE Column stats: NONE
+                        Reduce Output Operator
+                          key expressions: _col1 (type: string)
+                          sort order: +
+                          Map-reduce partition columns: _col1 (type: string)
+                          Statistics: Num rows: 1025 Data size: 10815 Basic stats: COMPLETE Column stats: NONE
             Execution mode: llap
             LLAP IO: no inputs
         Map 13 
@@ -3055,44 +3067,48 @@ STAGE PLANS:
                       expressions: value (type: string)
                       outputColumnNames: _col1
                       Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
-                      Map Join Operator
-                        condition map:
-                             Inner Join 0 to 1
-                        keys:
-                          0 _col1 (type: string)
-                          1 _col1 (type: string)
-                        outputColumnNames: _col0, _col6
-                        input vertices:
-                          0 Map 20
-                        Statistics: Num rows: 1677 Data size: 17739 Basic stats: COMPLETE Column stats: NONE
-                        Select Operator
-                          expressions: _col0 (type: string), _col6 (type: string)
-                          outputColumnNames: _col0, _col1
+                      Select Operator
+                        expressions: _col1 (type: string)
+                        outputColumnNames: _col1
+                        Statistics: Num rows: 1525 Data size: 16127 Basic stats: COMPLETE Column stats: NONE
+                        Map Join Operator
+                          condition map:
+                               Inner Join 0 to 1
+                          keys:
+                            0 _col1 (type: string)
+                            1 _col1 (type: string)
+                          outputColumnNames: _col0, _col6
+                          input vertices:
+                            0 Map 20
                           Statistics: Num rows: 1677 Data size: 17739 Basic stats: COMPLETE Column stats: NONE
-                          File Output Operator
-                            compressed: false
-                            Statistics: Num rows: 3409 Data size: 36062 Basic stats: COMPLETE Column stats: NONE
-                            table:
-                                input format: org.apache.hadoop.mapred.TextInputFormat
-                                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                                name: default.a
-                          File Output Operator
-                            compressed: false
-                            Statistics: Num rows: 3409 Data size: 36062 Basic stats: COMPLETE Column stats: NONE
-                            table:
-                                input format: org.apache.hadoop.mapred.TextInputFormat
-                                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                                name: default.b
-                          File Output Operator
-                            compressed: false
-                            Statistics: Num rows: 3409 Data size: 36062 Basic stats: COMPLETE Column stats: NONE
-                            table:
-                                input format: org.apache.hadoop.mapred.TextInputFormat
-                                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                                name: default.c
+                          Select Operator
+                            expressions: _col0 (type: string), _col6 (type: string)
+                            outputColumnNames: _col0, _col1
+                            Statistics: Num rows: 1677 Data size: 17739 Basic stats: COMPLETE Column stats: NONE
+                            File Output Operator
+                              compressed: false
+                              Statistics: Num rows: 3409 Data size: 36062 Basic stats: COMPLETE Column stats: NONE
+                              table:
+                                  input format: org.apache.hadoop.mapred.TextInputFormat
+                                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                                  name: default.a
+                            File Output Operator
+                              compressed: false
+                              Statistics: Num rows: 3409 Data size: 36062 Basic stats: COMPLETE Column stats: NONE
+                              table:
+                                  input format: org.apache.hadoop.mapred.TextInputFormat
+                                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                                  name: default.b
+                            File Output Operator
+                              compressed: false
+                              Statistics: Num rows: 3409 Data size: 36062 Basic stats: COMPLETE Column stats: NONE
+                              table:
+                                  input format: org.apache.hadoop.mapred.TextInputFormat
+                                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                                  name: default.c
             Execution mode: llap
             LLAP IO: no inputs
         Map 17 
@@ -3107,44 +3123,48 @@ STAGE PLANS:
                       expressions: value (type: string)
                       outputColumnNames: _col1
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                      Map Join Operator
-                        condition map:
-                             Inner Join 0 to 1
-                        keys:
-                          0 _col1 (type: string)
-                          1 _col1 (type: string)
-                        outputColumnNames: _col0, _col6
-                        input vertices:
-                          0 Map 20
-                        Statistics: Num rows: 1677 Data size: 17739 Basic stats: COMPLETE Column stats: NONE
-                        Select Operator
-                          expressions: _col0 (type: string), _col6 (type: string)
-                          outputColumnNames: _col0, _col1
+                      Select Operator
+                        expressions: _col1 (type: string)
+                        outputColumnNames: _col1
+                        Statistics: Num rows: 1525 Data size: 16127 Basic stats: COMPLETE Column stats: NONE
+                        Map Join Operator
+                          condition map:
+                               Inner Join 0 to 1
+                          keys:
+                            0 _col1 (type: string)
+                            1 _col1 (type: string)
+                          outputColumnNames: _col0, _col6
+                          input vertices:
+                            0 Map 20
                           Statistics: Num rows: 1677 Data size: 17739 Basic stats: COMPLETE Column stats: NONE
-                          File Output Operator
-                            compressed: false
-                            Statistics: Num rows: 3409 Data size: 36062 Basic stats: COMPLETE Column stats: NONE
-                            table:
-                                input format: org.apache.hadoop.mapred.TextInputFormat
-                                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                                name: default.a
-                          File Output Operator
-                            compressed: false
-                            Statistics: Num rows: 3409 Data size: 36062 Basic stats: COMPLETE Column stats: NONE
-                            table:
-                                input format: org.apache.hadoop.mapred.TextInputFormat
-                                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                                name: default.b
-                          File Output Operator
-                            compressed: false
-                            Statistics: Num rows: 3409 Data size: 36062 Basic stats: COMPLETE Column stats: NONE
-                            table:
-                                input format: org.apache.hadoop.mapred.TextInputFormat
-                                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                                name: default.c
+                          Select Operator
+                            expressions: _col0 (type: string), _col6 (type: string)
+                            outputColumnNames: _col0, _col1
+                            Statistics: Num rows: 1677 Data size: 17739 Basic stats: COMPLETE Column stats: NONE
+                            File Output Operator
+                              compressed: false
+                              Statistics: Num rows: 3409 Data size: 36062 Basic stats: COMPLETE Column stats: NONE
+                              table:
+                                  input format: org.apache.hadoop.mapred.TextInputFormat
+                                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                                  name: default.a
+                            File Output Operator
+                              compressed: false
+                              Statistics: Num rows: 3409 Data size: 36062 Basic stats: COMPLETE Column stats: NONE
+                              table:
+                                  input format: org.apache.hadoop.mapred.TextInputFormat
+                                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                                  name: default.b
+                            File Output Operator
+                              compressed: false
+                              Statistics: Num rows: 3409 Data size: 36062 Basic stats: COMPLETE Column stats: NONE
+                              table:
+                                  input format: org.apache.hadoop.mapred.TextInputFormat
+                                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                                  name: default.c
             Execution mode: llap
             LLAP IO: no inputs
         Map 18 
@@ -3159,44 +3179,48 @@ STAGE PLANS:
                       expressions: value (type: string)
                       outputColumnNames: _col1
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                      Map Join Operator
-                        condition map:
-                             Inner Join 0 to 1
-                        keys:
-                          0 _col1 (type: string)
-                          1 _col1 (type: string)
-                        outputColumnNames: _col0, _col6
-                        input vertices:
-                          0 Map 20
-                        Statistics: Num rows: 1677 Data size: 17739 Basic stats: COMPLETE Column stats: NONE
-                        Select Operator
-                          expressions: _col0 (type: string), _col6 (type: string)
-                          outputColumnNames: _col0, _col1
+                      Select Operator
+                        expressions: _col1 (type: string)
+                        outputColumnNames: _col1
+                        Statistics: Num rows: 1525 Data size: 16127 Basic stats: COMPLETE Column stats: NONE
+                        Map Join Operator
+                          condition map:
+                               Inner Join 0 to 1
+                          keys:
+                            0 _col1 (type: string)
+                            1 _col1 (type: string)
+                          outputColumnNames: _col0, _col6
+                          input vertices:
+                            0 Map 20
                           Statistics: Num rows: 1677 Data size: 17739 Basic stats: COMPLETE Column stats: NONE
-                          File Output Operator
-                            compressed: false
-                            Statistics: Num rows: 3409 Data size: 36062 Basic stats: COMPLETE Column stats: NONE
-                            table:
-                                input format: org.apache.hadoop.mapred.TextInputFormat
-                                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                                name: default.a
-                          File Output Operator
-                            compressed: false
-                            Statistics: Num rows: 3409 Data size: 36062 Basic stats: COMPLETE Column stats: NONE
-                            table:
-                                input format: org.apache.hadoop.mapred.TextInputFormat
-                                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                                name: default.b
-                          File Output Operator
-                            compressed: false
-                            Statistics: Num rows: 3409 Data size: 36062 Basic stats: COMPLETE Column stats: NONE
-                            table:
-                                input format: org.apache.hadoop.mapred.TextInputFormat
-                                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                                name: default.c
+                          Select Operator
+                            expressions: _col0 (type: string), _col6 (type: string)
+                            outputColumnNames: _col0, _col1
+                            Statistics: Num rows: 1677 Data size: 17739 Basic stats: COMPLETE Column stats: NONE
+                            File Output Operator
+                              compressed: false
+                              Statistics: Num rows: 3409 Data size: 36062 Basic stats: COMPLETE Column stats: NONE
+                              table:
+                                  input format: org.apache.hadoop.mapred.TextInputFormat
+                                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                                  name: default.a
+                            File Output Operator
+                              compressed: false
+                              Statistics: Num rows: 3409 Data size: 36062 Basic stats: COMPLETE Column stats: NONE
+                              table:
+                                  input format: org.apache.hadoop.mapred.TextInputFormat
+                                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                                  name: default.b
+                            File Output Operator
+                              compressed: false
+                              Statistics: Num rows: 3409 Data size: 36062 Basic stats: COMPLETE Column stats: NONE
+                              table:
+                                  input format: org.apache.hadoop.mapred.TextInputFormat
+                                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                                  name: default.c
             Execution mode: llap
             LLAP IO: no inputs
         Map 19 
@@ -3211,44 +3235,48 @@ STAGE PLANS:
                       expressions: value (type: string)
                       outputColumnNames: _col1
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                      Map Join Operator
-                        condition map:
-                             Inner Join 0 to 1
-                        keys:
-                          0 _col1 (type: string)
-                          1 _col1 (type: string)
-                        outputColumnNames: _col0, _col6
-                        input vertices:
-                          0 Map 20
-                        Statistics: Num rows: 1677 Data size: 17739 Basic stats: COMPLETE Column stats: NONE
-                        Select Operator
-                          expressions: _col0 (type: string), _col6 (type: string)
-                          outputColumnNames: _col0, _col1
+                      Select Operator
+                        expressions: _col1 (type: string)
+                        outputColumnNames: _col1
+                        Statistics: Num rows: 1525 Data size: 16127 Basic stats: COMPLETE Column stats: NONE
+                        Map Join Operator
+                          condition map:
+                               Inner Join 0 to 1
+                          keys:
+                            0 _col1 (type: string)
+                            1 _col1 (type: string)
+                          outputColumnNames: _col0, _col6
+                          input vertices:
+                            0 Map 20
                           Statistics: Num rows: 1677 Data size: 17739 Basic stats: COMPLETE Column stats: NONE
-                          File Output Operator
-                            compressed: false
-                            Statistics: Num rows: 3409 Data size: 36062 Basic stats: COMPLETE Column stats: NONE
-                            table:
-                                input format: org.apache.hadoop.mapred.TextInputFormat
-                                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                                name: default.a
-                          File Output Operator
-                            compressed: false
-                            Statistics: Num rows: 3409 Data size: 36062 Basic stats: COMPLETE Column stats: NONE
-                            table:
-                                input format: org.apache.hadoop.mapred.TextInputFormat
-                                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                                name: default.b
-                          File Output Operator
-                            compressed: false
-                            Statistics: Num rows: 3409 Data size: 36062 Basic stats: COMPLETE Column stats: NONE
-                            table:
-                                input format: org.apache.hadoop.mapred.TextInputFormat
-                                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                                name: default.c
+                          Select Operator
+                            expressions: _col0 (type: string), _col6 (type: string)
+                            outputColumnNames: _col0, _col1
+                            Statistics: Num rows: 1677 Data size: 17739 Basic stats: COMPLETE Column stats: NONE
+                            File Output Operator
+                              compressed: false
+                              Statistics: Num rows: 3409 Data size: 36062 Basic stats: COMPLETE Column stats: NONE
+                              table:
+                                  input format: org.apache.hadoop.mapred.TextInputFormat
+                                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                                  name: default.a
+                            File Output Operator
+                              compressed: false
+                              Statistics: Num rows: 3409 Data size: 36062 Basic stats: COMPLETE Column stats: NONE
+                              table:
+                                  input format: org.apache.hadoop.mapred.TextInputFormat
+                                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                                  name: default.b
+                            File Output Operator
+                              compressed: false
+                              Statistics: Num rows: 3409 Data size: 36062 Basic stats: COMPLETE Column stats: NONE
+                              table:
+                                  input format: org.apache.hadoop.mapred.TextInputFormat
+                                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                                  name: default.c
             Execution mode: llap
             LLAP IO: no inputs
         Map 20 
@@ -3323,11 +3351,15 @@ STAGE PLANS:
                       expressions: value (type: string)
                       outputColumnNames: _col1
                       Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                      Reduce Output Operator
-                        key expressions: _col1 (type: string)
-                        sort order: +
-                        Map-reduce partition columns: _col1 (type: string)
+                      Select Operator
+                        expressions: _col1 (type: string)
+                        outputColumnNames: _col1
                         Statistics: Num rows: 525 Data size: 5503 Basic stats: COMPLETE Column stats: NONE
+                        Reduce Output Operator
+                          key expressions: _col1 (type: string)
+                          sort order: +
+                          Map-reduce partition columns: _col1 (type: string)
+                          Statistics: Num rows: 525 Data size: 5503 Basic stats: COMPLETE Column stats: NONE
             Execution mode: llap
             LLAP IO: no inputs
         Map 6 
@@ -3384,11 +3416,15 @@ STAGE PLANS:
                       expressions: value (type: string)
                       outputColumnNames: _col1
                       Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
-                      Reduce Output Operator
-                        key expressions: _col1 (type: string)
-                        sort order: +
-                        Map-reduce partition columns: _col1 (type: string)
+                      Select Operator
+                        expressions: _col1 (type: string)
+                        outputColumnNames: _col1
                         Statistics: Num rows: 1025 Data size: 10815 Basic stats: COMPLETE Column stats: NONE
+                        Reduce Output Operator
+                          key expressions: _col1 (type: string)
+                          sort order: +
+                          Map-reduce partition columns: _col1 (type: string)
+                          Statistics: Num rows: 1025 Data size: 10815 Basic stats: COMPLETE Column stats: NONE
             Execution mode: llap
             LLAP IO: no inputs
         Reducer 10 
@@ -4075,15 +4111,15 @@ Stage-4
           Dependency Collection{}
             Stage-2
               Reducer 4 llap
-              File Output Operator [FS_18]
+              File Output Operator [FS_20]
                 table:{"name:":"default.dest1"}
-                Group By Operator [GBY_16] (rows=1 width=96)
+                Group By Operator [GBY_18] (rows=1 width=96)
                   Output:["_col0","_col1"],aggregations:["count(DISTINCT KEY._col1:0._col0)"],keys:KEY._col0
                 <-Union 3 [SIMPLE_EDGE]
                   <-Map 6 [CONTAINS] llap
-                    Reduce Output Operator [RS_15]
+                    Reduce Output Operator [RS_17]
                       PartitionCols:_col0
-                      Group By Operator [GBY_14] (rows=1 width=280)
+                      Group By Operator [GBY_16] (rows=1 width=280)
                         Output:["_col0","_col1","_col2"],aggregations:["count(DISTINCT substr(_col1, 5))"],keys:_col0, substr(_col1, 5)
                         Select Operator [SEL_9] (rows=501 width=272)
                           Output:["_col0","_col1"]
@@ -4091,29 +4127,29 @@ Stage-4
                             Output:["_col0","_col1"]
                             TableScan [TS_6] (rows=500 width=10)
                               Output:["key","value"]
-                    Reduce Output Operator [RS_21]
+                    Reduce Output Operator [RS_23]
                       PartitionCols:_col0, _col1
-                      Group By Operator [GBY_20] (rows=1 width=464)
+                      Group By Operator [GBY_22] (rows=1 width=464)
                         Output:["_col0","_col1","_col2","_col3"],aggregations:["count(DISTINCT substr(_col1, 5))"],keys:_col0, _col1, substr(_col1, 5)
                          Please refer to the previous Select Operator [SEL_9]
                   <-Map 7 [CONTAINS] llap
-                    Reduce Output Operator [RS_15]
+                    Reduce Output Operator [RS_17]
                       PartitionCols:_col0
-                      Group By Operator [GBY_14] (rows=1 width=280)
+                      Group By Operator [GBY_16] (rows=1 width=280)
                         Output:["_col0","_col1","_col2"],aggregations:["count(DISTINCT substr(_col1, 5))"],keys:_col0, substr(_col1, 5)
-                        Select Operator [SEL_11] (rows=500 width=10)
+                        Select Operator [SEL_12] (rows=500 width=10)
                           Output:["_col0","_col1"]
-                          TableScan [TS_10] (rows=500 width=10)
+                          TableScan [TS_11] (rows=500 width=10)
                             Output:["key","value"]
-                    Reduce Output Operator [RS_21]
+                    Reduce Output Operator [RS_23]
                       PartitionCols:_col0, _col1
-                      Group By Operator [GBY_20] (rows=1 width=464)
+                      Group By Operator [GBY_22] (rows=1 width=464)
                         Output:["_col0","_col1","_col2","_col3"],aggregations:["count(DISTINCT substr(_col1, 5))"],keys:_col0, _col1, substr(_col1, 5)
-                         Please refer to the previous Select Operator [SEL_11]
+                         Please refer to the previous Select Operator [SEL_12]
                   <-Reducer 2 [CONTAINS] llap
-                    Reduce Output Operator [RS_15]
+                    Reduce Output Operator [RS_17]
                       PartitionCols:_col0
-                      Group By Operator [GBY_14] (rows=1 width=280)
+                      Group By Operator [GBY_16] (rows=1 width=280)
                         Output:["_col0","_col1","_col2"],aggregations:["count(DISTINCT substr(_col1, 5))"],keys:_col0, substr(_col1, 5)
                         Select Operator [SEL_9] (rows=501 width=272)
                           Output:["_col0","_col1"]
@@ -4128,15 +4164,15 @@ Stage-4
                                   Select Operator [SEL_1] (rows=500 width=10)
                                     TableScan [TS_0] (rows=500 width=10)
                                       default@src,s1,Tbl:COMPLETE,Col:COMPLETE
-                    Reduce Output Operator [RS_21]
+                    Reduce Output Operator [RS_23]
                       PartitionCols:_col0, _col1
-                      Group By Operator [GBY_20] (rows=1 width=464)
+                      Group By Operator [GBY_22] (rows=1 width=464)
                         Output:["_col0","_col1","_col2","_col3"],aggregations:["count(DISTINCT substr(_col1, 5))"],keys:_col0, _col1, substr(_col1, 5)
                          Please refer to the previous Select Operator [SEL_9]
               Reducer 5 llap
-              File Output Operator [FS_24]
+              File Output Operator [FS_26]
                 table:{"name:":"default.dest2"}
-                Group By Operator [GBY_22] (rows=1 width=280)
+                Group By Operator [GBY_24] (rows=1 width=280)
                   Output:["_col0","_col1","_col2"],aggregations:["count(DISTINCT KEY._col2:0._col0)"],keys:KEY._col0, KEY._col1
                 <- Please refer to the previous Union 3 [SIMPLE_EDGE]
 Stage-5
@@ -4183,15 +4219,15 @@ Stage-4
           Dependency Collection{}
             Stage-2
               Reducer 4 llap
-              File Output Operator [FS_14]
+              File Output Operator [FS_15]
                 table:{"name:":"default.dest1"}
-                Group By Operator [GBY_12] (rows=1 width=96)
+                Group By Operator [GBY_13] (rows=1 width=96)
                   Output:["_col0","_col1"],aggregations:["count(DISTINCT KEY._col1:0._col0)"],keys:KEY._col0
                 <-Union 3 [SIMPLE_EDGE]
                   <-Map 6 [CONTAINS] llap
-                    Reduce Output Operator [RS_11]
+                    Reduce Output Operator [RS_12]
                       PartitionCols:_col0
-                      Group By Operator [GBY_10] (rows=1 width=280)
+                      Group By Operator [GBY_11] (rows=1 width=280)
                         Output:["_col0","_col1","_col2"],aggregations:["count(DISTINCT substr(_col1, 5))"],keys:_col0, substr(_col1, 5)
                         Select Operator [SEL_9] (rows=501 width=11)
                           Output:["_col0","_col1"]
@@ -4199,17 +4235,15 @@ Stage-4
                             Output:["_col0","_col1"]
                             TableScan [TS_6] (rows=500 width=10)
                               Output:["key","value"]
-                    Reduce Output Operator [RS_17]
+                    Reduce Output Operator [RS_18]
                       PartitionCols:_col0, _col1
-                      Group By Operator [GBY_16] (rows=1 width=464)
+                      Group By Operator [GBY_17] (rows=1 width=464)
                         Output:["_col0","_col1","_col2","_col3"],aggregations:["count(DISTINCT substr(_col1, 5))"],keys:_col0, _col1, substr(_col1, 5)
-                        Select Operator [SEL_15] (rows=501 width=11)
-                          Output:["_col0","_col1"]
-                           Please refer to the previous Select Operator [SEL_7]
+                         Please refer to the previous Select Operator [SEL_9]
                   <-Reducer 2 [CONTAINS] llap
-                    Reduce Output Operator [RS_11]
+                    Reduce Output Operator [RS_12]
                       PartitionCols:_col0
-                      Group By Operator [GBY_10] (rows=1 width=280)
+                      Group By Operator [GBY_11] (rows=1 width=280)
                         Output:["_col0","_col1","_col2"],aggregations:["count(DISTINCT substr(_col1, 5))"],keys:_col0, substr(_col1, 5)
                         Select Operator [SEL_9] (rows=501 width=11)
                           Output:["_col0","_col1"]
@@ -4224,17 +4258,15 @@ Stage-4
                                   Select Operator [SEL_1] (rows=500 width=10)
                                     TableScan [TS_0] (rows=500 width=10)
                                       default@src,s1,Tbl:COMPLETE,Col:COMPLETE
-                    Reduce Output Operator [RS_17]
+                    Reduce Output Operator [RS_18]
                       PartitionCols:_col0, _col1
-                      Group By Operator [GBY_16] (rows=1 width=464)
+                      Group By Operator [GBY_17] (rows=1 width=464)
                         Output:["_col0","_col1","_col2","_col3"],aggregations:["count(DISTINCT substr(_col1, 5))"],keys:_col0, _col1, substr(_col1, 5)
-                        Select Operator [SEL_15] (rows=501 width=11)
-                          Output:["_col0","_col1"]
-                           Please refer to the previous Select Operator [SEL_5]
+                         Please refer to the previous Select Operator [SEL_9]
               Reducer 5 llap
-              File Output Operator [FS_20]
+              File Output Operator [FS_21]
                 table:{"name:":"default.dest2"}
-                Group By Operator [GBY_18] (rows=1 width=280)
+                Group By Operator [GBY_19] (rows=1 width=280)
                   Output:["_col0","_col1","_col2"],aggregations:["count(DISTINCT KEY._col2:0._col0)"],keys:KEY._col0, KEY._col1
                 <- Please refer to the previous Union 3 [SIMPLE_EDGE]
 Stage-5

http://git-wip-us.apache.org/repos/asf/hive/blob/2435e702/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 5f99e2a..1775a3f 100644
--- a/ql/src/test/results/clientpositive/optimize_nullscan.q.out
+++ b/ql/src/test/results/clientpositive/optimize_nullscan.q.out
@@ -1551,7 +1551,7 @@ STAGE PLANS:
                 tag: 1
                 auto parallelism: false
       Path -> Alias:
-        nullscan://null/default.src/part_ [null-subquery2:a-subquery2:src]
+        nullscan://null/default.src/part_ [a-subquery2:_u1-subquery2:src]
       Path -> Partition:
         nullscan://null/default.src/part_ 
           Partition
@@ -1597,7 +1597,7 @@ STAGE PLANS:
               name: default.src
             name: default.src
       Truncated Path -> Alias:
-        nullscan://null/default.src/part_ [null-subquery2:a-subquery2:src]
+        nullscan://null/default.src/part_ [a-subquery2:_u1-subquery2:src]
       Needs Tagging: true
       Reduce Operator Tree:
         Join Operator
@@ -1693,7 +1693,7 @@ STAGE PLANS:
                 MultiFileSpray: false
       Path -> Alias:
 #### A masked pattern was here ####
-        nullscan://null/default.src/part_ [null-subquery1:a-subquery1:src]
+        nullscan://null/default.src/part_ [a-subquery1:_u1-subquery1:src]
       Path -> Partition:
 #### A masked pattern was here ####
           Partition
@@ -1760,7 +1760,7 @@ STAGE PLANS:
             name: default.src
       Truncated Path -> Alias:
 #### A masked pattern was here ####
-        nullscan://null/default.src/part_ [null-subquery1:a-subquery1:src]
+        nullscan://null/default.src/part_ [a-subquery1:_u1-subquery1:src]
 
   Stage: Stage-0
     Fetch Operator


[14/22] hive git commit: HIVE-14875: Enhancement and refactoring of TestLdapAtnProviderWithMiniDS (Illya Yalovyy, reviewed by Aihua Xu)

Posted by se...@apache.org.
HIVE-14875: Enhancement and refactoring of TestLdapAtnProviderWithMiniDS (Illya Yalovyy, reviewed by Aihua Xu)


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

Branch: refs/heads/hive-14535
Commit: 66f1f9bf70eb6c3587472b13196b0847f47d09b2
Parents: d729b454
Author: Aihua Xu <ai...@apache.org>
Authored: Mon Oct 10 11:05:35 2016 -0400
Committer: Aihua Xu <ai...@apache.org>
Committed: Mon Oct 10 11:05:35 2016 -0400

----------------------------------------------------------------------
 .../auth/TestLdapAtnProviderWithMiniDS.java     | 349 +++++--------------
 .../src/test/resources/ldap/example.com.ldif    |  96 +++++
 2 files changed, 184 insertions(+), 261 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/66f1f9bf/service/src/test/org/apache/hive/service/auth/TestLdapAtnProviderWithMiniDS.java
----------------------------------------------------------------------
diff --git a/service/src/test/org/apache/hive/service/auth/TestLdapAtnProviderWithMiniDS.java b/service/src/test/org/apache/hive/service/auth/TestLdapAtnProviderWithMiniDS.java
index 23a048a..8c723cf 100644
--- a/service/src/test/org/apache/hive/service/auth/TestLdapAtnProviderWithMiniDS.java
+++ b/service/src/test/org/apache/hive/service/auth/TestLdapAtnProviderWithMiniDS.java
@@ -20,41 +20,28 @@
 
 package org.apache.hive.service.auth;
 
-import java.io.ByteArrayOutputStream;
-import java.io.File;
-import java.io.FileOutputStream;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Iterator;
 
-import javax.naming.NamingEnumeration;
-import javax.naming.ldap.LdapContext;
 import javax.security.sasl.AuthenticationException;
 
-import static org.apache.directory.server.integ.ServerIntegrationUtils.getWiredContext;
 import org.apache.directory.server.annotations.CreateLdapServer;
 import org.apache.directory.server.annotations.CreateTransport;
-import org.apache.directory.server.core.annotations.ApplyLdifs;
+import org.apache.directory.server.core.annotations.ApplyLdifFiles;
 import org.apache.directory.server.core.annotations.ContextEntry;
 import org.apache.directory.server.core.annotations.CreateDS;
 import org.apache.directory.server.core.annotations.CreateIndex;
 import org.apache.directory.server.core.annotations.CreatePartition;
 import org.apache.directory.server.core.integ.AbstractLdapTestUnit;
 import org.apache.directory.server.core.integ.FrameworkRunner;
-import org.apache.directory.server.ldap.LdapServer;
 
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hive.service.auth.LdapAuthenticationProviderImpl;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
-import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 
@@ -65,146 +52,34 @@ import org.junit.runner.RunWith;
  *
  */
 @RunWith(FrameworkRunner.class)
-@CreateLdapServer(transports =
-    { @CreateTransport(protocol = "LDAP"), @CreateTransport(protocol = "LDAPS") })
-// Define the DirectoryService
-@CreateDS(
-partitions = {
-    @CreatePartition(
-        name = "example",
-        suffix = "dc=example,dc=com",
-        contextEntry = @ContextEntry(
-            entryLdif = "dn: dc=example,dc=com\n" +
-            "dc: example\n" +
-            "objectClass: top\n" +
-            "objectClass: domain\n\n"
-        ),
-        indexes = {
-            @CreateIndex( attribute = "objectClass" ),
-            @CreateIndex( attribute = "dc" ),
-            @CreateIndex( attribute = "ou"),
-            @CreateIndex( attribute = "distinguishedName")
-        } )
-    }
-)
-
-@ApplyLdifs(
-    {
-      "dn: ou=People,dc=example,dc=com",
-      "distinguishedName: ou=People,dc=example,dc=com",
-      "objectClass: top",
-      "objectClass: organizationalUnit",
-      "objectClass: ExtensibleObject",
-      "ou: People",
-      "description: Contains entries which describe persons (seamen)",
-
-      "dn: ou=Groups,dc=example,dc=com",
-      "distinguishedName: ou=Groups,dc=example,dc=com",
-      "objectClass: top",
-      "objectClass: organizationalUnit",
-      "objectClass: ExtensibleObject",
-      "ou: Groups",
-      "description: Contains entries which describe groups (crews, for instance)",
-
-      "dn: uid=group1,ou=Groups,dc=example,dc=com",
-      "distinguishedName: uid=group1,ou=Groups,dc=example,dc=com",
-      "objectClass: top",
-      "objectClass: groupOfNames",
-      "objectClass: ExtensibleObject",
-      "cn: group1",
-      "ou: Groups",
-      "sn: group1",
-      "member: uid=user1,ou=People,dc=example,dc=com",
-
-      "dn: uid=group2,ou=Groups,dc=example,dc=com",
-      "distinguishedName: uid=group2,ou=Groups,dc=example,dc=com",
-      "objectClass: top",
-      "objectClass: groupOfNames",
-      "objectClass: ExtensibleObject",
-      "givenName: Group2",
-      "ou: Groups",
-      "cn: group2",
-      "sn: group2",
-      "member: uid=user2,ou=People,dc=example,dc=com",
-
-      "dn: cn=group3,ou=Groups,dc=example,dc=com",
-      "distinguishedName: cn=group3,ou=Groups,dc=example,dc=com",
-      "objectClass: top",
-      "objectClass: groupOfNames",
-      "objectClass: ExtensibleObject",
-      "cn: group3",
-      "ou: Groups",
-      "sn: group3",
-      "member: cn=user3,ou=People,dc=example,dc=com",
-
-      "dn: cn=group4,ou=Groups,dc=example,dc=com",
-      "distinguishedName: cn=group4,ou=Groups,dc=example,dc=com",
-      "objectClass: top",
-      "objectClass: groupOfUniqueNames",
-      "objectClass: ExtensibleObject",
-      "ou: Groups",
-      "cn: group4",
-      "sn: group4",
-      "uniqueMember: cn=user4,ou=People,dc=example,dc=com",
-
-      "dn: uid=user1,ou=People,dc=example,dc=com",
-      "distinguishedName: uid=user1,ou=People,dc=example,dc=com",
-      "objectClass: inetOrgPerson",
-      "objectClass: person",
-      "objectClass: top",
-      "objectClass: ExtensibleObject",
-      "givenName: Test1",
-      "cn: Test User1",
-      "sn: user1",
-      "uid: user1",
-      "userPassword: user1",
-
-      "dn: uid=user2,ou=People,dc=example,dc=com",
-      "distinguishedName: uid=user2,ou=People,dc=example,dc=com",
-      "objectClass: inetOrgPerson",
-      "objectClass: person",
-      "objectClass: top",
-      "objectClass: ExtensibleObject",
-      "givenName: Test2",
-      "cn: Test User2",
-      "sn: user2",
-      "uid: user2",
-      "userPassword: user2",
-
-      "dn: cn=user3,ou=People,dc=example,dc=com",
-      "distinguishedName: cn=user3,ou=People,dc=example,dc=com",
-      "objectClass: inetOrgPerson",
-      "objectClass: person",
-      "objectClass: top",
-      "objectClass: ExtensibleObject",
-      "givenName: Test1",
-      "cn: Test User3",
-      "sn: user3",
-      "uid: user3",
-      "userPassword: user3",
-
-      "dn: cn=user4,ou=People,dc=example,dc=com",
-      "distinguishedName: cn=user4,ou=People,dc=example,dc=com",
-      "objectClass: inetOrgPerson",
-      "objectClass: person",
-      "objectClass: top",
-      "objectClass: ExtensibleObject",
-      "givenName: Test4",
-      "cn: Test User4",
-      "sn: user4",
-      "uid: user4",
-      "userPassword: user4"
+@CreateLdapServer(transports = {
+  @CreateTransport(protocol = "LDAP"),
+  @CreateTransport(protocol = "LDAPS")
+})
 
+@CreateDS(partitions = {
+  @CreatePartition(
+      name = "example",
+      suffix = "dc=example,dc=com",
+      contextEntry = @ContextEntry(entryLdif =
+          "dn: dc=example,dc=com\n" +
+          "dc: example\n" +
+          "objectClass: top\n" +
+          "objectClass: domain\n\n"
+      ),
+    indexes = {
+      @CreateIndex(attribute = "objectClass"),
+      @CreateIndex(attribute = "cn"),
+      @CreateIndex(attribute = "uid")
+    }
+  )
 })
 
+@ApplyLdifFiles("ldap/example.com.ldif")
 public class TestLdapAtnProviderWithMiniDS extends AbstractLdapTestUnit {
 
-  private static String ldapUrl;
-  private static LdapServer server;
-  private static HiveConf hiveConf;
-  private static byte[] hiveConfBackup;
-  private static LdapContext ctx;
   private static LdapAuthenticationProviderImpl ldapProvider;
+  Map<String, String> ldapProperties;
 
   static final User USER1 = new User("user1", "user1", "uid=user1,ou=People,dc=example,dc=com");
   static final User USER2 = new User("user2", "user2", "uid=user2,ou=People,dc=example,dc=com");
@@ -213,18 +88,12 @@ public class TestLdapAtnProviderWithMiniDS extends AbstractLdapTestUnit {
 
   @Before
   public void setup() throws Exception {
-    ctx = ( LdapContext ) getWiredContext( ldapServer, null ).lookup( "dc=example,dc=com" );
-  }
-
-  @After
-  public void shutdown() throws Exception {
+    ldapProperties = new HashMap<>();
   }
 
   @BeforeClass
   public static void init() throws Exception {
-    hiveConf = new HiveConf();
-
-    ldapProvider = new LdapAuthenticationProviderImpl(hiveConf);
+    ldapProvider = new LdapAuthenticationProviderImpl(new HiveConf());
   }
 
   @AfterClass
@@ -234,28 +103,15 @@ public class TestLdapAtnProviderWithMiniDS extends AbstractLdapTestUnit {
     }
   }
 
-  private static void initLdapAtn(Map<String, String> hiveProperties)
-        throws Exception {
-    hiveConf = new HiveConf();
-
-    int port;
-    if (ldapUrl == null) {
-      port = ldapServer.getPort();
-      ldapUrl = new String("ldap://localhost:" + port);
-    }
+  private void initLdapAtn() throws Exception {
+    String ldapUrl = "ldap://localhost:" + ldapServer.getPort();
 
+    HiveConf hiveConf = new HiveConf();
     hiveConf.set("hive.root.logger", "DEBUG,console");
     hiveConf.set("hive.server2.authentication.ldap.url", ldapUrl);
 
-    if (hiveProperties != null) {
-      String key;
-      String value;
-      Iterator<String> iter = hiveProperties.keySet().iterator();
-      while (iter.hasNext()) {
-        key = iter.next();
-        value = hiveProperties.get(key);
-        hiveConf.set(key, value);
-      }
+    for (Map.Entry<String, String> entry : ldapProperties.entrySet()) {
+      hiveConf.set(entry.getKey(), entry.getValue());
     }
 
     ldapProvider = new LdapAuthenticationProviderImpl(hiveConf);
@@ -263,17 +119,16 @@ public class TestLdapAtnProviderWithMiniDS extends AbstractLdapTestUnit {
 
   @Test
   public void testLDAPServer() throws Exception {
-    initLdapAtn(null);
+    initLdapAtn();
     assertTrue(ldapServer.isStarted());
     assertTrue(ldapServer.getPort() > 0);
   }
 
   @Test
   public void testUserBindPositiveWithShortname() throws Exception {
-    Map<String, String> ldapProperties = new HashMap<String, String>();
     ldapProperties.put("hive.server2.authentication.ldap.userDNPattern", "uid=%s,ou=People,dc=example,dc=com");
     ldapProperties.put("hive.server2.authentication.ldap.groupDNPattern", "uid=%s,ou=Groups,dc=example,dc=com");
-    initLdapAtn(ldapProperties);
+    initLdapAtn();
     String user;
 
     user = USER1.getUID();
@@ -297,9 +152,8 @@ public class TestLdapAtnProviderWithMiniDS extends AbstractLdapTestUnit {
 
   @Test
   public void testUserBindPositiveWithShortnameOldConfig() throws Exception {
-    Map<String, String> ldapProperties = new HashMap<String, String>();
     ldapProperties.put("hive.server2.authentication.ldap.baseDN", "ou=People,dc=example,dc=com");
-    initLdapAtn(ldapProperties);
+    initLdapAtn();
     String user;
 
     user = USER1.getUID();
@@ -323,10 +177,9 @@ public class TestLdapAtnProviderWithMiniDS extends AbstractLdapTestUnit {
 
   @Test
   public void testUserBindNegativeWithShortname() throws Exception {
-    Map<String, String> ldapProperties = new HashMap<String, String>();
     ldapProperties.put("hive.server2.authentication.ldap.userDNPattern", "uid=%s,ou=People,dc=example,dc=com");
     ldapProperties.put("hive.server2.authentication.ldap.groupDNPattern", "uid=%s,ou=Groups,dc=example,dc=com");
-    initLdapAtn(ldapProperties);
+    initLdapAtn();
 
     try {
       ldapProvider.Authenticate(USER1.getUID(), USER2.getPassword());
@@ -346,9 +199,8 @@ public class TestLdapAtnProviderWithMiniDS extends AbstractLdapTestUnit {
 
   @Test
   public void testUserBindNegativeWithShortnameOldConfig() throws Exception {
-    Map<String, String> ldapProperties = new HashMap<String, String>();
     ldapProperties.put("hive.server2.authentication.ldap.baseDN", "ou=People,dc=example,dc=com");
-    initLdapAtn(ldapProperties);
+    initLdapAtn();
 
     try {
       ldapProvider.Authenticate(USER1.getUID(), USER2.getPassword());
@@ -369,11 +221,9 @@ public class TestLdapAtnProviderWithMiniDS extends AbstractLdapTestUnit {
   @Test
   public void testUserBindPositiveWithDN() throws Exception {
     String user;
-    Map<String, String> ldapProperties = new HashMap<String, String>();
-
     ldapProperties.put("hive.server2.authentication.ldap.userDNPattern", "uid=%s,ou=People,dc=example,dc=com");
     ldapProperties.put("hive.server2.authentication.ldap.groupDNPattern", "uid=%s,ou=Groups,dc=example,dc=com");
-    initLdapAtn(ldapProperties);
+    initLdapAtn();
     assertTrue(ldapServer.getPort() > 0);
 
     user = USER1.getDN();
@@ -398,10 +248,8 @@ public class TestLdapAtnProviderWithMiniDS extends AbstractLdapTestUnit {
   @Test
   public void testUserBindPositiveWithDNOldConfig() throws Exception {
     String user;
-    Map<String, String> ldapProperties = new HashMap<String, String>();
-
     ldapProperties.put("hive.server2.authentication.ldap.baseDN", "ou=People,dc=example,dc=com");
-    initLdapAtn(ldapProperties);
+    initLdapAtn();
     assertTrue(ldapServer.getPort() > 0);
 
     user = USER1.getDN();
@@ -426,10 +274,8 @@ public class TestLdapAtnProviderWithMiniDS extends AbstractLdapTestUnit {
   @Test
   public void testUserBindPositiveWithDNWrongOldConfig() throws Exception {
     String user;
-    Map<String, String> ldapProperties = new HashMap<String, String>();
-
     ldapProperties.put("hive.server2.authentication.ldap.baseDN", "ou=DummyPeople,dc=example,dc=com");
-    initLdapAtn(ldapProperties);
+    initLdapAtn();
     assertTrue(ldapServer.getPort() > 0);
 
     user = USER1.getDN();
@@ -455,11 +301,9 @@ public class TestLdapAtnProviderWithMiniDS extends AbstractLdapTestUnit {
   @Test
   public void testUserBindPositiveWithDNWrongConfig() throws Exception {
     String user;
-    Map<String, String> ldapProperties = new HashMap<String, String>();
-
     ldapProperties.put("hive.server2.authentication.ldap.userDNPattern", "uid=%s,ou=DummyPeople,dc=example,dc=com");
     ldapProperties.put("hive.server2.authentication.ldap.groupDNPattern", "uid=%s,ou=DummyGroups,dc=example,dc=com");
-    initLdapAtn(ldapProperties);
+    initLdapAtn();
     assertTrue(ldapServer.getPort() > 0);
 
     user = USER1.getDN();
@@ -484,11 +328,9 @@ public class TestLdapAtnProviderWithMiniDS extends AbstractLdapTestUnit {
   @Test
   public void testUserBindPositiveWithDNBlankConfig() throws Exception {
     String user;
-    Map<String, String> ldapProperties = new HashMap<String, String>();
-
     ldapProperties.put("hive.server2.authentication.ldap.userDNPattern", " ");
     ldapProperties.put("hive.server2.authentication.ldap.groupDNPattern", " ");
-    initLdapAtn(ldapProperties);
+    initLdapAtn();
     assertTrue(ldapServer.getPort() > 0);
 
     user = USER1.getDN();
@@ -513,10 +355,8 @@ public class TestLdapAtnProviderWithMiniDS extends AbstractLdapTestUnit {
   @Test
   public void testUserBindPositiveWithDNBlankOldConfig() throws Exception {
     String user;
-    Map<String, String> ldapProperties = new HashMap<String, String>();
-
     ldapProperties.put("hive.server2.authentication.ldap.baseDN", "");
-    initLdapAtn(ldapProperties);
+    initLdapAtn();
     assertTrue(ldapServer.getPort() > 0);
 
     user = USER1.getDN();
@@ -541,10 +381,9 @@ public class TestLdapAtnProviderWithMiniDS extends AbstractLdapTestUnit {
   @Test
   public void testUserBindNegativeWithDN() throws Exception {
     String user;
-    Map<String, String> ldapProperties = new HashMap<String, String>();
     ldapProperties.put("hive.server2.authentication.ldap.userDNPattern", "uid=%s,ou=People,dc=example,dc=com");
     ldapProperties.put("hive.server2.authentication.ldap.groupDNPattern", "uid=%s,ou=Groups,dc=example,dc=com");
-    initLdapAtn(ldapProperties);
+    initLdapAtn();
     assertTrue(ldapServer.getPort() > 0);
 
     user = USER1.getDN();
@@ -569,9 +408,8 @@ public class TestLdapAtnProviderWithMiniDS extends AbstractLdapTestUnit {
   @Test
   public void testUserBindNegativeWithDNOldConfig() throws Exception {
     String user;
-    Map<String, String> ldapProperties = new HashMap<String, String>();
     ldapProperties.put("hive.server2.authentication.ldap.baseDN", "ou=People,dc=example,dc=com");
-    initLdapAtn(ldapProperties);
+    initLdapAtn();
     assertTrue(ldapServer.getPort() > 0);
 
     user = USER1.getDN();
@@ -596,10 +434,9 @@ public class TestLdapAtnProviderWithMiniDS extends AbstractLdapTestUnit {
   @Test
   public void testUserFilterPositive() throws Exception {
     String user;
-    Map<String, String> ldapProperties = new HashMap<String, String>();
     ldapProperties.put("hive.server2.authentication.ldap.userDNPattern", "uid=%s,ou=People,dc=example,dc=com");
     ldapProperties.put("hive.server2.authentication.ldap.userFilter", USER2.getUID());
-    initLdapAtn(ldapProperties);
+    initLdapAtn();
 
     user = USER2.getDN();
     try {
@@ -613,10 +450,10 @@ public class TestLdapAtnProviderWithMiniDS extends AbstractLdapTestUnit {
       Assert.fail("testUserFilterPositive: Authentication failed for " + user + ",user expected to pass userfilter");
     }
 
-    ldapProperties = new HashMap<String, String>();
+    ldapProperties = new HashMap<>();
     ldapProperties.put("hive.server2.authentication.ldap.userDNPattern", "uid=%s,ou=People,dc=example,dc=com");
     ldapProperties.put("hive.server2.authentication.ldap.userFilter", USER1.getUID());
-    initLdapAtn(ldapProperties);
+    initLdapAtn();
 
     try {
       user = USER1.getDN();
@@ -630,10 +467,10 @@ public class TestLdapAtnProviderWithMiniDS extends AbstractLdapTestUnit {
       Assert.fail("testUserFilterPositive: Authentication failed for " + user + ",user expected to pass userfilter");
     }
 
-    ldapProperties = new HashMap<String, String>();
+    ldapProperties = new HashMap<>();
     ldapProperties.put("hive.server2.authentication.ldap.userDNPattern", "uid=%s,ou=People,dc=example,dc=com");
     ldapProperties.put("hive.server2.authentication.ldap.userFilter", USER2.getUID() + "," + USER1.getUID());
-    initLdapAtn(ldapProperties);
+    initLdapAtn();
 
     try {
       user = USER1.getDN();
@@ -652,10 +489,9 @@ public class TestLdapAtnProviderWithMiniDS extends AbstractLdapTestUnit {
   @Test
   public void testUserFilterNegative() throws Exception {
     String user;
-    Map<String, String> ldapProperties = new HashMap<String, String>();
     ldapProperties.put("hive.server2.authentication.ldap.userDNPattern", "uid=%s,ou=People,dc=example,dc=com");
     ldapProperties.put("hive.server2.authentication.ldap.userFilter", USER2.getUID());
-    initLdapAtn(ldapProperties);
+    initLdapAtn();
 
     user = USER1.getDN();
     try {
@@ -673,10 +509,10 @@ public class TestLdapAtnProviderWithMiniDS extends AbstractLdapTestUnit {
       assertTrue("testUserFilterNegative: Authentication failed for " + user + " as expected", true);
     }
 
-    ldapProperties = new HashMap<String, String>();
+    ldapProperties = new HashMap<>();
     ldapProperties.put("hive.server2.authentication.ldap.userDNPattern", "uid=%s,ou=People,dc=example,dc=com");
     ldapProperties.put("hive.server2.authentication.ldap.userFilter", USER1.getUID());
-    initLdapAtn(ldapProperties);
+    initLdapAtn();
 
     user = USER2.getDN();
     try {
@@ -694,10 +530,10 @@ public class TestLdapAtnProviderWithMiniDS extends AbstractLdapTestUnit {
       assertTrue("testUserFilterNegative: Authentication failed for " + user + " as expected", true);
     }
 
-    ldapProperties = new HashMap<String, String>();
+    ldapProperties = new HashMap<>();
     ldapProperties.put("hive.server2.authentication.ldap.userDNPattern", "uid=%s,ou=People,dc=example,dc=com");
     ldapProperties.put("hive.server2.authentication.ldap.userFilter", USER3.getUID());
-    initLdapAtn(ldapProperties);
+    initLdapAtn();
 
     user = USER1.getUID();
     try {
@@ -719,11 +555,10 @@ public class TestLdapAtnProviderWithMiniDS extends AbstractLdapTestUnit {
   @Test
   public void testGroupFilterPositive() throws Exception {
     String user;
-    Map<String, String> ldapProperties = new HashMap<String, String>();
     ldapProperties.put("hive.server2.authentication.ldap.userDNPattern", "uid=%s,ou=People,dc=example,dc=com");
     ldapProperties.put("hive.server2.authentication.ldap.groupDNPattern", "uid=%s,ou=Groups,dc=example,dc=com");
     ldapProperties.put("hive.server2.authentication.ldap.groupFilter", "group1,group2");
-    initLdapAtn(ldapProperties);
+    initLdapAtn();
 
     user = USER1.getDN();
     try {
@@ -741,11 +576,11 @@ public class TestLdapAtnProviderWithMiniDS extends AbstractLdapTestUnit {
       Assert.fail("testGroupFilterPositive: Authentication failed for " + user + ",user expected to pass groupfilter");
     }
 
-    ldapProperties = new HashMap<String, String>();
+    ldapProperties = new HashMap<>();
     ldapProperties.put("hive.server2.authentication.ldap.userDNPattern", "uid=%s,ou=People,dc=example,dc=com");
     ldapProperties.put("hive.server2.authentication.ldap.groupDNPattern", "uid=%s,ou=Groups,dc=example,dc=com");
     ldapProperties.put("hive.server2.authentication.ldap.groupFilter", "group2");
-    initLdapAtn(ldapProperties);
+    initLdapAtn();
 
     user = USER2.getDN();
     try {
@@ -759,11 +594,10 @@ public class TestLdapAtnProviderWithMiniDS extends AbstractLdapTestUnit {
   @Test
   public void testGroupFilterNegative() throws Exception {
     String user;
-    Map<String, String> ldapProperties = new HashMap<String, String>();
     ldapProperties.put("hive.server2.authentication.ldap.userDNPattern", "uid=%s,ou=People,dc=example,dc=com");
     ldapProperties.put("hive.server2.authentication.ldap.groupDNPattern", "uid=%s,ou=Groups,dc=example,dc=com");
     ldapProperties.put("hive.server2.authentication.ldap.groupFilter", "group1");
-    initLdapAtn(ldapProperties);
+    initLdapAtn();
 
     user = USER2.getDN();
     try {
@@ -773,11 +607,11 @@ public class TestLdapAtnProviderWithMiniDS extends AbstractLdapTestUnit {
       assertTrue("testGroupFilterNegative: Authentication failed for " + user + " as expected", true);
     }
 
-    ldapProperties = new HashMap<String, String>();
+    ldapProperties = new HashMap<>();
     ldapProperties.put("hive.server2.authentication.ldap.userDNPattern", "uid=%s,ou=People,dc=example,dc=com");
     ldapProperties.put("hive.server2.authentication.ldap.groupDNPattern", "uid=%s,ou=Groups,dc=example,dc=com");
     ldapProperties.put("hive.server2.authentication.ldap.groupFilter", "group2");
-    initLdapAtn(ldapProperties);
+    initLdapAtn();
 
     user = USER1.getDN();
     try {
@@ -791,12 +625,11 @@ public class TestLdapAtnProviderWithMiniDS extends AbstractLdapTestUnit {
   @Test
   public void testUserAndGroupFilterPositive() throws Exception {
     String user;
-    Map<String, String> ldapProperties = new HashMap<String, String>();
     ldapProperties.put("hive.server2.authentication.ldap.userDNPattern", "uid=%s,ou=People,dc=example,dc=com");
     ldapProperties.put("hive.server2.authentication.ldap.groupDNPattern", "uid=%s,ou=Groups,dc=example,dc=com");
     ldapProperties.put("hive.server2.authentication.ldap.userFilter", USER1.getUID() + "," + USER2.getUID());
     ldapProperties.put("hive.server2.authentication.ldap.groupFilter", "group1,group2");
-    initLdapAtn(ldapProperties);
+    initLdapAtn();
 
     user = USER1.getDN();
     try {
@@ -823,12 +656,11 @@ public class TestLdapAtnProviderWithMiniDS extends AbstractLdapTestUnit {
   @Test
   public void testUserAndGroupFilterNegative() throws Exception {
     String user;
-    Map<String, String> ldapProperties = new HashMap<String, String>();
     ldapProperties.put("hive.server2.authentication.ldap.userDNPattern", "uid=%s,ou=People,dc=example,dc=com");
     ldapProperties.put("hive.server2.authentication.ldap.groupDNPattern", "uid=%s,ou=Groups,dc=example,dc=com");
     ldapProperties.put("hive.server2.authentication.ldap.userFilter", USER1.getUID() + "," + USER2.getUID());
     ldapProperties.put("hive.server2.authentication.ldap.groupFilter", "group1");
-    initLdapAtn(ldapProperties);
+    initLdapAtn();
 
     user = USER2.getDN();
     try {
@@ -850,13 +682,12 @@ public class TestLdapAtnProviderWithMiniDS extends AbstractLdapTestUnit {
   @Test
   public void testCustomQueryPositive() throws Exception {
     String user;
-    Map<String, String> ldapProperties = new HashMap<String, String>();
     ldapProperties.put("hive.server2.authentication.ldap.baseDN", "ou=People,dc=example,dc=com");
     ldapProperties.put("hive.server2.authentication.ldap.userDNPattern", "cn=%s,ou=People,dc=example,dc=com:uid=%s,ou=People,dc=example,dc=com");
     ldapProperties.put("hive.server2.authentication.ldap.groupDNPattern", "cn=%s,ou=People,dc=example,dc=com");
     ldapProperties.put("hive.server2.authentication.ldap.customLDAPQuery", "(&(objectClass=person)(|(uid="
                        + USER1.getUID() + ")(uid=" + USER4.getUID() + ")))");
-    initLdapAtn(ldapProperties);
+    initLdapAtn();
 
     user = USER1.getDN();
     try {
@@ -878,12 +709,11 @@ public class TestLdapAtnProviderWithMiniDS extends AbstractLdapTestUnit {
   @Test
   public void testCustomQueryNegative() throws Exception {
     String user;
-    Map<String, String> ldapProperties = new HashMap<String, String>();
     ldapProperties.put("hive.server2.authentication.ldap.baseDN", "ou=People,dc=example,dc=com");
     // ldap query will only return user1
     ldapProperties.put("hive.server2.authentication.ldap.customLDAPQuery", "(&(objectClass=person)(uid="
                        + USER1.getUID() + "))");
-    initLdapAtn(ldapProperties);
+    initLdapAtn();
 
     user = USER2.getDN();
     try {
@@ -912,12 +742,11 @@ public class TestLdapAtnProviderWithMiniDS extends AbstractLdapTestUnit {
   @Test
   public void testCustomQueryWithGroupsPositive() throws Exception {
     String user;
-    Map<String, String> ldapProperties = new HashMap<String, String>();
     ldapProperties.put("hive.server2.authentication.ldap.baseDN", "dc=example,dc=com");
     ldapProperties.put("hive.server2.authentication.ldap.userDNPattern", "cn=%s,ou=People,dc=example,dc=com:uid=%s,ou=People,dc=example,dc=com");
     ldapProperties.put("hive.server2.authentication.ldap.customLDAPQuery",
                          "(&(objectClass=groupOfNames)(|(cn=group1)(cn=group2)))");
-    initLdapAtn(ldapProperties);
+    initLdapAtn();
 
     user = USER1.getDN();
     try {
@@ -940,7 +769,7 @@ public class TestLdapAtnProviderWithMiniDS extends AbstractLdapTestUnit {
     // following query should return group1 and user2
     ldapProperties.put("hive.server2.authentication.ldap.customLDAPQuery",
                          "(|(&(objectClass=groupOfNames)(cn=group1))(&(objectClass=person)(sn=user4)))");
-    initLdapAtn(ldapProperties);
+    initLdapAtn();
 
     user = USER1.getUID();
     try {
@@ -959,7 +788,7 @@ public class TestLdapAtnProviderWithMiniDS extends AbstractLdapTestUnit {
     ldapProperties.put("hive.server2.authentication.ldap.groupMembershipKey", "uniqueMember");
     ldapProperties.put("hive.server2.authentication.ldap.customLDAPQuery",
                          "(&(objectClass=groupOfUniqueNames)(cn=group4))");
-    initLdapAtn(ldapProperties);
+    initLdapAtn();
 
     user = USER4.getDN();
     try {
@@ -977,12 +806,11 @@ public class TestLdapAtnProviderWithMiniDS extends AbstractLdapTestUnit {
   @Test
   public void testCustomQueryWithGroupsNegative() throws Exception {
     String user;
-    Map<String, String> ldapProperties = new HashMap<String, String>();
     ldapProperties.put("hive.server2.authentication.ldap.baseDN", "dc=example,dc=com");
     ldapProperties.put("hive.server2.authentication.ldap.userDNPattern", "cn=%s,ou=People,dc=example,dc=com:uid=%s,ou=People,dc=example,dc=com");
     ldapProperties.put("hive.server2.authentication.ldap.customLDAPQuery",
                          "(&(objectClass=groupOfNames)(|(cn=group1)(cn=group2)))");
-    initLdapAtn(ldapProperties);
+    initLdapAtn();
 
     user = USER3.getDN();
     try {
@@ -1004,12 +832,11 @@ public class TestLdapAtnProviderWithMiniDS extends AbstractLdapTestUnit {
   @Test
   public void testGroupFilterPositiveWithCustomGUID() throws Exception {
     String user;
-    Map<String, String> ldapProperties = new HashMap<String, String>();
     ldapProperties.put("hive.server2.authentication.ldap.userDNPattern", "cn=%s,ou=People,dc=example,dc=com");
     ldapProperties.put("hive.server2.authentication.ldap.groupDNPattern", "cn=%s,ou=Groups,dc=example,dc=com");
     ldapProperties.put("hive.server2.authentication.ldap.guidKey", "cn");
     ldapProperties.put("hive.server2.authentication.ldap.groupFilter", "group3");
-    initLdapAtn(ldapProperties);
+    initLdapAtn();
 
     user = USER3.getDN();
     try {
@@ -1027,14 +854,13 @@ public class TestLdapAtnProviderWithMiniDS extends AbstractLdapTestUnit {
   @Test
   public void testGroupFilterPositiveWithCustomAttributes() throws Exception {
     String user;
-    Map<String, String> ldapProperties = new HashMap<String, String>();
     ldapProperties.put("hive.server2.authentication.ldap.userDNPattern", "cn=%s,ou=People,dc=example,dc=com");
     ldapProperties.put("hive.server2.authentication.ldap.groupDNPattern", "cn=%s,ou=Groups,dc=example,dc=com");
     ldapProperties.put("hive.server2.authentication.ldap.groupFilter", "group4");
     ldapProperties.put("hive.server2.authentication.ldap.guidKey", "cn");
     ldapProperties.put("hive.server2.authentication.ldap.groupMembershipKey", "uniqueMember");
     ldapProperties.put("hive.server2.authentication.ldap.groupClassKey", "groupOfUniqueNames");
-    initLdapAtn(ldapProperties);
+    initLdapAtn();
 
     user = USER4.getDN();
     try {
@@ -1049,28 +875,29 @@ public class TestLdapAtnProviderWithMiniDS extends AbstractLdapTestUnit {
     }
 
   }
-}
 
-class User {
-  String uid;
-  String pwd;
-  String ldapDN;
+  private static class User {
+    String uid;
+    String pwd;
+    String ldapDN;
 
-  User(String uid, String password, String ldapDN) {
-    this.uid    = uid;
-    this.pwd    = password;
-    this.ldapDN = ldapDN;
-  }
+    User(String uid, String password, String ldapDN) {
+      this.uid    = uid;
+      this.pwd    = password;
+      this.ldapDN = ldapDN;
+    }
 
-  public String getUID() {
-    return uid;
-  }
+    public String getUID() {
+      return uid;
+    }
 
-  public String getPassword() {
-    return pwd;
-  }
+    public String getPassword() {
+      return pwd;
+    }
 
-  public String getDN() {
-    return ldapDN;
+    public String getDN() {
+      return ldapDN;
+    }
   }
 }
+

http://git-wip-us.apache.org/repos/asf/hive/blob/66f1f9bf/service/src/test/resources/ldap/example.com.ldif
----------------------------------------------------------------------
diff --git a/service/src/test/resources/ldap/example.com.ldif b/service/src/test/resources/ldap/example.com.ldif
new file mode 100644
index 0000000..a7717ca
--- /dev/null
+++ b/service/src/test/resources/ldap/example.com.ldif
@@ -0,0 +1,96 @@
+dn: ou=People,dc=example,dc=com
+objectClass: top
+objectClass: organizationalUnit
+ou: People
+description: Contains entries which describe persons (seamen)
+
+dn: ou=Groups,dc=example,dc=com
+objectClass: top
+objectClass: organizationalUnit
+ou: Groups
+description: Contains entries which describe groups (crews, for instance)
+
+dn: uid=group1,ou=Groups,dc=example,dc=com
+objectClass: top
+objectClass: groupOfNames
+objectClass: uidObject
+uid: group1
+cn: group1
+ou: Groups
+member: uid=user1,ou=People,dc=example,dc=com
+
+dn: uid=group2,ou=Groups,dc=example,dc=com
+objectClass: top
+objectClass: groupOfNames
+objectClass: uidObject
+uid: group2
+cn: group2
+ou: Groups
+member: uid=user2,ou=People,dc=example,dc=com
+
+dn: cn=group3,ou=Groups,dc=example,dc=com
+objectClass: top
+objectClass: groupOfNames
+objectClass: uidObject
+uid: group3
+cn: group3
+ou: Groups
+member: cn=user3,ou=People,dc=example,dc=com
+
+dn: cn=group4,ou=Groups,dc=example,dc=com
+objectClass: top
+objectClass: groupOfUniqueNames
+objectClass: uidObject
+uid: group4
+ou: Groups
+cn: group4
+uniqueMember: cn=user4,ou=People,dc=example,dc=com
+
+dn: uid=user1,ou=People,dc=example,dc=com
+objectClass: top
+objectClass: person
+objectClass: organizationalPerson
+objectClass: inetOrgPerson
+objectClass: uidObject
+givenName: Test1
+cn: Test User1
+sn: user1
+uid: user1
+userPassword: user1
+
+dn: uid=user2,ou=People,dc=example,dc=com
+objectClass: top
+objectClass: person
+objectClass: organizationalPerson
+objectClass: inetOrgPerson
+objectClass: uidObject
+givenName: Test2
+cn: Test User2
+sn: user2
+uid: user2
+userPassword: user2
+
+dn: cn=user3,ou=People,dc=example,dc=com
+objectClass: top
+objectClass: person
+objectClass: organizationalPerson
+objectClass: inetOrgPerson
+objectClass: uidObject
+givenName: Test3
+cn: Test User3
+sn: user3
+uid: user3
+userPassword: user3
+
+dn: cn=user4,ou=People,dc=example,dc=com
+objectClass: top
+objectClass: person
+objectClass: organizationalPerson
+objectClass: inetOrgPerson
+objectClass: uidObject
+givenName: Test4
+cn: Test User4
+sn: user4
+uid: user4
+userPassword: user4
+


[20/22] hive git commit: HIVE-14690: Query fail when hive.exec.parallel=true, with conflicting session dir (Daniel Dai, reviewed by Thejas Nair)

Posted by se...@apache.org.
HIVE-14690: Query fail when hive.exec.parallel=true, with conflicting session dir (Daniel Dai, reviewed by Thejas Nair)


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

Branch: refs/heads/hive-14535
Commit: 9feed2f61aeb91d34b8c4231ca1d6149d81c8a02
Parents: f4a857c
Author: Daniel Dai <da...@hortonworks.com>
Authored: Tue Oct 11 15:53:26 2016 -0700
Committer: Daniel Dai <da...@hortonworks.com>
Committed: Tue Oct 11 15:53:26 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/ql/session/SessionState.java  | 11 ++++++++++-
 1 file changed, 10 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/9feed2f6/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java b/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
index d23a51f..71aea3a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
@@ -144,6 +144,10 @@ public class SessionState {
    */
   private boolean isUsingThriftJDBCBinarySerDe = false;
 
+  /**
+   * The flag to indicate if the session already started so we can skip the init
+   */
+  private boolean isStarted = false;
   /*
    * HiveHistory Object
    */
@@ -547,9 +551,14 @@ public class SessionState {
     startSs.tezSessionState.endOpen();
   }
 
-  private static void start(SessionState startSs, boolean isAsync, LogHelper console) {
+  synchronized private static void start(SessionState startSs, boolean isAsync, LogHelper console) {
     setCurrentSessionState(startSs);
 
+    if (startSs.isStarted) {
+      return;
+    }
+    startSs.isStarted = true;
+
     if (startSs.hiveHist == null){
       if (startSs.getConf().getBoolVar(HiveConf.ConfVars.HIVE_SESSION_HISTORY_ENABLED)) {
         startSs.hiveHist = new HiveHistoryImpl(startSs);


[03/22] hive git commit: HIVE-14099: Hive security authorization can be disabled by users (Aihua Xu, reviewed by Yongzhi Chen)

Posted by se...@apache.org.
HIVE-14099: Hive security authorization can be disabled by users (Aihua Xu, reviewed by Yongzhi Chen)


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

Branch: refs/heads/hive-14535
Commit: 96bcee86c8df1c1a511fe70c1a37840d996abb9e
Parents: 54ff3f5
Author: Aihua Xu <ai...@apache.org>
Authored: Wed Oct 5 10:19:33 2016 -0400
Committer: Aihua Xu <ai...@apache.org>
Committed: Wed Oct 5 10:19:33 2016 -0400

----------------------------------------------------------------------
 common/src/java/org/apache/hadoop/hive/conf/HiveConf.java | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/96bcee86/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index 4c3ef3e..5ea9751 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -3116,8 +3116,9 @@ public class HiveConf extends Configuration {
 
 
     HIVE_CONF_RESTRICTED_LIST("hive.conf.restricted.list",
-        "hive.security.authenticator.manager,hive.security.authorization.manager,hive.users.in.admin.role," +
-        "hive.server2.xsrf.filter.enabled",
+        "hive.security.authenticator.manager,hive.security.authorization.manager," +
+        "hive.security.metastore.authorization.manager,hive.security.metastore.authenticator.manager," +
+        "hive.users.in.admin.role,hive.server2.xsrf.filter.enabled,hive.security.authorization.enabled",
         "Comma separated list of configuration options which are immutable at runtime"),
     HIVE_CONF_HIDDEN_LIST("hive.conf.hidden.list",
         METASTOREPWD.varname + "," + HIVE_SERVER2_SSL_KEYSTORE_PASSWORD.varname


[16/22] hive git commit: HIVE-14917: explainanalyze_2.q fails after HIVE-14861

Posted by se...@apache.org.
HIVE-14917: explainanalyze_2.q fails after HIVE-14861


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

Branch: refs/heads/hive-14535
Commit: 1876723702bbad67a43f1ce80b4902c0c426b1ba
Parents: 66f1f9b
Author: Pengcheng Xiong <px...@apache.org>
Authored: Mon Oct 10 10:07:40 2016 -0700
Committer: Pengcheng Xiong <px...@apache.org>
Committed: Mon Oct 10 10:07:40 2016 -0700

----------------------------------------------------------------------
 .../queries/clientpositive/explainanalyze_2.q   |  162 -
 .../clientpositive/tez/explainanalyze_2.q.out   | 3706 +++---------------
 2 files changed, 562 insertions(+), 3306 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/18767237/ql/src/test/queries/clientpositive/explainanalyze_2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/explainanalyze_2.q b/ql/src/test/queries/clientpositive/explainanalyze_2.q
index dfee826..8b3df87 100644
--- a/ql/src/test/queries/clientpositive/explainanalyze_2.q
+++ b/ql/src/test/queries/clientpositive/explainanalyze_2.q
@@ -1,140 +1,4 @@
 set hive.explain.user=true;
-set hive.metastore.aggregate.stats.cache.enabled=false;
-
--- SORT_QUERY_RESULTS
-
-CREATE TABLE dest_j1(key STRING, value STRING, val2 STRING) STORED AS TEXTFILE;
-
-CREATE TABLE ss(k1 STRING,v1 STRING,k2 STRING,v2 STRING,k3 STRING,v3 STRING) STORED AS TEXTFILE;
-
-CREATE TABLE sr(k1 STRING,v1 STRING,k2 STRING,v2 STRING,k3 STRING,v3 STRING) STORED AS TEXTFILE;
-
-CREATE TABLE cs(k1 STRING,v1 STRING,k2 STRING,v2 STRING,k3 STRING,v3 STRING) STORED AS TEXTFILE;
-
-INSERT OVERWRITE TABLE ss
-SELECT x.key,x.value,y.key,y.value,z.key,z.value
-FROM src1 x 
-JOIN src y ON (x.key = y.key) 
-JOIN srcpart z ON (x.value = z.value and z.ds='2008-04-08' and z.hr=11);
-
-INSERT OVERWRITE TABLE sr
-SELECT x.key,x.value,y.key,y.value,z.key,z.value
-FROM src1 x 
-JOIN src y ON (x.key = y.key) 
-JOIN srcpart z ON (x.value = z.value and z.ds='2008-04-08' and z.hr=12);
-
-INSERT OVERWRITE TABLE cs
-SELECT x.key,x.value,y.key,y.value,z.key,z.value
-FROM src1 x 
-JOIN src y ON (x.key = y.key) 
-JOIN srcpart z ON (x.value = z.value and z.ds='2008-04-08');
-
-
-ANALYZE TABLE ss COMPUTE STATISTICS;
-ANALYZE TABLE ss COMPUTE STATISTICS FOR COLUMNS k1,v1,k2,v2,k3,v3;
-
-ANALYZE TABLE sr COMPUTE STATISTICS;
-ANALYZE TABLE sr COMPUTE STATISTICS FOR COLUMNS k1,v1,k2,v2,k3,v3;
-
-ANALYZE TABLE cs COMPUTE STATISTICS;
-ANALYZE TABLE cs COMPUTE STATISTICS FOR COLUMNS k1,v1,k2,v2,k3,v3;
-
-set hive.auto.convert.join=false;
-
-explain analyze
-SELECT x.key, z.value, y.value
-FROM src1 x JOIN src y ON (x.key = y.key) 
-JOIN srcpart z ON (x.value = z.value and z.ds='2008-04-08' and z.hr=11);
-
-explain analyze
-select 
-ss.k1,sr.k2,cs.k3,count(ss.v1),count(sr.v2),count(cs.v3)
-FROM 
-ss,sr,cs,src d1,src d2,src d3,src1,srcpart
-where
-    ss.k1 = d1.key 
-and sr.k1 = d2.key 
-and cs.k1 = d3.key 
-and ss.k2 = sr.k2
-and ss.k3 = sr.k3
-and ss.v1 = src1.value
-and ss.v2 = srcpart.value
-and sr.v2 = cs.v2
-and sr.v3 = cs.v3
-and ss.v3='ssv3'
-and sr.v1='srv1'
-and src1.key = 'src1key'
-and srcpart.key = 'srcpartkey'
-and d1.value = 'd1value'
-and d2.value in ('2000Q1','2000Q2','2000Q3')
-and d3.value in ('2000Q1','2000Q2','2000Q3')
-group by 
-ss.k1,sr.k2,cs.k3
-order by 
-ss.k1,sr.k2,cs.k3
-limit 100;
-
-explain analyze
-SELECT x.key, z.value, y.value
-FROM src1 x JOIN src y ON (x.key = y.key) 
-JOIN (select * from src1 union select * from src)z ON (x.value = z.value)
-union
-SELECT x.key, z.value, y.value
-FROM src1 x JOIN src y ON (x.key = y.key) 
-JOIN (select * from src1 union select * from src)z ON (x.value = z.value);
-
-explain analyze
-SELECT x.key, y.value
-FROM src1 x JOIN src y ON (x.key = y.key) 
-JOIN (select * from src1 union select * from src)z ON (x.value = z.value)
-union
-SELECT x.key, y.value
-FROM src1 x JOIN src y ON (x.key = y.key) 
-JOIN (select key, value from src1 union select key, value from src union select key, value from src)z ON (x.value = z.value)
-union
-SELECT x.key, y.value
-FROM src1 x JOIN src y ON (x.key = y.key) 
-JOIN (select key, value from src1 union select key, value from src union select key, value from src union select key, value from src)z ON (x.value = z.value);
-
-
-set hive.auto.convert.join=true;
-set hive.auto.convert.join.noconditionaltask=true;
-set hive.auto.convert.join.noconditionaltask.size=10000;
-set hive.stats.fetch.column.stats=false;
-
-
-explain analyze
-SELECT x.key, z.value, y.value
-FROM src1 x JOIN src y ON (x.key = y.key) 
-JOIN srcpart z ON (x.value = z.value and z.ds='2008-04-08' and z.hr=11);
-
-explain analyze
-select 
-ss.k1,sr.k2,cs.k3,count(ss.v1),count(sr.v2),count(cs.v3)
-FROM 
-ss,sr,cs,src d1,src d2,src d3,src1,srcpart
-where
-    ss.k1 = d1.key 
-and sr.k1 = d2.key 
-and cs.k1 = d3.key 
-and ss.k2 = sr.k2
-and ss.k3 = sr.k3
-and ss.v1 = src1.value
-and ss.v2 = srcpart.value
-and sr.v2 = cs.v2
-and sr.v3 = cs.v3
-and ss.v3='ssv3'
-and sr.v1='srv1'
-and src1.key = 'src1key'
-and srcpart.key = 'srcpartkey'
-and d1.value = 'd1value'
-and d2.value in ('2000Q1','2000Q2','2000Q3')
-and d3.value in ('2000Q1','2000Q2','2000Q3')
-group by 
-ss.k1,sr.k2,cs.k3
-order by 
-ss.k1,sr.k2,cs.k3
-limit 100;
 
 explain analyze
 SELECT x.key, z.value, y.value
@@ -219,32 +83,6 @@ UNION  ALL
 select s2.key as key, s2.value as value from tab s2
 ) a join tab_part b on (a.key = b.key);
 
-explain analyze
-SELECT x.key, y.value
-FROM src1 x JOIN src y ON (x.key = y.key) 
-JOIN (select * from src1 union all select * from src)z ON (x.value = z.value)
-union all
-SELECT x.key, y.value
-FROM src x JOIN src y ON (x.key = y.key) 
-JOIN (select key, value from src1 union all select key, value from src union all select key, value from src)z ON (x.value = z.value)
-union all
-SELECT x.key, y.value
-FROM src1 x JOIN src1 y ON (x.key = y.key) 
-JOIN (select key, value from src1 union all select key, value from src union all select key, value from src union all select key, value from src)z ON (x.value = z.value);
-
-explain analyze
-SELECT x.key, y.value
-FROM src1 x JOIN src y ON (x.key = y.key) 
-JOIN (select * from src1 union select * from src)z ON (x.value = z.value)
-union
-SELECT x.key, y.value
-FROM src x JOIN src y ON (x.key = y.key) 
-JOIN (select key, value from src1 union select key, value from src union select key, value from src)z ON (x.value = z.value)
-union
-SELECT x.key, y.value
-FROM src1 x JOIN src1 y ON (x.key = y.key) 
-JOIN (select key, value from src1 union select key, value from src union select key, value from src union select key, value from src)z ON (x.value = z.value);
-
 CREATE TABLE a(key STRING, value STRING) STORED AS TEXTFILE;
 CREATE TABLE b(key STRING, value STRING) STORED AS TEXTFILE;
 CREATE TABLE c(key STRING, value STRING) STORED AS TEXTFILE;


[02/22] hive git commit: HIVE-14858: Analyze command should support custom input formats (Chao Sun, reviewed by Xuefu Zhang and Prasanth Jayachandran)

Posted by se...@apache.org.
HIVE-14858: Analyze command should support custom input formats (Chao Sun, reviewed by Xuefu Zhang and Prasanth Jayachandran)


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

Branch: refs/heads/hive-14535
Commit: 54ff3f56d495e24352624de2655be69d433bd179
Parents: 481d7cd
Author: Chao Sun <su...@apache.org>
Authored: Thu Sep 29 13:33:17 2016 -0700
Committer: Chao Sun <su...@apache.org>
Committed: Wed Oct 5 12:33:36 2016 +0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/ql/optimizer/GenMRTableScan1.java     | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/54ff3f56/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRTableScan1.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRTableScan1.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRTableScan1.java
index d6f1b7a0..9297a0b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRTableScan1.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRTableScan1.java
@@ -91,8 +91,8 @@ public class GenMRTableScan1 implements NodeProcessor {
         if (parseCtx.getQueryProperties().isAnalyzeCommand()) {
           boolean partialScan = parseCtx.getQueryProperties().isPartialScanAnalyzeCommand();
           boolean noScan = parseCtx.getQueryProperties().isNoScanAnalyzeCommand();
-          if (inputFormat.equals(OrcInputFormat.class) ||
-                  inputFormat.equals(MapredParquetInputFormat.class)) {
+          if (OrcInputFormat.class.isAssignableFrom(inputFormat) ||
+                  MapredParquetInputFormat.class.isAssignableFrom(inputFormat)) {
             // For ORC and Parquet, all the following statements are the same
             // ANALYZE TABLE T [PARTITION (...)] COMPUTE STATISTICS
             // ANALYZE TABLE T [PARTITION (...)] COMPUTE STATISTICS partialscan;


[05/22] hive git commit: HIVE-14873: Add UDF for extraction of 'day of week' (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by se...@apache.org.
HIVE-14873: Add UDF for extraction of 'day of week' (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/59539885
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/59539885
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/59539885

Branch: refs/heads/hive-14535
Commit: 59539885725a96cca4b3f0759a5b26e0d8198dc8
Parents: 96bcee86
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Sat Oct 1 09:30:35 2016 +0100
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Wed Oct 5 17:51:41 2016 +0100

----------------------------------------------------------------------
 .../hadoop/hive/ql/exec/FunctionRegistry.java   |  18 +-
 .../expressions/VectorUDFDayOfWeekDate.java     |  39 +
 .../expressions/VectorUDFDayOfWeekString.java   |  61 ++
 .../VectorUDFDayOfWeekTimestamp.java            |  39 +
 .../hive/ql/optimizer/physical/Vectorizer.java  |   2 +
 .../org/apache/hadoop/hive/ql/parse/HiveLexer.g |   1 +
 .../hadoop/hive/ql/parse/IdentifiersParser.g    |   3 +-
 .../apache/hadoop/hive/ql/udf/UDFDayOfWeek.java | 105 +++
 ql/src/test/queries/clientpositive/extract.q    |   3 +
 .../clientpositive/vectorized_date_funcs.q      |   8 +-
 .../test/results/clientpositive/extract.q.out   |  11 +
 .../llap/vectorized_date_funcs.q.out            | 846 ++++++++++---------
 .../results/clientpositive/show_functions.q.out |   1 +
 .../clientpositive/vectorized_date_funcs.q.out  | 846 ++++++++++---------
 14 files changed, 1139 insertions(+), 844 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/59539885/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
index 6b29be1..0dbbc1d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
@@ -37,14 +37,6 @@ import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
 import org.apache.hadoop.hive.ql.session.SessionState;
-import org.apache.hadoop.hive.ql.udf.UDFDateFloorDay;
-import org.apache.hadoop.hive.ql.udf.UDFDateFloorHour;
-import org.apache.hadoop.hive.ql.udf.UDFDateFloorMinute;
-import org.apache.hadoop.hive.ql.udf.UDFDateFloorMonth;
-import org.apache.hadoop.hive.ql.udf.UDFDateFloorQuarter;
-import org.apache.hadoop.hive.ql.udf.UDFDateFloorSecond;
-import org.apache.hadoop.hive.ql.udf.UDFDateFloorWeek;
-import org.apache.hadoop.hive.ql.udf.UDFDateFloorYear;
 import org.apache.hadoop.hive.ql.udf.SettableUDF;
 import org.apache.hadoop.hive.ql.udf.UDAFPercentile;
 import org.apache.hadoop.hive.ql.udf.UDFAcos;
@@ -57,7 +49,16 @@ import org.apache.hadoop.hive.ql.udf.UDFChr;
 import org.apache.hadoop.hive.ql.udf.UDFConv;
 import org.apache.hadoop.hive.ql.udf.UDFCos;
 import org.apache.hadoop.hive.ql.udf.UDFCrc32;
+import org.apache.hadoop.hive.ql.udf.UDFDateFloorDay;
+import org.apache.hadoop.hive.ql.udf.UDFDateFloorHour;
+import org.apache.hadoop.hive.ql.udf.UDFDateFloorMinute;
+import org.apache.hadoop.hive.ql.udf.UDFDateFloorMonth;
+import org.apache.hadoop.hive.ql.udf.UDFDateFloorQuarter;
+import org.apache.hadoop.hive.ql.udf.UDFDateFloorSecond;
+import org.apache.hadoop.hive.ql.udf.UDFDateFloorWeek;
+import org.apache.hadoop.hive.ql.udf.UDFDateFloorYear;
 import org.apache.hadoop.hive.ql.udf.UDFDayOfMonth;
+import org.apache.hadoop.hive.ql.udf.UDFDayOfWeek;
 import org.apache.hadoop.hive.ql.udf.UDFDegrees;
 import org.apache.hadoop.hive.ql.udf.UDFE;
 import org.apache.hadoop.hive.ql.udf.UDFExp;
@@ -283,6 +284,7 @@ public final class FunctionRegistry {
 
     system.registerUDF("day", UDFDayOfMonth.class, false);
     system.registerUDF("dayofmonth", UDFDayOfMonth.class, false);
+    system.registerUDF("dayofweek", UDFDayOfWeek.class, false);
     system.registerUDF("month", UDFMonth.class, false);
     system.registerGenericUDF("quarter", GenericUDFQuarter.class);
     system.registerUDF("year", UDFYear.class, false);

http://git-wip-us.apache.org/repos/asf/hive/blob/59539885/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDayOfWeekDate.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDayOfWeekDate.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDayOfWeekDate.java
new file mode 100644
index 0000000..bd9c480
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDayOfWeekDate.java
@@ -0,0 +1,39 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector.expressions;
+
+import java.util.Calendar;
+
+/**
+ * Expression to get day of week.
+ * Extends {@link VectorUDFTimestampFieldDate}
+ */
+public final class VectorUDFDayOfWeekDate extends VectorUDFTimestampFieldDate {
+
+  private static final long serialVersionUID = 1L;
+
+  public VectorUDFDayOfWeekDate(int colNum, int outputColumn) {
+    super(Calendar.DAY_OF_WEEK, colNum, outputColumn);
+  }
+
+  public VectorUDFDayOfWeekDate() {
+    super();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/59539885/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDayOfWeekString.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDayOfWeekString.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDayOfWeekString.java
new file mode 100644
index 0000000..069d888
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDayOfWeekString.java
@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector.expressions;
+
+import java.nio.charset.CharacterCodingException;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.Calendar;
+import java.util.Date;
+
+import org.apache.hadoop.io.Text;
+
+/**
+ * Expression to get day of week.
+ * Extends {@link VectorUDFTimestampFieldString}
+ */
+public final class VectorUDFDayOfWeekString extends VectorUDFTimestampFieldString {
+
+  private static final long serialVersionUID = 1L;
+
+  private transient final SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd");
+  private transient final Calendar calendar = Calendar.getInstance();
+
+  public VectorUDFDayOfWeekString(int colNum, int outputColumn) {
+    super(colNum, outputColumn, -1, -1);
+  }
+
+  public VectorUDFDayOfWeekString() {
+    super();
+  }
+
+  @Override
+  protected long doGetField(byte[] bytes, int start, int length) throws ParseException {
+    Date date = null;
+    try {
+      String decoded = Text.decode(bytes, start, length);
+      date = format.parse(decoded);
+    } catch (CharacterCodingException e) {
+      throw new ParseException(e.getMessage(), 0);
+    }
+    calendar.setTime(date);
+    return calendar.get(Calendar.DAY_OF_WEEK);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/59539885/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDayOfWeekTimestamp.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDayOfWeekTimestamp.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDayOfWeekTimestamp.java
new file mode 100644
index 0000000..8e7c180
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDayOfWeekTimestamp.java
@@ -0,0 +1,39 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector.expressions;
+
+import java.util.Calendar;
+
+/**
+ * Expression to get day of week.
+ * Extends {@link VectorUDFTimestampFieldTimestamp}
+ */
+public final class VectorUDFDayOfWeekTimestamp extends VectorUDFTimestampFieldTimestamp {
+
+  private static final long serialVersionUID = 1L;
+
+  public VectorUDFDayOfWeekTimestamp(int colNum, int outputColumn) {
+    super(Calendar.DAY_OF_WEEK, colNum, outputColumn);
+  }
+
+  public VectorUDFDayOfWeekTimestamp() {
+    super();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/59539885/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
index 46bdba6..3a179a3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
@@ -129,6 +129,7 @@ import org.apache.hadoop.hive.ql.udf.UDFBin;
 import org.apache.hadoop.hive.ql.udf.UDFConv;
 import org.apache.hadoop.hive.ql.udf.UDFCos;
 import org.apache.hadoop.hive.ql.udf.UDFDayOfMonth;
+import org.apache.hadoop.hive.ql.udf.UDFDayOfWeek;
 import org.apache.hadoop.hive.ql.udf.UDFDegrees;
 import org.apache.hadoop.hive.ql.udf.UDFExp;
 import org.apache.hadoop.hive.ql.udf.UDFFromUnixTime;
@@ -265,6 +266,7 @@ public class Vectorizer implements PhysicalPlanResolver {
     supportedGenericUDFs.add(UDFYear.class);
     supportedGenericUDFs.add(UDFMonth.class);
     supportedGenericUDFs.add(UDFDayOfMonth.class);
+    supportedGenericUDFs.add(UDFDayOfWeek.class);
     supportedGenericUDFs.add(UDFHour.class);
     supportedGenericUDFs.add(UDFMinute.class);
     supportedGenericUDFs.add(UDFSecond.class);

http://git-wip-us.apache.org/repos/asf/hive/blob/59539885/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 58e76c8..5d3fa6a 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
@@ -307,6 +307,7 @@ KW_QUARTER: 'QUARTER';
 KW_MONTH: 'MONTH';
 KW_WEEK: 'WEEK';
 KW_DAY: 'DAY';
+KW_DOW: 'DAYOFWEEK';
 KW_HOUR: 'HOUR';
 KW_MINUTE: 'MINUTE';
 KW_SECOND: 'SECOND';

http://git-wip-us.apache.org/repos/asf/hive/blob/59539885/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 04f87b8..4a44173 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
@@ -282,6 +282,7 @@ timeQualifiers
     | KW_MONTH -> Identifier["month"]
     | KW_WEEK -> Identifier["weekofyear"]
     | KW_DAY -> Identifier["day"]
+    | KW_DOW -> Identifier["dayofweek"]
     | KW_HOUR -> Identifier["hour"]
     | KW_MINUTE -> Identifier["minute"]
     | KW_SECOND -> Identifier["second"]
@@ -696,7 +697,7 @@ nonReserved
     | KW_CASCADE | KW_CHANGE | KW_CLUSTER | KW_CLUSTERED | KW_CLUSTERSTATUS | KW_COLLECTION | KW_COLUMNS
     | KW_COMMENT | KW_COMPACT | KW_COMPACTIONS | KW_COMPUTE | KW_CONCATENATE | KW_CONTINUE | KW_DATA | KW_DAY
     | KW_DATABASES | KW_DATETIME | KW_DBPROPERTIES | KW_DEFERRED | KW_DEFINED | KW_DELIMITED | KW_DEPENDENCY 
-    | KW_DESC | KW_DIRECTORIES | KW_DIRECTORY | KW_DISABLE | KW_DISTRIBUTE | KW_ELEM_TYPE 
+    | KW_DESC | KW_DIRECTORIES | KW_DIRECTORY | KW_DISABLE | KW_DISTRIBUTE | KW_DOW | KW_ELEM_TYPE 
     | 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

http://git-wip-us.apache.org/repos/asf/hive/blob/59539885/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDayOfWeek.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDayOfWeek.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDayOfWeek.java
new file mode 100644
index 0000000..8c3e461
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDayOfWeek.java
@@ -0,0 +1,105 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.udf;
+
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.Calendar;
+import java.util.Date;
+
+import org.apache.hadoop.hive.ql.exec.Description;
+import org.apache.hadoop.hive.ql.exec.UDF;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedExpressions;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorUDFDayOfWeekDate;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorUDFDayOfWeekString;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorUDFDayOfWeekTimestamp;
+import org.apache.hadoop.hive.ql.udf.generic.NDV;
+import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.hadoop.hive.serde2.io.TimestampWritable;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.Text;
+
+/**
+ * UDFDayOfWeek.
+ *
+ */
+@Description(name = "dayofweek",
+    value = "_FUNC_(param) - Returns the day of the week of date/timestamp "
+    + "(1 = Sunday, 2 = Monday, ..., 7 = Saturday)",
+    extended = "param can be one of:\n"
+    + "1. A string in the format of 'yyyy-MM-dd HH:mm:ss' or 'yyyy-MM-dd'.\n"
+    + "2. A date value\n"
+    + "3. A timestamp value"
+    + "Example:\n "
+    + "  > SELECT _FUNC_('2009-07-30') FROM src LIMIT 1;\n" + "  5")
+@VectorizedExpressions({VectorUDFDayOfWeekDate.class, VectorUDFDayOfWeekString.class, VectorUDFDayOfWeekTimestamp.class})
+@NDV(maxNdv = 7)
+public class UDFDayOfWeek extends UDF {
+  private final SimpleDateFormat formatter = new SimpleDateFormat("yyyy-MM-dd");
+  private final Calendar calendar = Calendar.getInstance();
+
+  private final IntWritable result = new IntWritable();
+
+  public UDFDayOfWeek() {
+  }
+
+  /**
+   * Get the day of week from a date string.
+   *
+   * @param dateString
+   *          the dateString in the format of "yyyy-MM-dd HH:mm:ss" or
+   *          "yyyy-MM-dd".
+   * @return an int from 1 to 7. null if the dateString is not a valid date
+   *         string.
+   */
+  public IntWritable evaluate(Text dateString) {
+    if (dateString == null) {
+      return null;
+    }
+    try {
+      Date date = formatter.parse(dateString.toString());
+      calendar.setTime(date);
+      result.set(calendar.get(Calendar.DAY_OF_WEEK));
+      return result;
+    } catch (ParseException e) {
+      return null;
+    }
+  }
+
+  public IntWritable evaluate(DateWritable d) {
+    if (d == null) {
+      return null;
+    }
+
+    calendar.setTime(d.get(false)); // Time doesn't matter.
+    result.set(calendar.get(Calendar.DAY_OF_WEEK));
+    return result;
+  }
+
+  public IntWritable evaluate(TimestampWritable t) {
+    if (t == null) {
+      return null;
+    }
+
+    calendar.setTime(t.getTimestamp());
+    result.set(calendar.get(Calendar.DAY_OF_WEEK));
+    return result;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/59539885/ql/src/test/queries/clientpositive/extract.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/extract.q b/ql/src/test/queries/clientpositive/extract.q
index c09574b..70c1267 100644
--- a/ql/src/test/queries/clientpositive/extract.q
+++ b/ql/src/test/queries/clientpositive/extract.q
@@ -30,6 +30,9 @@ from extract_udf;
 select extract(hour from t)
 from extract_udf;
 
+select extract(dayofweek from t)
+from extract_udf;
+
 select extract(week from t)
 from extract_udf;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/59539885/ql/src/test/queries/clientpositive/vectorized_date_funcs.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/vectorized_date_funcs.q b/ql/src/test/queries/clientpositive/vectorized_date_funcs.q
index 59df32a..7d7b1cf 100644
--- a/ql/src/test/queries/clientpositive/vectorized_date_funcs.q
+++ b/ql/src/test/queries/clientpositive/vectorized_date_funcs.q
@@ -30,6 +30,7 @@ EXPLAIN SELECT
   month(fl_time),
   day(fl_time),
   dayofmonth(fl_time),
+  dayofweek(fl_time),
   weekofyear(fl_time),
   date(fl_time),
   to_date(fl_time),
@@ -44,6 +45,7 @@ SELECT
   month(fl_time),
   day(fl_time),
   dayofmonth(fl_time),
+  dayofweek(fl_time),
   weekofyear(fl_time),
   date(fl_time),
   to_date(fl_time),
@@ -58,6 +60,7 @@ EXPLAIN SELECT
   month(fl_date),
   day(fl_date),
   dayofmonth(fl_date),
+  dayofweek(fl_date),
   weekofyear(fl_date),
   date(fl_date),
   to_date(fl_date),
@@ -72,6 +75,7 @@ SELECT
   month(fl_date),
   day(fl_date),
   dayofmonth(fl_date),
+  dayofweek(fl_date),
   weekofyear(fl_date),
   date(fl_date),
   to_date(fl_date),
@@ -85,6 +89,7 @@ EXPLAIN SELECT
   month(fl_time) = month(fl_date),
   day(fl_time) = day(fl_date),
   dayofmonth(fl_time) = dayofmonth(fl_date),
+  dayofweek(fl_time) = dayofweek(fl_date),
   weekofyear(fl_time) = weekofyear(fl_date),
   date(fl_time) = date(fl_date),
   to_date(fl_time) = to_date(fl_date),
@@ -99,6 +104,7 @@ SELECT
   month(fl_time) = month(fl_date),
   day(fl_time) = day(fl_date),
   dayofmonth(fl_time) = dayofmonth(fl_date),
+  dayofweek(fl_time) = dayofweek(fl_date),
   weekofyear(fl_time) = weekofyear(fl_date),
   date(fl_time) = date(fl_date),
   to_date(fl_time) = to_date(fl_date),
@@ -142,4 +148,4 @@ SELECT
   count(fl_date),
   count(*)
 FROM date_udf_flight_orc
-ORDER BY c1;
\ No newline at end of file
+ORDER BY c1;

http://git-wip-us.apache.org/repos/asf/hive/blob/59539885/ql/src/test/results/clientpositive/extract.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/extract.q.out b/ql/src/test/results/clientpositive/extract.q.out
index 73c9bc6..aee4427 100644
--- a/ql/src/test/results/clientpositive/extract.q.out
+++ b/ql/src/test/results/clientpositive/extract.q.out
@@ -154,6 +154,17 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@extract_udf
 #### A masked pattern was here ####
 7
+PREHOOK: query: select extract(dayofweek from t)
+from extract_udf
+PREHOOK: type: QUERY
+PREHOOK: Input: default@extract_udf
+#### A masked pattern was here ####
+POSTHOOK: query: select extract(dayofweek from t)
+from extract_udf
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@extract_udf
+#### A masked pattern was here ####
+6
 PREHOOK: query: select extract(week from t)
 from extract_udf
 PREHOOK: type: QUERY

http://git-wip-us.apache.org/repos/asf/hive/blob/59539885/ql/src/test/results/clientpositive/llap/vectorized_date_funcs.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vectorized_date_funcs.q.out b/ql/src/test/results/clientpositive/llap/vectorized_date_funcs.q.out
index b18da27..f45e730 100644
--- a/ql/src/test/results/clientpositive/llap/vectorized_date_funcs.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorized_date_funcs.q.out
@@ -209,6 +209,7 @@ PREHOOK: query: EXPLAIN SELECT
   month(fl_time),
   day(fl_time),
   dayofmonth(fl_time),
+  dayofweek(fl_time),
   weekofyear(fl_time),
   date(fl_time),
   to_date(fl_time),
@@ -223,6 +224,7 @@ POSTHOOK: query: EXPLAIN SELECT
   month(fl_time),
   day(fl_time),
   dayofmonth(fl_time),
+  dayofweek(fl_time),
   weekofyear(fl_time),
   date(fl_time),
   to_date(fl_time),
@@ -242,8 +244,8 @@ STAGE PLANS:
         TableScan
           alias: date_udf_flight_orc
           Select Operator
-            expressions: to_unix_timestamp(fl_time) (type: bigint), year(fl_time) (type: int), month(fl_time) (type: int), day(fl_time) (type: int), dayofmonth(fl_time) (type: int), weekofyear(fl_time) (type: int), CAST( fl_time AS DATE) (type: date), to_date(fl_time) (type: date), date_add(fl_time, 2) (type: date), date_sub(fl_time, 2) (type: date), datediff(fl_time, '2000-01-01') (type: int)
-            outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
+            expressions: to_unix_timestamp(fl_time) (type: bigint), year(fl_time) (type: int), month(fl_time) (type: int), day(fl_time) (type: int), dayofmonth(fl_time) (type: int), dayofweek(fl_time) (type: int), weekofyear(fl_time) (type: int), CAST( fl_time AS DATE) (type: date), to_date(fl_time) (type: date), date_add(fl_time, 2) (type: date), date_sub(fl_time, 2) (type: date), datediff(fl_time, '2000-01-01') (type: int)
+            outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
             ListSink
 
 PREHOOK: query: SELECT
@@ -252,6 +254,7 @@ PREHOOK: query: SELECT
   month(fl_time),
   day(fl_time),
   dayofmonth(fl_time),
+  dayofweek(fl_time),
   weekofyear(fl_time),
   date(fl_time),
   to_date(fl_time),
@@ -268,6 +271,7 @@ POSTHOOK: query: SELECT
   month(fl_time),
   day(fl_time),
   dayofmonth(fl_time),
+  dayofweek(fl_time),
   weekofyear(fl_time),
   date(fl_time),
   to_date(fl_time),
@@ -278,149 +282,150 @@ FROM date_udf_flight_orc
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@date_udf_flight_orc
 #### A masked pattern was here ####
-1287583200	2010	10	20	20	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
-1287583200	2010	10	20	20	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
-1287583200	2010	10	20	20	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
-1287583200	2010	10	20	20	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
-1287583200	2010	10	20	20	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
-1287583200	2010	10	20	20	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
-1287583200	2010	10	20	20	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
-1287583200	2010	10	20	20	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
-1287583200	2010	10	20	20	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
-1287583200	2010	10	20	20	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
-1287583200	2010	10	20	20	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
-1287669600	2010	10	21	21	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
-1287669600	2010	10	21	21	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
-1287669600	2010	10	21	21	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
-1287669600	2010	10	21	21	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
-1287669600	2010	10	21	21	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
-1287669600	2010	10	21	21	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
-1287669600	2010	10	21	21	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
-1287669600	2010	10	21	21	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
-1287669600	2010	10	21	21	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
-1287669600	2010	10	21	21	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
-1287669600	2010	10	21	21	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
-1287669600	2010	10	21	21	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
-1287756000	2010	10	22	22	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
-1287756000	2010	10	22	22	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
-1287756000	2010	10	22	22	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
-1287756000	2010	10	22	22	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
-1287756000	2010	10	22	22	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
-1287756000	2010	10	22	22	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
-1287756000	2010	10	22	22	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
-1287756000	2010	10	22	22	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
-1287756000	2010	10	22	22	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
-1287756000	2010	10	22	22	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
-1287756000	2010	10	22	22	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
-1287842400	2010	10	23	23	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
-1287842400	2010	10	23	23	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
-1287842400	2010	10	23	23	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
-1287842400	2010	10	23	23	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
-1287842400	2010	10	23	23	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
-1287842400	2010	10	23	23	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
-1287842400	2010	10	23	23	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
-1287842400	2010	10	23	23	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
-1287842400	2010	10	23	23	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
-1287842400	2010	10	23	23	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
-1287842400	2010	10	23	23	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
-1287842400	2010	10	23	23	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
-1287928800	2010	10	24	24	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
-1287928800	2010	10	24	24	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
-1287928800	2010	10	24	24	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
-1287928800	2010	10	24	24	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
-1287928800	2010	10	24	24	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
-1287928800	2010	10	24	24	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
-1287928800	2010	10	24	24	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
-1287928800	2010	10	24	24	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
-1287928800	2010	10	24	24	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
-1287928800	2010	10	24	24	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
-1287928800	2010	10	24	24	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
-1287928800	2010	10	24	24	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
-1288015200	2010	10	25	25	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
-1288015200	2010	10	25	25	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
-1288015200	2010	10	25	25	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
-1288015200	2010	10	25	25	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
-1288015200	2010	10	25	25	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
-1288015200	2010	10	25	25	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
-1288015200	2010	10	25	25	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
-1288015200	2010	10	25	25	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
-1288015200	2010	10	25	25	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
-1288015200	2010	10	25	25	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
-1288015200	2010	10	25	25	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
-1288015200	2010	10	25	25	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
-1288101600	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288101600	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288101600	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288101600	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288101600	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288101600	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288101600	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288101600	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288101600	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288101600	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288101600	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288101600	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288101600	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288188000	2010	10	27	27	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
-1288188000	2010	10	27	27	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
-1288188000	2010	10	27	27	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
-1288188000	2010	10	27	27	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
-1288188000	2010	10	27	27	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
-1288188000	2010	10	27	27	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
-1288188000	2010	10	27	27	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
-1288188000	2010	10	27	27	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
-1288188000	2010	10	27	27	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
-1288188000	2010	10	27	27	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
-1288188000	2010	10	27	27	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
-1288274400	2010	10	28	28	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
-1288274400	2010	10	28	28	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
-1288274400	2010	10	28	28	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
-1288274400	2010	10	28	28	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
-1288274400	2010	10	28	28	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
-1288274400	2010	10	28	28	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
-1288274400	2010	10	28	28	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
-1288274400	2010	10	28	28	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
-1288274400	2010	10	28	28	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
-1288274400	2010	10	28	28	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
-1288274400	2010	10	28	28	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
-1288274400	2010	10	28	28	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
-1288360800	2010	10	29	29	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
-1288360800	2010	10	29	29	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
-1288360800	2010	10	29	29	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
-1288360800	2010	10	29	29	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
-1288360800	2010	10	29	29	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
-1288360800	2010	10	29	29	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
-1288360800	2010	10	29	29	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
-1288360800	2010	10	29	29	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
-1288360800	2010	10	29	29	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
-1288360800	2010	10	29	29	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
-1288360800	2010	10	29	29	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
-1288360800	2010	10	29	29	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
-1288447200	2010	10	30	30	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
-1288447200	2010	10	30	30	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
-1288447200	2010	10	30	30	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
-1288447200	2010	10	30	30	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
-1288447200	2010	10	30	30	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
-1288447200	2010	10	30	30	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
-1288447200	2010	10	30	30	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
-1288447200	2010	10	30	30	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
-1288447200	2010	10	30	30	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
-1288447200	2010	10	30	30	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
-1288447200	2010	10	30	30	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
-1288533600	2010	10	31	31	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
-1288533600	2010	10	31	31	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
-1288533600	2010	10	31	31	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
-1288533600	2010	10	31	31	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
-1288533600	2010	10	31	31	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
-1288533600	2010	10	31	31	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
-1288533600	2010	10	31	31	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
-1288533600	2010	10	31	31	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
+1287583200	2010	10	20	20	4	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
+1287583200	2010	10	20	20	4	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
+1287583200	2010	10	20	20	4	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
+1287583200	2010	10	20	20	4	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
+1287583200	2010	10	20	20	4	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
+1287583200	2010	10	20	20	4	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
+1287583200	2010	10	20	20	4	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
+1287583200	2010	10	20	20	4	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
+1287583200	2010	10	20	20	4	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
+1287583200	2010	10	20	20	4	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
+1287583200	2010	10	20	20	4	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
+1287669600	2010	10	21	21	5	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
+1287669600	2010	10	21	21	5	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
+1287669600	2010	10	21	21	5	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
+1287669600	2010	10	21	21	5	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
+1287669600	2010	10	21	21	5	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
+1287669600	2010	10	21	21	5	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
+1287669600	2010	10	21	21	5	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
+1287669600	2010	10	21	21	5	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
+1287669600	2010	10	21	21	5	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
+1287669600	2010	10	21	21	5	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
+1287669600	2010	10	21	21	5	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
+1287669600	2010	10	21	21	5	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
+1287756000	2010	10	22	22	6	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
+1287756000	2010	10	22	22	6	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
+1287756000	2010	10	22	22	6	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
+1287756000	2010	10	22	22	6	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
+1287756000	2010	10	22	22	6	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
+1287756000	2010	10	22	22	6	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
+1287756000	2010	10	22	22	6	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
+1287756000	2010	10	22	22	6	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
+1287756000	2010	10	22	22	6	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
+1287756000	2010	10	22	22	6	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
+1287756000	2010	10	22	22	6	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
+1287842400	2010	10	23	23	7	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
+1287842400	2010	10	23	23	7	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
+1287842400	2010	10	23	23	7	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
+1287842400	2010	10	23	23	7	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
+1287842400	2010	10	23	23	7	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
+1287842400	2010	10	23	23	7	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
+1287842400	2010	10	23	23	7	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
+1287842400	2010	10	23	23	7	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
+1287842400	2010	10	23	23	7	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
+1287842400	2010	10	23	23	7	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
+1287842400	2010	10	23	23	7	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
+1287842400	2010	10	23	23	7	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
+1287928800	2010	10	24	24	1	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
+1287928800	2010	10	24	24	1	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
+1287928800	2010	10	24	24	1	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
+1287928800	2010	10	24	24	1	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
+1287928800	2010	10	24	24	1	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
+1287928800	2010	10	24	24	1	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
+1287928800	2010	10	24	24	1	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
+1287928800	2010	10	24	24	1	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
+1287928800	2010	10	24	24	1	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
+1287928800	2010	10	24	24	1	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
+1287928800	2010	10	24	24	1	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
+1287928800	2010	10	24	24	1	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
+1288015200	2010	10	25	25	2	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
+1288015200	2010	10	25	25	2	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
+1288015200	2010	10	25	25	2	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
+1288015200	2010	10	25	25	2	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
+1288015200	2010	10	25	25	2	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
+1288015200	2010	10	25	25	2	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
+1288015200	2010	10	25	25	2	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
+1288015200	2010	10	25	25	2	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
+1288015200	2010	10	25	25	2	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
+1288015200	2010	10	25	25	2	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
+1288015200	2010	10	25	25	2	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
+1288015200	2010	10	25	25	2	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
+1288101600	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288101600	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288101600	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288101600	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288101600	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288101600	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288101600	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288101600	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288101600	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288101600	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288101600	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288101600	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288101600	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288188000	2010	10	27	27	4	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
+1288188000	2010	10	27	27	4	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
+1288188000	2010	10	27	27	4	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
+1288188000	2010	10	27	27	4	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
+1288188000	2010	10	27	27	4	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
+1288188000	2010	10	27	27	4	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
+1288188000	2010	10	27	27	4	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
+1288188000	2010	10	27	27	4	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
+1288188000	2010	10	27	27	4	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
+1288188000	2010	10	27	27	4	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
+1288188000	2010	10	27	27	4	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
+1288274400	2010	10	28	28	5	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
+1288274400	2010	10	28	28	5	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
+1288274400	2010	10	28	28	5	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
+1288274400	2010	10	28	28	5	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
+1288274400	2010	10	28	28	5	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
+1288274400	2010	10	28	28	5	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
+1288274400	2010	10	28	28	5	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
+1288274400	2010	10	28	28	5	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
+1288274400	2010	10	28	28	5	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
+1288274400	2010	10	28	28	5	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
+1288274400	2010	10	28	28	5	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
+1288274400	2010	10	28	28	5	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
+1288360800	2010	10	29	29	6	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
+1288360800	2010	10	29	29	6	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
+1288360800	2010	10	29	29	6	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
+1288360800	2010	10	29	29	6	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
+1288360800	2010	10	29	29	6	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
+1288360800	2010	10	29	29	6	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
+1288360800	2010	10	29	29	6	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
+1288360800	2010	10	29	29	6	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
+1288360800	2010	10	29	29	6	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
+1288360800	2010	10	29	29	6	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
+1288360800	2010	10	29	29	6	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
+1288360800	2010	10	29	29	6	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
+1288447200	2010	10	30	30	7	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
+1288447200	2010	10	30	30	7	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
+1288447200	2010	10	30	30	7	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
+1288447200	2010	10	30	30	7	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
+1288447200	2010	10	30	30	7	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
+1288447200	2010	10	30	30	7	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
+1288447200	2010	10	30	30	7	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
+1288447200	2010	10	30	30	7	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
+1288447200	2010	10	30	30	7	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
+1288447200	2010	10	30	30	7	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
+1288447200	2010	10	30	30	7	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
+1288533600	2010	10	31	31	1	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
+1288533600	2010	10	31	31	1	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
+1288533600	2010	10	31	31	1	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
+1288533600	2010	10	31	31	1	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
+1288533600	2010	10	31	31	1	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
+1288533600	2010	10	31	31	1	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
+1288533600	2010	10	31	31	1	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
+1288533600	2010	10	31	31	1	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
 PREHOOK: query: EXPLAIN SELECT
   to_unix_timestamp(fl_date),
   year(fl_date),
   month(fl_date),
   day(fl_date),
   dayofmonth(fl_date),
+  dayofweek(fl_date),
   weekofyear(fl_date),
   date(fl_date),
   to_date(fl_date),
@@ -435,6 +440,7 @@ POSTHOOK: query: EXPLAIN SELECT
   month(fl_date),
   day(fl_date),
   dayofmonth(fl_date),
+  dayofweek(fl_date),
   weekofyear(fl_date),
   date(fl_date),
   to_date(fl_date),
@@ -454,8 +460,8 @@ STAGE PLANS:
         TableScan
           alias: date_udf_flight_orc
           Select Operator
-            expressions: to_unix_timestamp(fl_date) (type: bigint), year(fl_date) (type: int), month(fl_date) (type: int), day(fl_date) (type: int), dayofmonth(fl_date) (type: int), weekofyear(fl_date) (type: int), fl_date (type: date), to_date(fl_date) (type: date), date_add(fl_date, 2) (type: date), date_sub(fl_date, 2) (type: date), datediff(fl_date, '2000-01-01') (type: int)
-            outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
+            expressions: to_unix_timestamp(fl_date) (type: bigint), year(fl_date) (type: int), month(fl_date) (type: int), day(fl_date) (type: int), dayofmonth(fl_date) (type: int), dayofweek(fl_date) (type: int), weekofyear(fl_date) (type: int), fl_date (type: date), to_date(fl_date) (type: date), date_add(fl_date, 2) (type: date), date_sub(fl_date, 2) (type: date), datediff(fl_date, '2000-01-01') (type: int)
+            outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
             ListSink
 
 PREHOOK: query: SELECT
@@ -464,6 +470,7 @@ PREHOOK: query: SELECT
   month(fl_date),
   day(fl_date),
   dayofmonth(fl_date),
+  dayofweek(fl_date),
   weekofyear(fl_date),
   date(fl_date),
   to_date(fl_date),
@@ -480,6 +487,7 @@ POSTHOOK: query: SELECT
   month(fl_date),
   day(fl_date),
   dayofmonth(fl_date),
+  dayofweek(fl_date),
   weekofyear(fl_date),
   date(fl_date),
   to_date(fl_date),
@@ -490,148 +498,149 @@ FROM date_udf_flight_orc
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@date_udf_flight_orc
 #### A masked pattern was here ####
-1287558000	2010	10	20	20	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
-1287558000	2010	10	20	20	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
-1287558000	2010	10	20	20	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
-1287558000	2010	10	20	20	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
-1287558000	2010	10	20	20	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
-1287558000	2010	10	20	20	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
-1287558000	2010	10	20	20	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
-1287558000	2010	10	20	20	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
-1287558000	2010	10	20	20	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
-1287558000	2010	10	20	20	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
-1287558000	2010	10	20	20	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
-1287644400	2010	10	21	21	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
-1287644400	2010	10	21	21	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
-1287644400	2010	10	21	21	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
-1287644400	2010	10	21	21	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
-1287644400	2010	10	21	21	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
-1287644400	2010	10	21	21	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
-1287644400	2010	10	21	21	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
-1287644400	2010	10	21	21	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
-1287644400	2010	10	21	21	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
-1287644400	2010	10	21	21	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
-1287644400	2010	10	21	21	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
-1287644400	2010	10	21	21	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
-1287730800	2010	10	22	22	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
-1287730800	2010	10	22	22	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
-1287730800	2010	10	22	22	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
-1287730800	2010	10	22	22	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
-1287730800	2010	10	22	22	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
-1287730800	2010	10	22	22	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
-1287730800	2010	10	22	22	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
-1287730800	2010	10	22	22	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
-1287730800	2010	10	22	22	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
-1287730800	2010	10	22	22	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
-1287730800	2010	10	22	22	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
-1287817200	2010	10	23	23	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
-1287817200	2010	10	23	23	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
-1287817200	2010	10	23	23	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
-1287817200	2010	10	23	23	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
-1287817200	2010	10	23	23	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
-1287817200	2010	10	23	23	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
-1287817200	2010	10	23	23	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
-1287817200	2010	10	23	23	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
-1287817200	2010	10	23	23	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
-1287817200	2010	10	23	23	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
-1287817200	2010	10	23	23	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
-1287817200	2010	10	23	23	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
-1287903600	2010	10	24	24	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
-1287903600	2010	10	24	24	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
-1287903600	2010	10	24	24	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
-1287903600	2010	10	24	24	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
-1287903600	2010	10	24	24	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
-1287903600	2010	10	24	24	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
-1287903600	2010	10	24	24	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
-1287903600	2010	10	24	24	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
-1287903600	2010	10	24	24	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
-1287903600	2010	10	24	24	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
-1287903600	2010	10	24	24	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
-1287903600	2010	10	24	24	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
-1287990000	2010	10	25	25	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
-1287990000	2010	10	25	25	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
-1287990000	2010	10	25	25	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
-1287990000	2010	10	25	25	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
-1287990000	2010	10	25	25	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
-1287990000	2010	10	25	25	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
-1287990000	2010	10	25	25	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
-1287990000	2010	10	25	25	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
-1287990000	2010	10	25	25	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
-1287990000	2010	10	25	25	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
-1287990000	2010	10	25	25	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
-1287990000	2010	10	25	25	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
-1288076400	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288076400	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288076400	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288076400	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288076400	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288076400	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288076400	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288076400	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288076400	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288076400	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288076400	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288076400	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288076400	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288162800	2010	10	27	27	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
-1288162800	2010	10	27	27	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
-1288162800	2010	10	27	27	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
-1288162800	2010	10	27	27	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
-1288162800	2010	10	27	27	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
-1288162800	2010	10	27	27	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
-1288162800	2010	10	27	27	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
-1288162800	2010	10	27	27	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
-1288162800	2010	10	27	27	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
-1288162800	2010	10	27	27	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
-1288162800	2010	10	27	27	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
-1288249200	2010	10	28	28	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
-1288249200	2010	10	28	28	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
-1288249200	2010	10	28	28	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
-1288249200	2010	10	28	28	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
-1288249200	2010	10	28	28	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
-1288249200	2010	10	28	28	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
-1288249200	2010	10	28	28	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
-1288249200	2010	10	28	28	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
-1288249200	2010	10	28	28	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
-1288249200	2010	10	28	28	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
-1288249200	2010	10	28	28	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
-1288249200	2010	10	28	28	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
-1288335600	2010	10	29	29	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
-1288335600	2010	10	29	29	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
-1288335600	2010	10	29	29	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
-1288335600	2010	10	29	29	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
-1288335600	2010	10	29	29	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
-1288335600	2010	10	29	29	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
-1288335600	2010	10	29	29	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
-1288335600	2010	10	29	29	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
-1288335600	2010	10	29	29	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
-1288335600	2010	10	29	29	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
-1288335600	2010	10	29	29	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
-1288335600	2010	10	29	29	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
-1288422000	2010	10	30	30	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
-1288422000	2010	10	30	30	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
-1288422000	2010	10	30	30	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
-1288422000	2010	10	30	30	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
-1288422000	2010	10	30	30	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
-1288422000	2010	10	30	30	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
-1288422000	2010	10	30	30	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
-1288422000	2010	10	30	30	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
-1288422000	2010	10	30	30	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
-1288422000	2010	10	30	30	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
-1288422000	2010	10	30	30	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
-1288508400	2010	10	31	31	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
-1288508400	2010	10	31	31	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
-1288508400	2010	10	31	31	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
-1288508400	2010	10	31	31	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
-1288508400	2010	10	31	31	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
-1288508400	2010	10	31	31	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
-1288508400	2010	10	31	31	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
-1288508400	2010	10	31	31	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
+1287558000	2010	10	20	20	4	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
+1287558000	2010	10	20	20	4	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
+1287558000	2010	10	20	20	4	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
+1287558000	2010	10	20	20	4	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
+1287558000	2010	10	20	20	4	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
+1287558000	2010	10	20	20	4	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
+1287558000	2010	10	20	20	4	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
+1287558000	2010	10	20	20	4	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
+1287558000	2010	10	20	20	4	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
+1287558000	2010	10	20	20	4	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
+1287558000	2010	10	20	20	4	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
+1287644400	2010	10	21	21	5	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
+1287644400	2010	10	21	21	5	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
+1287644400	2010	10	21	21	5	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
+1287644400	2010	10	21	21	5	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
+1287644400	2010	10	21	21	5	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
+1287644400	2010	10	21	21	5	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
+1287644400	2010	10	21	21	5	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
+1287644400	2010	10	21	21	5	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
+1287644400	2010	10	21	21	5	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
+1287644400	2010	10	21	21	5	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
+1287644400	2010	10	21	21	5	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
+1287644400	2010	10	21	21	5	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
+1287730800	2010	10	22	22	6	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
+1287730800	2010	10	22	22	6	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
+1287730800	2010	10	22	22	6	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
+1287730800	2010	10	22	22	6	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
+1287730800	2010	10	22	22	6	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
+1287730800	2010	10	22	22	6	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
+1287730800	2010	10	22	22	6	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
+1287730800	2010	10	22	22	6	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
+1287730800	2010	10	22	22	6	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
+1287730800	2010	10	22	22	6	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
+1287730800	2010	10	22	22	6	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
+1287817200	2010	10	23	23	7	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
+1287817200	2010	10	23	23	7	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
+1287817200	2010	10	23	23	7	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
+1287817200	2010	10	23	23	7	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
+1287817200	2010	10	23	23	7	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
+1287817200	2010	10	23	23	7	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
+1287817200	2010	10	23	23	7	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
+1287817200	2010	10	23	23	7	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
+1287817200	2010	10	23	23	7	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
+1287817200	2010	10	23	23	7	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
+1287817200	2010	10	23	23	7	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
+1287817200	2010	10	23	23	7	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
+1287903600	2010	10	24	24	1	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
+1287903600	2010	10	24	24	1	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
+1287903600	2010	10	24	24	1	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
+1287903600	2010	10	24	24	1	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
+1287903600	2010	10	24	24	1	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
+1287903600	2010	10	24	24	1	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
+1287903600	2010	10	24	24	1	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
+1287903600	2010	10	24	24	1	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
+1287903600	2010	10	24	24	1	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
+1287903600	2010	10	24	24	1	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
+1287903600	2010	10	24	24	1	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
+1287903600	2010	10	24	24	1	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
+1287990000	2010	10	25	25	2	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
+1287990000	2010	10	25	25	2	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
+1287990000	2010	10	25	25	2	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
+1287990000	2010	10	25	25	2	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
+1287990000	2010	10	25	25	2	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
+1287990000	2010	10	25	25	2	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
+1287990000	2010	10	25	25	2	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
+1287990000	2010	10	25	25	2	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
+1287990000	2010	10	25	25	2	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
+1287990000	2010	10	25	25	2	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
+1287990000	2010	10	25	25	2	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
+1287990000	2010	10	25	25	2	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
+1288076400	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288076400	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288076400	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288076400	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288076400	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288076400	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288076400	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288076400	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288076400	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288076400	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288076400	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288076400	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288076400	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288162800	2010	10	27	27	4	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
+1288162800	2010	10	27	27	4	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
+1288162800	2010	10	27	27	4	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
+1288162800	2010	10	27	27	4	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
+1288162800	2010	10	27	27	4	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
+1288162800	2010	10	27	27	4	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
+1288162800	2010	10	27	27	4	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
+1288162800	2010	10	27	27	4	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
+1288162800	2010	10	27	27	4	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
+1288162800	2010	10	27	27	4	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
+1288162800	2010	10	27	27	4	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
+1288249200	2010	10	28	28	5	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
+1288249200	2010	10	28	28	5	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
+1288249200	2010	10	28	28	5	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
+1288249200	2010	10	28	28	5	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
+1288249200	2010	10	28	28	5	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
+1288249200	2010	10	28	28	5	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
+1288249200	2010	10	28	28	5	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
+1288249200	2010	10	28	28	5	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
+1288249200	2010	10	28	28	5	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
+1288249200	2010	10	28	28	5	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
+1288249200	2010	10	28	28	5	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
+1288249200	2010	10	28	28	5	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
+1288335600	2010	10	29	29	6	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
+1288335600	2010	10	29	29	6	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
+1288335600	2010	10	29	29	6	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
+1288335600	2010	10	29	29	6	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
+1288335600	2010	10	29	29	6	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
+1288335600	2010	10	29	29	6	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
+1288335600	2010	10	29	29	6	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
+1288335600	2010	10	29	29	6	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
+1288335600	2010	10	29	29	6	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
+1288335600	2010	10	29	29	6	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
+1288335600	2010	10	29	29	6	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
+1288335600	2010	10	29	29	6	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
+1288422000	2010	10	30	30	7	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
+1288422000	2010	10	30	30	7	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
+1288422000	2010	10	30	30	7	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
+1288422000	2010	10	30	30	7	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
+1288422000	2010	10	30	30	7	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
+1288422000	2010	10	30	30	7	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
+1288422000	2010	10	30	30	7	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
+1288422000	2010	10	30	30	7	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
+1288422000	2010	10	30	30	7	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
+1288422000	2010	10	30	30	7	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
+1288422000	2010	10	30	30	7	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
+1288508400	2010	10	31	31	1	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
+1288508400	2010	10	31	31	1	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
+1288508400	2010	10	31	31	1	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
+1288508400	2010	10	31	31	1	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
+1288508400	2010	10	31	31	1	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
+1288508400	2010	10	31	31	1	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
+1288508400	2010	10	31	31	1	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
+1288508400	2010	10	31	31	1	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
 PREHOOK: query: EXPLAIN SELECT
   year(fl_time) = year(fl_date),
   month(fl_time) = month(fl_date),
   day(fl_time) = day(fl_date),
   dayofmonth(fl_time) = dayofmonth(fl_date),
+  dayofweek(fl_time) = dayofweek(fl_date),
   weekofyear(fl_time) = weekofyear(fl_date),
   date(fl_time) = date(fl_date),
   to_date(fl_time) = to_date(fl_date),
@@ -645,6 +654,7 @@ POSTHOOK: query: EXPLAIN SELECT
   month(fl_time) = month(fl_date),
   day(fl_time) = day(fl_date),
   dayofmonth(fl_time) = dayofmonth(fl_date),
+  dayofweek(fl_time) = dayofweek(fl_date),
   weekofyear(fl_time) = weekofyear(fl_date),
   date(fl_time) = date(fl_date),
   to_date(fl_time) = to_date(fl_date),
@@ -664,8 +674,8 @@ STAGE PLANS:
         TableScan
           alias: date_udf_flight_orc
           Select Operator
-            expressions: (year(fl_time) = year(fl_date)) (type: boolean), (month(fl_time) = month(fl_date)) (type: boolean), (day(fl_time) = day(fl_date)) (type: boolean), (dayofmonth(fl_time) = dayofmonth(fl_date)) (type: boolean), (weekofyear(fl_time) = weekofyear(fl_date)) (type: boolean), (CAST( fl_time AS DATE) = fl_date) (type: boolean), (to_date(fl_time) = to_date(fl_date)) (type: boolean), (date_add(fl_time, 2) = date_add(fl_date, 2)) (type: boolean), (date_sub(fl_time, 2) = date_sub(fl_date, 2)) (type: boolean), (datediff(fl_time, '2000-01-01') = datediff(fl_date, '2000-01-01')) (type: boolean)
-            outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
+            expressions: (year(fl_time) = year(fl_date)) (type: boolean), (month(fl_time) = month(fl_date)) (type: boolean), (day(fl_time) = day(fl_date)) (type: boolean), (dayofmonth(fl_time) = dayofmonth(fl_date)) (type: boolean), (dayofweek(fl_time) = dayofweek(fl_date)) (type: boolean), (weekofyear(fl_time) = weekofyear(fl_date)) (type: boolean), (CAST( fl_time AS DATE) = fl_date) (type: boolean), (to_date(fl_time) = to_date(fl_date)) (type: boolean), (date_add(fl_time, 2) = date_add(fl_date, 2)) (type: boolean), (date_sub(fl_time, 2) = date_sub(fl_date, 2)) (type: boolean), (datediff(fl_time, '2000-01-01') = datediff(fl_date, '2000-01-01')) (type: boolean)
+            outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
             ListSink
 
 PREHOOK: query: -- Should all be true or NULL
@@ -674,6 +684,7 @@ SELECT
   month(fl_time) = month(fl_date),
   day(fl_time) = day(fl_date),
   dayofmonth(fl_time) = dayofmonth(fl_date),
+  dayofweek(fl_time) = dayofweek(fl_date),
   weekofyear(fl_time) = weekofyear(fl_date),
   date(fl_time) = date(fl_date),
   to_date(fl_time) = to_date(fl_date),
@@ -690,6 +701,7 @@ SELECT
   month(fl_time) = month(fl_date),
   day(fl_time) = day(fl_date),
   dayofmonth(fl_time) = dayofmonth(fl_date),
+  dayofweek(fl_time) = dayofweek(fl_date),
   weekofyear(fl_time) = weekofyear(fl_date),
   date(fl_time) = date(fl_date),
   to_date(fl_time) = to_date(fl_date),
@@ -700,143 +712,143 @@ FROM date_udf_flight_orc
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@date_udf_flight_orc
 #### A masked pattern was here ####
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
 PREHOOK: query: EXPLAIN SELECT 
   fl_date, 
   to_date(date_add(fl_date, 2)), 

http://git-wip-us.apache.org/repos/asf/hive/blob/59539885/ql/src/test/results/clientpositive/show_functions.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/show_functions.q.out b/ql/src/test/results/clientpositive/show_functions.q.out
index 7b746ff..d2f3b97 100644
--- a/ql/src/test/results/clientpositive/show_functions.q.out
+++ b/ql/src/test/results/clientpositive/show_functions.q.out
@@ -68,6 +68,7 @@ date_sub
 datediff
 day
 dayofmonth
+dayofweek
 decode
 default.qtest_get_java_boolean
 degrees


[09/22] hive git commit: HIVE-14896 : Stabilize golden files for currently failing tests

Posted by se...@apache.org.
HIVE-14896 : Stabilize golden files for currently failing tests


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

Branch: refs/heads/hive-14535
Commit: e1fa2787ffb5b65e54ed5c087ca3ed8060f18d83
Parents: 8e0b19b
Author: Ashutosh Chauhan <ha...@apache.org>
Authored: Wed Oct 5 15:26:15 2016 -0700
Committer: Ashutosh Chauhan <ha...@apache.org>
Committed: Wed Oct 5 22:50:45 2016 -0700

----------------------------------------------------------------------
 .../test/queries/clientpositive/acid_mapjoin.q  |   3 +-
 ql/src/test/queries/clientpositive/ctas.q       |   2 +-
 .../test/queries/clientpositive/ctas_hadoop20.q |  63 --
 .../results/clientpositive/acid_mapjoin.q.out   |  30 +-
 ql/src/test/results/clientpositive/ctas.q.out   | 118 +--
 .../results/clientpositive/ctas_hadoop20.q.out  | 945 -------------------
 .../test/results/clientpositive/llap/ctas.q.out |  81 +-
 .../results/clientpositive/spark/ctas.q.out     |  81 +-
 .../vector_join_part_col_char.q.out             |   6 +-
 9 files changed, 35 insertions(+), 1294 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/e1fa2787/ql/src/test/queries/clientpositive/acid_mapjoin.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/acid_mapjoin.q b/ql/src/test/queries/clientpositive/acid_mapjoin.q
index 5eee6e7..e3d14cc 100644
--- a/ql/src/test/queries/clientpositive/acid_mapjoin.q
+++ b/ql/src/test/queries/clientpositive/acid_mapjoin.q
@@ -13,7 +13,8 @@ create table acid2 (key int, value string) clustered by (key) into 2 buckets sto
 
 insert into acid1 values (1, 'a'), (2, 'b'), (3, 'c'), (4, 'd'), (5, 'e'), (6, 'f'), (7, 'g'), (8, 'h');
 insert into acid2 values (1,'a'),(3,'c'),(5,'e'),(7,'g');
-
+alter table acid2 update statistics set('numRows'='210', 'rawDataSize'='840');
+alter table acid1 update statistics set('numRows'='316', 'rawDataSize'='1265');
 explain
 select count(*) from acid1 join acid2 on acid1.key = acid2.key;
 select count(*) from acid1 join acid2 on acid1.key = acid2.key;

http://git-wip-us.apache.org/repos/asf/hive/blob/e1fa2787/ql/src/test/queries/clientpositive/ctas.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/ctas.q b/ql/src/test/queries/clientpositive/ctas.q
index edd1f6a..57a4729 100644
--- a/ql/src/test/queries/clientpositive/ctas.q
+++ b/ql/src/test/queries/clientpositive/ctas.q
@@ -49,7 +49,7 @@ select * from nzhang_ctas4;
 
 describe formatted nzhang_CTAS4;
 
-explain extended create table nzhang_ctas5 row format delimited fields terminated by ',' lines terminated by '\012' stored as textfile as select key, value from src sort by key, value limit 10;
+explain create table nzhang_ctas5 row format delimited fields terminated by ',' lines terminated by '\012' stored as textfile as select key, value from src sort by key, value limit 10;
 
 set mapreduce.framework.name=yarn;
 set mapreduce.jobtracker.address=localhost:58;

http://git-wip-us.apache.org/repos/asf/hive/blob/e1fa2787/ql/src/test/queries/clientpositive/ctas_hadoop20.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/ctas_hadoop20.q b/ql/src/test/queries/clientpositive/ctas_hadoop20.q
deleted file mode 100644
index e275b7b..0000000
--- a/ql/src/test/queries/clientpositive/ctas_hadoop20.q
+++ /dev/null
@@ -1,63 +0,0 @@
--- INCLUDE_HADOOP_MAJOR_VERSIONS(0.20S)
-
-create table nzhang_Tmp(a int, b string);
-select * from nzhang_Tmp;
-
-explain create table nzhang_CTAS1 as select key k, value from src sort by k, value limit 10;
-
-create table nzhang_CTAS1 as select key k, value from src sort by k, value limit 10;
-
-select * from nzhang_CTAS1;
-
-describe formatted nzhang_CTAS1;
-
-
-explain create table nzhang_ctas2 as select * from src sort by key, value limit 10;
-
-create table nzhang_ctas2 as select * from src sort by key, value limit 10;
-
-select * from nzhang_ctas2;
-
-describe formatted nzhang_CTAS2;
-
-
-explain create table nzhang_ctas3 row format serde "org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe" stored as RCFile as select key/2 half_key, concat(value, "_con") conb  from src sort by half_key, conb limit 10;
-
-create table nzhang_ctas3 row format serde "org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe" stored as RCFile as select key/2 half_key, concat(value, "_con") conb  from src sort by half_key, conb limit 10;
-
-select * from nzhang_ctas3;
-
-describe formatted nzhang_CTAS3;
-
-
-explain create table if not exists nzhang_ctas3 as select key, value from src sort by key, value limit 2;
-
-create table if not exists nzhang_ctas3 as select key, value from src sort by key, value limit 2;
-
-select * from nzhang_ctas3;
-
-describe formatted nzhang_CTAS3;
-
-
-explain create table nzhang_ctas4 row format delimited fields terminated by ',' stored as textfile as select key, value from src sort by key, value limit 10;
-
-create table nzhang_ctas4 row format delimited fields terminated by ',' stored as textfile as select key, value from src sort by key, value limit 10;
-
-select * from nzhang_ctas4;
-
-describe formatted nzhang_CTAS4;
-
-explain extended create table nzhang_ctas5 row format delimited fields terminated by ',' lines terminated by '\012' stored as textfile as select key, value from src sort by key, value limit 10;
-
-set mapred.job.tracker=localhost:58;
-set hive.exec.mode.local.auto=true;
-
-create table nzhang_ctas5 row format delimited fields terminated by ',' lines terminated by '\012' stored as textfile as select key, value from src sort by key, value limit 10;
-
-create table nzhang_ctas6 (key string, `to` string);
-insert overwrite table nzhang_ctas6 select key, value from src limit 10;
-create table nzhang_ctas7 as select key, `to` from nzhang_ctas6;
-
-create table nzhang_ctas8 as select 3.14BD from nzhang_ctas6 limit 1;
-desc nzhang_ctas8;
-drop table nzhang_ctas8;

http://git-wip-us.apache.org/repos/asf/hive/blob/e1fa2787/ql/src/test/results/clientpositive/acid_mapjoin.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/acid_mapjoin.q.out b/ql/src/test/results/clientpositive/acid_mapjoin.q.out
index 036c56c..402c6ab 100644
--- a/ql/src/test/results/clientpositive/acid_mapjoin.q.out
+++ b/ql/src/test/results/clientpositive/acid_mapjoin.q.out
@@ -42,6 +42,22 @@ POSTHOOK: Input: default@values__tmp__table__2
 POSTHOOK: Output: default@acid2
 POSTHOOK: Lineage: acid2.key EXPRESSION [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
 POSTHOOK: Lineage: acid2.value SIMPLE [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
+PREHOOK: query: alter table acid2 update statistics set('numRows'='210', 'rawDataSize'='840')
+PREHOOK: type: ALTERTABLE_UPDATETABLESTATS
+PREHOOK: Input: default@acid2
+PREHOOK: Output: default@acid2
+POSTHOOK: query: alter table acid2 update statistics set('numRows'='210', 'rawDataSize'='840')
+POSTHOOK: type: ALTERTABLE_UPDATETABLESTATS
+POSTHOOK: Input: default@acid2
+POSTHOOK: Output: default@acid2
+PREHOOK: query: alter table acid1 update statistics set('numRows'='316', 'rawDataSize'='1265')
+PREHOOK: type: ALTERTABLE_UPDATETABLESTATS
+PREHOOK: Input: default@acid1
+PREHOOK: Output: default@acid1
+POSTHOOK: query: alter table acid1 update statistics set('numRows'='316', 'rawDataSize'='1265')
+POSTHOOK: type: ALTERTABLE_UPDATETABLESTATS
+POSTHOOK: Input: default@acid1
+POSTHOOK: Output: default@acid1
 PREHOOK: query: explain
 select count(*) from acid1 join acid2 on acid1.key = acid2.key
 PREHOOK: type: QUERY
@@ -64,14 +80,14 @@ STAGE PLANS:
         $hdt$_1:acid2 
           TableScan
             alias: acid2
-            Statistics: Num rows: 209 Data size: 839 Basic stats: COMPLETE Column stats: NONE
+            Statistics: Num rows: 210 Data size: 840 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
               predicate: key is not null (type: boolean)
-              Statistics: Num rows: 209 Data size: 839 Basic stats: COMPLETE Column stats: NONE
+              Statistics: Num rows: 210 Data size: 840 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: key (type: int)
                 outputColumnNames: _col0
-                Statistics: Num rows: 209 Data size: 839 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 210 Data size: 840 Basic stats: COMPLETE Column stats: NONE
                 HashTable Sink Operator
                   keys:
                     0 _col0 (type: int)
@@ -82,21 +98,21 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: acid1
-            Statistics: Num rows: 312 Data size: 1251 Basic stats: COMPLETE Column stats: NONE
+            Statistics: Num rows: 316 Data size: 1265 Basic stats: COMPLETE Column stats: NONE
             Filter Operator
               predicate: key is not null (type: boolean)
-              Statistics: Num rows: 312 Data size: 1251 Basic stats: COMPLETE Column stats: NONE
+              Statistics: Num rows: 316 Data size: 1265 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: key (type: int)
                 outputColumnNames: _col0
-                Statistics: Num rows: 312 Data size: 1251 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 316 Data size: 1265 Basic stats: COMPLETE Column stats: NONE
                 Map Join Operator
                   condition map:
                        Inner Join 0 to 1
                   keys:
                     0 _col0 (type: int)
                     1 _col0 (type: int)
-                  Statistics: Num rows: 343 Data size: 1376 Basic stats: COMPLETE Column stats: NONE
+                  Statistics: Num rows: 347 Data size: 1391 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count()
                     mode: hash

http://git-wip-us.apache.org/repos/asf/hive/blob/e1fa2787/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 7f1c912..a9c3136b 100644
--- a/ql/src/test/results/clientpositive/ctas.q.out
+++ b/ql/src/test/results/clientpositive/ctas.q.out
@@ -690,9 +690,9 @@ Sort Columns:       	[]
 Storage Desc Params:	 	 
 	field.delim         	,                   
 	serialization.format	,                   
-PREHOOK: query: explain extended create table nzhang_ctas5 row format delimited fields terminated by ',' lines terminated by '\012' stored as textfile as select key, value from src sort by key, value limit 10
+PREHOOK: query: explain create table nzhang_ctas5 row format delimited fields terminated by ',' lines terminated by '\012' stored as textfile as select key, value from src sort by key, value limit 10
 PREHOOK: type: CREATETABLE_AS_SELECT
-POSTHOOK: query: explain extended create table nzhang_ctas5 row format delimited fields terminated by ',' lines terminated by '\012' stored as textfile as select key, value from src sort by key, value limit 10
+POSTHOOK: query: explain create table nzhang_ctas5 row format delimited fields terminated by ',' lines terminated by '\012' stored as textfile as select key, value from src sort by key, value limit 10
 POSTHOOK: type: CREATETABLE_AS_SELECT
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
@@ -708,70 +708,15 @@ STAGE PLANS:
           TableScan
             alias: src
             Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-            GatherStats: false
             Select Operator
               expressions: key (type: string), value (type: string)
               outputColumnNames: _col0, _col1
               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
-                TopN: 10
                 TopN Hash Memory Usage: 0.1
-                auto parallelism: false
-      Path -> Alias:
-#### A masked pattern was here ####
-      Path -> Partition:
-#### A masked pattern was here ####
-          Partition
-            base file name: src
-            input format: org.apache.hadoop.mapred.TextInputFormat
-            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-            properties:
-              COLUMN_STATS_ACCURATE {"BASIC_STATS":"true","COLUMN_STATS":{"key":"true","value":"true"}}
-              bucket_count -1
-              columns key,value
-              columns.comments 'default','default'
-              columns.types string:string
-#### A masked pattern was here ####
-              name default.src
-              numFiles 1
-              numRows 500
-              rawDataSize 5312
-              serialization.ddl struct src { string key, string value}
-              serialization.format 1
-              serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-              totalSize 5812
-#### A masked pattern was here ####
-            serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-          
-              input format: org.apache.hadoop.mapred.TextInputFormat
-              output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-              properties:
-                COLUMN_STATS_ACCURATE {"BASIC_STATS":"true","COLUMN_STATS":{"key":"true","value":"true"}}
-                bucket_count -1
-                columns key,value
-                columns.comments 'default','default'
-                columns.types string:string
-#### A masked pattern was here ####
-                name default.src
-                numFiles 1
-                numRows 500
-                rawDataSize 5312
-                serialization.ddl struct src { string key, string value}
-                serialization.format 1
-                serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                totalSize 5812
-#### A masked pattern was here ####
-              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-              name: default.src
-            name: default.src
-      Truncated Path -> Alias:
-        /src [src]
-      Needs Tagging: false
       Reduce Operator Tree:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string)
@@ -782,62 +727,20 @@ STAGE PLANS:
             Statistics: Num rows: 10 Data size: 100 Basic stats: COMPLETE Column stats: NONE
             File Output Operator
               compressed: false
-              GlobalTableId: 0
-#### A masked pattern was here ####
-              NumFilesPerFileSink: 1
               table:
                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                  properties:
-                    columns _col0,_col1
-                    columns.types string,string
-                    escape.delim \
-                    serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
                   serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
-              TotalFiles: 1
-              GatherStats: false
-              MultiFileSpray: false
 
   Stage: Stage-2
     Map Reduce
       Map Operator Tree:
           TableScan
-            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
-              TopN: 10
               TopN Hash Memory Usage: 0.1
-              auto parallelism: false
-      Path -> Alias:
-#### A masked pattern was here ####
-      Path -> Partition:
-#### A masked pattern was here ####
-          Partition
-            base file name: -mr-10003
-            input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-            output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-            properties:
-              columns _col0,_col1
-              columns.types string,string
-              escape.delim \
-              serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
-            serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
-          
-              input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-              output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-              properties:
-                columns _col0,_col1
-                columns.types string,string
-                escape.delim \
-                serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
-              serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
-      Truncated Path -> Alias:
-#### A masked pattern was here ####
-      Needs Tagging: false
       Reduce Operator Tree:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string)
@@ -848,28 +751,12 @@ STAGE PLANS:
             Statistics: Num rows: 10 Data size: 100 Basic stats: COMPLETE Column stats: NONE
             File Output Operator
               compressed: false
-              GlobalTableId: 1
-#### A masked pattern was here ####
-              NumFilesPerFileSink: 1
               Statistics: Num rows: 10 Data size: 100 Basic stats: COMPLETE Column stats: NONE
-#### A masked pattern was here ####
               table:
                   input format: org.apache.hadoop.mapred.TextInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                  properties:
-                    columns key,value
-                    columns.types string:string
-                    field.delim ,
-                    line.delim 
-
-                    name default.nzhang_ctas5
-                    serialization.format ,
-                    serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
                   serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
                   name: default.nzhang_ctas5
-              TotalFiles: 1
-              GatherStats: true
-              MultiFileSpray: false
 
   Stage: Stage-0
     Move Operator
@@ -891,7 +778,6 @@ STAGE PLANS:
 
   Stage: Stage-3
     Stats-Aggr Operator
-#### A masked pattern was here ####
 
 PREHOOK: query: create table nzhang_ctas5 row format delimited fields terminated by ',' lines terminated by '\012' stored as textfile as select key, value from src sort by key, value limit 10
 PREHOOK: type: CREATETABLE_AS_SELECT

http://git-wip-us.apache.org/repos/asf/hive/blob/e1fa2787/ql/src/test/results/clientpositive/ctas_hadoop20.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/ctas_hadoop20.q.out b/ql/src/test/results/clientpositive/ctas_hadoop20.q.out
deleted file mode 100644
index 4af1e0a..0000000
--- a/ql/src/test/results/clientpositive/ctas_hadoop20.q.out
+++ /dev/null
@@ -1,945 +0,0 @@
-PREHOOK: query: -- INCLUDE_HADOOP_MAJOR_VERSIONS(0.20, 0.20S)
-
-create table nzhang_Tmp(a int, b string)
-PREHOOK: type: CREATETABLE
-PREHOOK: Output: database:default
-POSTHOOK: query: -- INCLUDE_HADOOP_MAJOR_VERSIONS(0.20, 0.20S)
-
-create table nzhang_Tmp(a int, b string)
-POSTHOOK: type: CREATETABLE
-POSTHOOK: Output: database:default
-POSTHOOK: Output: default@nzhang_Tmp
-PREHOOK: query: select * from nzhang_Tmp
-PREHOOK: type: QUERY
-PREHOOK: Input: default@nzhang_tmp
-#### A masked pattern was here ####
-POSTHOOK: query: select * from nzhang_Tmp
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@nzhang_tmp
-#### A masked pattern was here ####
-PREHOOK: query: explain create table nzhang_CTAS1 as select key k, value from src sort by k, value limit 10
-PREHOOK: type: CREATETABLE_AS_SELECT
-POSTHOOK: query: explain create table nzhang_CTAS1 as select key k, value from src sort by k, value limit 10
-POSTHOOK: type: CREATETABLE_AS_SELECT
-STAGE DEPENDENCIES:
-  Stage-1 is a root stage
-  Stage-2 depends on stages: Stage-1
-  Stage-0 depends on stages: Stage-2
-  Stage-4 depends on stages: Stage-0
-  Stage-3 depends on stages: Stage-4
-
-STAGE PLANS:
-  Stage: Stage-1
-    Map Reduce
-      Map Operator Tree:
-          TableScan
-            alias: src
-            Statistics: Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE
-            Select Operator
-              expressions: key (type: string), value (type: string)
-              outputColumnNames: _col0, _col1
-              Statistics: Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE
-              Reduce Output Operator
-                key expressions: _col0 (type: string), _col1 (type: string)
-                sort order: ++
-                Statistics: Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE
-      Reduce Operator Tree:
-        Select Operator
-          expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string)
-          outputColumnNames: _col0, _col1
-          Statistics: Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE
-          Limit
-            Number of rows: 10
-            Statistics: Num rows: 10 Data size: 2000 Basic stats: COMPLETE Column stats: NONE
-            File Output Operator
-              compressed: false
-              table:
-                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                  serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
-
-  Stage: Stage-2
-    Map Reduce
-      Map Operator Tree:
-          TableScan
-            Reduce Output Operator
-              key expressions: _col0 (type: string), _col1 (type: string)
-              sort order: ++
-              Statistics: Num rows: 10 Data size: 2000 Basic stats: COMPLETE Column stats: NONE
-      Reduce Operator Tree:
-        Select Operator
-          expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string)
-          outputColumnNames: _col0, _col1
-          Statistics: Num rows: 10 Data size: 2000 Basic stats: COMPLETE Column stats: NONE
-          Limit
-            Number of rows: 10
-            Statistics: Num rows: 10 Data size: 2000 Basic stats: COMPLETE Column stats: NONE
-            File Output Operator
-              compressed: false
-              Statistics: Num rows: 10 Data size: 2000 Basic stats: COMPLETE Column stats: NONE
-              table:
-                  input format: org.apache.hadoop.mapred.TextInputFormat
-                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                  name: default.nzhang_CTAS1
-
-  Stage: Stage-0
-    Move Operator
-      files:
-          hdfs directory: true
-#### A masked pattern was here ####
-
-  Stage: Stage-4
-      Create Table Operator:
-        Create Table
-          columns: k string, value string
-          input format: org.apache.hadoop.mapred.TextInputFormat
-          output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat
-          name: nzhang_CTAS1
-
-  Stage: Stage-3
-    Stats-Aggr Operator
-
-PREHOOK: query: create table nzhang_CTAS1 as select key k, value from src sort by k, value limit 10
-PREHOOK: type: CREATETABLE_AS_SELECT
-PREHOOK: Input: default@src
-POSTHOOK: query: create table nzhang_CTAS1 as select key k, value from src sort by k, value limit 10
-POSTHOOK: type: CREATETABLE_AS_SELECT
-POSTHOOK: Input: default@src
-POSTHOOK: Output: default@nzhang_CTAS1
-PREHOOK: query: select * from nzhang_CTAS1
-PREHOOK: type: QUERY
-PREHOOK: Input: default@nzhang_ctas1
-#### A masked pattern was here ####
-POSTHOOK: query: select * from nzhang_CTAS1
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@nzhang_ctas1
-#### A masked pattern was here ####
-0	val_0
-0	val_0
-0	val_0
-10	val_10
-100	val_100
-100	val_100
-103	val_103
-103	val_103
-104	val_104
-104	val_104
-PREHOOK: query: describe formatted nzhang_CTAS1
-PREHOOK: type: DESCTABLE
-PREHOOK: Input: default@nzhang_ctas1
-POSTHOOK: query: describe formatted nzhang_CTAS1
-POSTHOOK: type: DESCTABLE
-POSTHOOK: Input: default@nzhang_ctas1
-# col_name            	data_type           	comment             
-	 	 
-k                   	string              	                    
-value               	string              	                    
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-#### A masked pattern was here ####
-Retention:          	0                   	 
-#### A masked pattern was here ####
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	COLUMN_STATS_ACCURATE	true                
-	numFiles            	1                   
-	numRows             	10                  
-	rawDataSize         	96                  
-	totalSize           	106                 
-#### A masked pattern was here ####
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
-PREHOOK: query: explain create table nzhang_ctas2 as select * from src sort by key, value limit 10
-PREHOOK: type: CREATETABLE_AS_SELECT
-POSTHOOK: query: explain create table nzhang_ctas2 as select * from src sort by key, value limit 10
-POSTHOOK: type: CREATETABLE_AS_SELECT
-STAGE DEPENDENCIES:
-  Stage-1 is a root stage
-  Stage-2 depends on stages: Stage-1
-  Stage-0 depends on stages: Stage-2
-  Stage-4 depends on stages: Stage-0
-  Stage-3 depends on stages: Stage-4
-
-STAGE PLANS:
-  Stage: Stage-1
-    Map Reduce
-      Map Operator Tree:
-          TableScan
-            alias: src
-            Statistics: Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE
-            Select Operator
-              expressions: key (type: string), value (type: string)
-              outputColumnNames: _col0, _col1
-              Statistics: Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE
-              Reduce Output Operator
-                key expressions: _col0 (type: string), _col1 (type: string)
-                sort order: ++
-                Statistics: Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE
-      Reduce Operator Tree:
-        Select Operator
-          expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string)
-          outputColumnNames: _col0, _col1
-          Statistics: Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE
-          Limit
-            Number of rows: 10
-            Statistics: Num rows: 10 Data size: 2000 Basic stats: COMPLETE Column stats: NONE
-            File Output Operator
-              compressed: false
-              table:
-                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                  serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
-
-  Stage: Stage-2
-    Map Reduce
-      Map Operator Tree:
-          TableScan
-            Reduce Output Operator
-              key expressions: _col0 (type: string), _col1 (type: string)
-              sort order: ++
-              Statistics: Num rows: 10 Data size: 2000 Basic stats: COMPLETE Column stats: NONE
-      Reduce Operator Tree:
-        Select Operator
-          expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string)
-          outputColumnNames: _col0, _col1
-          Statistics: Num rows: 10 Data size: 2000 Basic stats: COMPLETE Column stats: NONE
-          Limit
-            Number of rows: 10
-            Statistics: Num rows: 10 Data size: 2000 Basic stats: COMPLETE Column stats: NONE
-            File Output Operator
-              compressed: false
-              Statistics: Num rows: 10 Data size: 2000 Basic stats: COMPLETE Column stats: NONE
-              table:
-                  input format: org.apache.hadoop.mapred.TextInputFormat
-                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                  name: default.nzhang_ctas2
-
-  Stage: Stage-0
-    Move Operator
-      files:
-          hdfs directory: true
-#### A masked pattern was here ####
-
-  Stage: Stage-4
-      Create Table Operator:
-        Create Table
-          columns: key string, value string
-          input format: org.apache.hadoop.mapred.TextInputFormat
-          output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat
-          name: nzhang_ctas2
-
-  Stage: Stage-3
-    Stats-Aggr Operator
-
-PREHOOK: query: create table nzhang_ctas2 as select * from src sort by key, value limit 10
-PREHOOK: type: CREATETABLE_AS_SELECT
-PREHOOK: Input: default@src
-POSTHOOK: query: create table nzhang_ctas2 as select * from src sort by key, value limit 10
-POSTHOOK: type: CREATETABLE_AS_SELECT
-POSTHOOK: Input: default@src
-POSTHOOK: Output: default@nzhang_ctas2
-PREHOOK: query: select * from nzhang_ctas2
-PREHOOK: type: QUERY
-PREHOOK: Input: default@nzhang_ctas2
-#### A masked pattern was here ####
-POSTHOOK: query: select * from nzhang_ctas2
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@nzhang_ctas2
-#### A masked pattern was here ####
-0	val_0
-0	val_0
-0	val_0
-10	val_10
-100	val_100
-100	val_100
-103	val_103
-103	val_103
-104	val_104
-104	val_104
-PREHOOK: query: describe formatted nzhang_CTAS2
-PREHOOK: type: DESCTABLE
-PREHOOK: Input: default@nzhang_ctas2
-POSTHOOK: query: describe formatted nzhang_CTAS2
-POSTHOOK: type: DESCTABLE
-POSTHOOK: Input: default@nzhang_ctas2
-# col_name            	data_type           	comment             
-	 	 
-key                 	string              	                    
-value               	string              	                    
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-#### A masked pattern was here ####
-Retention:          	0                   	 
-#### A masked pattern was here ####
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	COLUMN_STATS_ACCURATE	true                
-	numFiles            	1                   
-	numRows             	10                  
-	rawDataSize         	96                  
-	totalSize           	106                 
-#### A masked pattern was here ####
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
-PREHOOK: query: explain create table nzhang_ctas3 row format serde "org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe" stored as RCFile as select key/2 half_key, concat(value, "_con") conb  from src sort by half_key, conb limit 10
-PREHOOK: type: CREATETABLE_AS_SELECT
-POSTHOOK: query: explain create table nzhang_ctas3 row format serde "org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe" stored as RCFile as select key/2 half_key, concat(value, "_con") conb  from src sort by half_key, conb limit 10
-POSTHOOK: type: CREATETABLE_AS_SELECT
-STAGE DEPENDENCIES:
-  Stage-1 is a root stage
-  Stage-2 depends on stages: Stage-1
-  Stage-0 depends on stages: Stage-2
-  Stage-4 depends on stages: Stage-0
-  Stage-3 depends on stages: Stage-4
-
-STAGE PLANS:
-  Stage: Stage-1
-    Map Reduce
-      Map Operator Tree:
-          TableScan
-            alias: src
-            Statistics: Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE
-            Select Operator
-              expressions: (key / 2) (type: double), concat(value, '_con') (type: string)
-              outputColumnNames: _col0, _col1
-              Statistics: Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE
-              Reduce Output Operator
-                key expressions: _col0 (type: double), _col1 (type: string)
-                sort order: ++
-                Statistics: Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE
-      Reduce Operator Tree:
-        Select Operator
-          expressions: KEY.reducesinkkey0 (type: double), KEY.reducesinkkey1 (type: string)
-          outputColumnNames: _col0, _col1
-          Statistics: Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE
-          Limit
-            Number of rows: 10
-            Statistics: Num rows: 10 Data size: 2000 Basic stats: COMPLETE Column stats: NONE
-            File Output Operator
-              compressed: false
-              table:
-                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                  serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
-
-  Stage: Stage-2
-    Map Reduce
-      Map Operator Tree:
-          TableScan
-            Reduce Output Operator
-              key expressions: _col0 (type: double), _col1 (type: string)
-              sort order: ++
-              Statistics: Num rows: 10 Data size: 2000 Basic stats: COMPLETE Column stats: NONE
-      Reduce Operator Tree:
-        Select Operator
-          expressions: KEY.reducesinkkey0 (type: double), KEY.reducesinkkey1 (type: string)
-          outputColumnNames: _col0, _col1
-          Statistics: Num rows: 10 Data size: 2000 Basic stats: COMPLETE Column stats: NONE
-          Limit
-            Number of rows: 10
-            Statistics: Num rows: 10 Data size: 2000 Basic stats: COMPLETE Column stats: NONE
-            File Output Operator
-              compressed: false
-              Statistics: Num rows: 10 Data size: 2000 Basic stats: COMPLETE Column stats: NONE
-              table:
-                  input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat
-                  output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat
-                  serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe
-                  name: default.nzhang_ctas3
-
-  Stage: Stage-0
-    Move Operator
-      files:
-          hdfs directory: true
-#### A masked pattern was here ####
-
-  Stage: Stage-4
-      Create Table Operator:
-        Create Table
-          columns: half_key double, conb string
-          input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat
-          output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat
-          serde name: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe
-          name: nzhang_ctas3
-
-  Stage: Stage-3
-    Stats-Aggr Operator
-
-PREHOOK: query: create table nzhang_ctas3 row format serde "org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe" stored as RCFile as select key/2 half_key, concat(value, "_con") conb  from src sort by half_key, conb limit 10
-PREHOOK: type: CREATETABLE_AS_SELECT
-PREHOOK: Input: default@src
-POSTHOOK: query: create table nzhang_ctas3 row format serde "org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe" stored as RCFile as select key/2 half_key, concat(value, "_con") conb  from src sort by half_key, conb limit 10
-POSTHOOK: type: CREATETABLE_AS_SELECT
-POSTHOOK: Input: default@src
-POSTHOOK: Output: default@nzhang_ctas3
-PREHOOK: query: select * from nzhang_ctas3
-PREHOOK: type: QUERY
-PREHOOK: Input: default@nzhang_ctas3
-#### A masked pattern was here ####
-POSTHOOK: query: select * from nzhang_ctas3
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@nzhang_ctas3
-#### A masked pattern was here ####
-0.0	val_0_con
-0.0	val_0_con
-0.0	val_0_con
-1.0	val_2_con
-2.0	val_4_con
-2.5	val_5_con
-2.5	val_5_con
-2.5	val_5_con
-4.0	val_8_con
-4.5	val_9_con
-PREHOOK: query: describe formatted nzhang_CTAS3
-PREHOOK: type: DESCTABLE
-PREHOOK: Input: default@nzhang_ctas3
-POSTHOOK: query: describe formatted nzhang_CTAS3
-POSTHOOK: type: DESCTABLE
-POSTHOOK: Input: default@nzhang_ctas3
-# col_name            	data_type           	comment             
-	 	 
-half_key            	double              	                    
-conb                	string              	                    
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-#### A masked pattern was here ####
-Retention:          	0                   	 
-#### A masked pattern was here ####
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	COLUMN_STATS_ACCURATE	true                
-	numFiles            	1                   
-	numRows             	10                  
-	rawDataSize         	120                 
-	totalSize           	199                 
-#### A masked pattern was here ####
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe	 
-InputFormat:        	org.apache.hadoop.hive.ql.io.RCFileInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.RCFileOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
-PREHOOK: query: explain create table if not exists nzhang_ctas3 as select key, value from src sort by key, value limit 2
-PREHOOK: type: CREATETABLE
-POSTHOOK: query: explain create table if not exists nzhang_ctas3 as select key, value from src sort by key, value limit 2
-POSTHOOK: type: CREATETABLE
-STAGE DEPENDENCIES:
-
-STAGE PLANS:
-PREHOOK: query: create table if not exists nzhang_ctas3 as select key, value from src sort by key, value limit 2
-PREHOOK: type: CREATETABLE
-POSTHOOK: query: create table if not exists nzhang_ctas3 as select key, value from src sort by key, value limit 2
-POSTHOOK: type: CREATETABLE
-PREHOOK: query: select * from nzhang_ctas3
-PREHOOK: type: QUERY
-PREHOOK: Input: default@nzhang_ctas3
-#### A masked pattern was here ####
-POSTHOOK: query: select * from nzhang_ctas3
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@nzhang_ctas3
-#### A masked pattern was here ####
-0.0	val_0_con
-0.0	val_0_con
-0.0	val_0_con
-1.0	val_2_con
-2.0	val_4_con
-2.5	val_5_con
-2.5	val_5_con
-2.5	val_5_con
-4.0	val_8_con
-4.5	val_9_con
-PREHOOK: query: describe formatted nzhang_CTAS3
-PREHOOK: type: DESCTABLE
-PREHOOK: Input: default@nzhang_ctas3
-POSTHOOK: query: describe formatted nzhang_CTAS3
-POSTHOOK: type: DESCTABLE
-POSTHOOK: Input: default@nzhang_ctas3
-# col_name            	data_type           	comment             
-	 	 
-half_key            	double              	                    
-conb                	string              	                    
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-#### A masked pattern was here ####
-Retention:          	0                   	 
-#### A masked pattern was here ####
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	COLUMN_STATS_ACCURATE	true                
-	numFiles            	1                   
-	numRows             	10                  
-	rawDataSize         	120                 
-	totalSize           	199                 
-#### A masked pattern was here ####
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe	 
-InputFormat:        	org.apache.hadoop.hive.ql.io.RCFileInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.RCFileOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
-PREHOOK: query: explain create table nzhang_ctas4 row format delimited fields terminated by ',' stored as textfile as select key, value from src sort by key, value limit 10
-PREHOOK: type: CREATETABLE_AS_SELECT
-POSTHOOK: query: explain create table nzhang_ctas4 row format delimited fields terminated by ',' stored as textfile as select key, value from src sort by key, value limit 10
-POSTHOOK: type: CREATETABLE_AS_SELECT
-STAGE DEPENDENCIES:
-  Stage-1 is a root stage
-  Stage-2 depends on stages: Stage-1
-  Stage-0 depends on stages: Stage-2
-  Stage-4 depends on stages: Stage-0
-  Stage-3 depends on stages: Stage-4
-
-STAGE PLANS:
-  Stage: Stage-1
-    Map Reduce
-      Map Operator Tree:
-          TableScan
-            alias: src
-            Statistics: Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE
-            Select Operator
-              expressions: key (type: string), value (type: string)
-              outputColumnNames: _col0, _col1
-              Statistics: Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE
-              Reduce Output Operator
-                key expressions: _col0 (type: string), _col1 (type: string)
-                sort order: ++
-                Statistics: Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE
-      Reduce Operator Tree:
-        Select Operator
-          expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string)
-          outputColumnNames: _col0, _col1
-          Statistics: Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE
-          Limit
-            Number of rows: 10
-            Statistics: Num rows: 10 Data size: 2000 Basic stats: COMPLETE Column stats: NONE
-            File Output Operator
-              compressed: false
-              table:
-                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                  serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
-
-  Stage: Stage-2
-    Map Reduce
-      Map Operator Tree:
-          TableScan
-            Reduce Output Operator
-              key expressions: _col0 (type: string), _col1 (type: string)
-              sort order: ++
-              Statistics: Num rows: 10 Data size: 2000 Basic stats: COMPLETE Column stats: NONE
-      Reduce Operator Tree:
-        Select Operator
-          expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string)
-          outputColumnNames: _col0, _col1
-          Statistics: Num rows: 10 Data size: 2000 Basic stats: COMPLETE Column stats: NONE
-          Limit
-            Number of rows: 10
-            Statistics: Num rows: 10 Data size: 2000 Basic stats: COMPLETE Column stats: NONE
-            File Output Operator
-              compressed: false
-              Statistics: Num rows: 10 Data size: 2000 Basic stats: COMPLETE Column stats: NONE
-              table:
-                  input format: org.apache.hadoop.mapred.TextInputFormat
-                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                  name: default.nzhang_ctas4
-
-  Stage: Stage-0
-    Move Operator
-      files:
-          hdfs directory: true
-#### A masked pattern was here ####
-
-  Stage: Stage-4
-      Create Table Operator:
-        Create Table
-          columns: key string, value string
-          field delimiter: ,
-          input format: org.apache.hadoop.mapred.TextInputFormat
-          output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat
-          name: nzhang_ctas4
-
-  Stage: Stage-3
-    Stats-Aggr Operator
-
-PREHOOK: query: create table nzhang_ctas4 row format delimited fields terminated by ',' stored as textfile as select key, value from src sort by key, value limit 10
-PREHOOK: type: CREATETABLE_AS_SELECT
-PREHOOK: Input: default@src
-POSTHOOK: query: create table nzhang_ctas4 row format delimited fields terminated by ',' stored as textfile as select key, value from src sort by key, value limit 10
-POSTHOOK: type: CREATETABLE_AS_SELECT
-POSTHOOK: Input: default@src
-POSTHOOK: Output: default@nzhang_ctas4
-PREHOOK: query: select * from nzhang_ctas4
-PREHOOK: type: QUERY
-PREHOOK: Input: default@nzhang_ctas4
-#### A masked pattern was here ####
-POSTHOOK: query: select * from nzhang_ctas4
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@nzhang_ctas4
-#### A masked pattern was here ####
-0	val_0
-0	val_0
-0	val_0
-10	val_10
-100	val_100
-100	val_100
-103	val_103
-103	val_103
-104	val_104
-104	val_104
-PREHOOK: query: describe formatted nzhang_CTAS4
-PREHOOK: type: DESCTABLE
-PREHOOK: Input: default@nzhang_ctas4
-POSTHOOK: query: describe formatted nzhang_CTAS4
-POSTHOOK: type: DESCTABLE
-POSTHOOK: Input: default@nzhang_ctas4
-# col_name            	data_type           	comment             
-	 	 
-key                 	string              	                    
-value               	string              	                    
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-#### A masked pattern was here ####
-Retention:          	0                   	 
-#### A masked pattern was here ####
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	COLUMN_STATS_ACCURATE	true                
-	numFiles            	1                   
-	numRows             	10                  
-	rawDataSize         	96                  
-	totalSize           	106                 
-#### A masked pattern was here ####
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	field.delim         	,                   
-	serialization.format	,                   
-PREHOOK: query: explain extended create table nzhang_ctas5 row format delimited fields terminated by ',' lines terminated by '\012' stored as textfile as select key, value from src sort by key, value limit 10
-PREHOOK: type: CREATETABLE_AS_SELECT
-POSTHOOK: query: explain extended create table nzhang_ctas5 row format delimited fields terminated by ',' lines terminated by '\012' stored as textfile as select key, value from src sort by key, value limit 10
-POSTHOOK: type: CREATETABLE_AS_SELECT
-ABSTRACT SYNTAX TREE:
-  
-TOK_CREATETABLE
-   TOK_TABNAME
-      nzhang_ctas5
-   TOK_LIKETABLE
-   TOK_TABLEROWFORMAT
-      TOK_SERDEPROPS
-         TOK_TABLEROWFORMATFIELD
-            ','
-         TOK_TABLEROWFORMATLINES
-            '\012'
-   TOK_TBLTEXTFILE
-   TOK_QUERY
-      TOK_FROM
-         TOK_TABREF
-            TOK_TABNAME
-               src
-      TOK_INSERT
-         TOK_DESTINATION
-            TOK_DIR
-               TOK_TMP_FILE
-         TOK_SELECT
-            TOK_SELEXPR
-               TOK_TABLE_OR_COL
-                  key
-            TOK_SELEXPR
-               TOK_TABLE_OR_COL
-                  value
-         TOK_SORTBY
-            TOK_TABSORTCOLNAMEASC
-               TOK_TABLE_OR_COL
-                  key
-            TOK_TABSORTCOLNAMEASC
-               TOK_TABLE_OR_COL
-                  value
-         TOK_LIMIT
-            10
-
-
-STAGE DEPENDENCIES:
-  Stage-1 is a root stage
-  Stage-2 depends on stages: Stage-1
-  Stage-0 depends on stages: Stage-2
-  Stage-4 depends on stages: Stage-0
-  Stage-3 depends on stages: Stage-4
-
-STAGE PLANS:
-  Stage: Stage-1
-    Map Reduce
-      Map Operator Tree:
-          TableScan
-            alias: src
-            Statistics: Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE
-            GatherStats: false
-            Select Operator
-              expressions: key (type: string), value (type: string)
-              outputColumnNames: _col0, _col1
-              Statistics: Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE
-              Reduce Output Operator
-                key expressions: _col0 (type: string), _col1 (type: string)
-                sort order: ++
-                Statistics: Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE
-                tag: -1
-      Path -> Alias:
-#### A masked pattern was here ####
-      Path -> Partition:
-#### A masked pattern was here ####
-          Partition
-            base file name: src
-            input format: org.apache.hadoop.mapred.TextInputFormat
-            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-            properties:
-              COLUMN_STATS_ACCURATE true
-              bucket_count -1
-              columns key,value
-              columns.comments 'default','default'
-              columns.types string:string
-#### A masked pattern was here ####
-              name default.src
-              numFiles 1
-              numRows 0
-              rawDataSize 0
-              serialization.ddl struct src { string key, string value}
-              serialization.format 1
-              serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-              totalSize 5812
-#### A masked pattern was here ####
-            serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-          
-              input format: org.apache.hadoop.mapred.TextInputFormat
-              output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-              properties:
-                COLUMN_STATS_ACCURATE true
-                bucket_count -1
-                columns key,value
-                columns.comments 'default','default'
-                columns.types string:string
-#### A masked pattern was here ####
-                name default.src
-                numFiles 1
-                numRows 0
-                rawDataSize 0
-                serialization.ddl struct src { string key, string value}
-                serialization.format 1
-                serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                totalSize 5812
-#### A masked pattern was here ####
-              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-              name: default.src
-            name: default.src
-      Truncated Path -> Alias:
-        /src [src]
-      Needs Tagging: false
-      Reduce Operator Tree:
-        Select Operator
-          expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string)
-          outputColumnNames: _col0, _col1
-          Statistics: Num rows: 29 Data size: 5812 Basic stats: COMPLETE Column stats: NONE
-          Limit
-            Number of rows: 10
-            Statistics: Num rows: 10 Data size: 2000 Basic stats: COMPLETE Column stats: NONE
-            File Output Operator
-              compressed: false
-              GlobalTableId: 0
-#### A masked pattern was here ####
-              NumFilesPerFileSink: 1
-              table:
-                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                  properties:
-                    columns _col0,_col1
-                    columns.types string,string
-                    escape.delim \
-                    serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
-                  serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
-              TotalFiles: 1
-              GatherStats: false
-              MultiFileSpray: false
-
-  Stage: Stage-2
-    Map Reduce
-      Map Operator Tree:
-          TableScan
-            GatherStats: false
-            Reduce Output Operator
-              key expressions: _col0 (type: string), _col1 (type: string)
-              sort order: ++
-              Statistics: Num rows: 10 Data size: 2000 Basic stats: COMPLETE Column stats: NONE
-              tag: -1
-      Path -> Alias:
-#### A masked pattern was here ####
-      Path -> Partition:
-#### A masked pattern was here ####
-          Partition
-            base file name: -mr-10002
-            input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-            output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-            properties:
-              columns _col0,_col1
-              columns.types string,string
-              escape.delim \
-              serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
-            serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
-          
-              input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-              output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-              properties:
-                columns _col0,_col1
-                columns.types string,string
-                escape.delim \
-                serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
-              serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
-      Truncated Path -> Alias:
-#### A masked pattern was here ####
-      Needs Tagging: false
-      Reduce Operator Tree:
-        Select Operator
-          expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string)
-          outputColumnNames: _col0, _col1
-          Statistics: Num rows: 10 Data size: 2000 Basic stats: COMPLETE Column stats: NONE
-          Limit
-            Number of rows: 10
-            Statistics: Num rows: 10 Data size: 2000 Basic stats: COMPLETE Column stats: NONE
-            File Output Operator
-              compressed: false
-              GlobalTableId: 1
-#### A masked pattern was here ####
-              NumFilesPerFileSink: 1
-              Statistics: Num rows: 10 Data size: 2000 Basic stats: COMPLETE Column stats: NONE
-#### A masked pattern was here ####
-              table:
-                  input format: org.apache.hadoop.mapred.TextInputFormat
-                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                  properties:
-                    columns key,value
-                    columns.types string:string
-                    field.delim ,
-                    line.delim 
-
-                    name default.nzhang_ctas5
-                    serialization.format ,
-                    serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                  name: default.nzhang_ctas5
-              TotalFiles: 1
-              GatherStats: true
-              MultiFileSpray: false
-
-  Stage: Stage-0
-    Move Operator
-      files:
-          hdfs directory: true
-#### A masked pattern was here ####
-
-  Stage: Stage-4
-      Create Table Operator:
-        Create Table
-          columns: key string, value string
-          field delimiter: ,
-          input format: org.apache.hadoop.mapred.TextInputFormat
-          line delimiter: 
-
-          output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat
-          name: nzhang_ctas5
-
-  Stage: Stage-3
-    Stats-Aggr Operator
-#### A masked pattern was here ####
-
-PREHOOK: query: create table nzhang_ctas5 row format delimited fields terminated by ',' lines terminated by '\012' stored as textfile as select key, value from src sort by key, value limit 10
-PREHOOK: type: CREATETABLE_AS_SELECT
-PREHOOK: Input: default@src
-POSTHOOK: query: create table nzhang_ctas5 row format delimited fields terminated by ',' lines terminated by '\012' stored as textfile as select key, value from src sort by key, value limit 10
-POSTHOOK: type: CREATETABLE_AS_SELECT
-POSTHOOK: Input: default@src
-POSTHOOK: Output: default@nzhang_ctas5
-PREHOOK: query: create table nzhang_ctas6 (key string, `to` string)
-PREHOOK: type: CREATETABLE
-PREHOOK: Output: database:default
-POSTHOOK: query: create table nzhang_ctas6 (key string, `to` string)
-POSTHOOK: type: CREATETABLE
-POSTHOOK: Output: database:default
-POSTHOOK: Output: default@nzhang_ctas6
-PREHOOK: query: insert overwrite table nzhang_ctas6 select key, value from src limit 10
-PREHOOK: type: QUERY
-PREHOOK: Input: default@src
-PREHOOK: Output: default@nzhang_ctas6
-POSTHOOK: query: insert overwrite table nzhang_ctas6 select key, value from src limit 10
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@src
-POSTHOOK: Output: default@nzhang_ctas6
-POSTHOOK: Lineage: nzhang_ctas6.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: nzhang_ctas6.to SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
-PREHOOK: query: create table nzhang_ctas7 as select key, `to` from nzhang_ctas6
-PREHOOK: type: CREATETABLE_AS_SELECT
-PREHOOK: Input: default@nzhang_ctas6
-POSTHOOK: query: create table nzhang_ctas7 as select key, `to` from nzhang_ctas6
-POSTHOOK: type: CREATETABLE_AS_SELECT
-POSTHOOK: Input: default@nzhang_ctas6
-POSTHOOK: Output: default@nzhang_ctas7
-PREHOOK: query: create table nzhang_ctas8 as select 3.14BD from nzhang_ctas6 limit 1
-PREHOOK: type: CREATETABLE_AS_SELECT
-PREHOOK: Input: default@nzhang_ctas6
-POSTHOOK: query: create table nzhang_ctas8 as select 3.14BD from nzhang_ctas6 limit 1
-POSTHOOK: type: CREATETABLE_AS_SELECT
-POSTHOOK: Input: default@nzhang_ctas6
-POSTHOOK: Output: default@nzhang_ctas8
-PREHOOK: query: desc nzhang_ctas8
-PREHOOK: type: DESCTABLE
-PREHOOK: Input: default@nzhang_ctas8
-POSTHOOK: query: desc nzhang_ctas8
-POSTHOOK: type: DESCTABLE
-POSTHOOK: Input: default@nzhang_ctas8
-_c0                 	decimal(3,2)        	                    
-PREHOOK: query: drop table nzhang_ctas8
-PREHOOK: type: DROPTABLE
-PREHOOK: Input: default@nzhang_ctas8
-PREHOOK: Output: default@nzhang_ctas8
-POSTHOOK: query: drop table nzhang_ctas8
-POSTHOOK: type: DROPTABLE
-POSTHOOK: Input: default@nzhang_ctas8
-POSTHOOK: Output: default@nzhang_ctas8

http://git-wip-us.apache.org/repos/asf/hive/blob/e1fa2787/ql/src/test/results/clientpositive/llap/ctas.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/ctas.q.out b/ql/src/test/results/clientpositive/llap/ctas.q.out
index dab0251..e2e7640 100644
--- a/ql/src/test/results/clientpositive/llap/ctas.q.out
+++ b/ql/src/test/results/clientpositive/llap/ctas.q.out
@@ -710,9 +710,9 @@ Sort Columns:       	[]
 Storage Desc Params:	 	 
 	field.delim         	,                   
 	serialization.format	,                   
-PREHOOK: query: explain extended create table nzhang_ctas5 row format delimited fields terminated by ',' lines terminated by '\012' stored as textfile as select key, value from src sort by key, value limit 10
+PREHOOK: query: explain create table nzhang_ctas5 row format delimited fields terminated by ',' lines terminated by '\012' stored as textfile as select key, value from src sort by key, value limit 10
 PREHOOK: type: CREATETABLE_AS_SELECT
-POSTHOOK: query: explain extended create table nzhang_ctas5 row format delimited fields terminated by ',' lines terminated by '\012' stored as textfile as select key, value from src sort by key, value limit 10
+POSTHOOK: query: explain create table nzhang_ctas5 row format delimited fields terminated by ',' lines terminated by '\012' stored as textfile as select key, value from src sort by key, value limit 10
 POSTHOOK: type: CREATETABLE_AS_SELECT
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
@@ -735,74 +735,19 @@ STAGE PLANS:
                 TableScan
                   alias: src
                   Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                  GatherStats: false
                   Select Operator
                     expressions: key (type: string), value (type: string)
                     outputColumnNames: _col0, _col1
                     Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
                       key expressions: _col0 (type: string), _col1 (type: string)
-                      null sort order: aa
                       sort order: ++
                       Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
-                      tag: -1
-                      TopN: 10
                       TopN Hash Memory Usage: 0.1
-                      auto parallelism: true
             Execution mode: llap
             LLAP IO: no inputs
-            Path -> Alias:
-#### A masked pattern was here ####
-            Path -> Partition:
-#### A masked pattern was here ####
-                Partition
-                  base file name: src
-                  input format: org.apache.hadoop.mapred.TextInputFormat
-                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                  properties:
-                    COLUMN_STATS_ACCURATE {"BASIC_STATS":"true","COLUMN_STATS":{"key":"true","value":"true"}}
-                    bucket_count -1
-                    columns key,value
-                    columns.comments 'default','default'
-                    columns.types string:string
-#### A masked pattern was here ####
-                    name default.src
-                    numFiles 1
-                    numRows 500
-                    rawDataSize 5312
-                    serialization.ddl struct src { string key, string value}
-                    serialization.format 1
-                    serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                    totalSize 5812
-#### A masked pattern was here ####
-                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                
-                    input format: org.apache.hadoop.mapred.TextInputFormat
-                    output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                    properties:
-                      COLUMN_STATS_ACCURATE {"BASIC_STATS":"true","COLUMN_STATS":{"key":"true","value":"true"}}
-                      bucket_count -1
-                      columns key,value
-                      columns.comments 'default','default'
-                      columns.types string:string
-#### A masked pattern was here ####
-                      name default.src
-                      numFiles 1
-                      numRows 500
-                      rawDataSize 5312
-                      serialization.ddl struct src { string key, string value}
-                      serialization.format 1
-                      serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                      totalSize 5812
-#### A masked pattern was here ####
-                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                    name: default.src
-                  name: default.src
-            Truncated Path -> Alias:
-              /src [src]
         Reducer 2 
             Execution mode: llap
-            Needs Tagging: false
             Reduce Operator Tree:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string)
@@ -813,16 +758,11 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 1780 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
                     key expressions: _col0 (type: string), _col1 (type: string)
-                    null sort order: aa
                     sort order: ++
                     Statistics: Num rows: 10 Data size: 1780 Basic stats: COMPLETE Column stats: COMPLETE
-                    tag: -1
-                    TopN: 10
                     TopN Hash Memory Usage: 0.1
-                    auto parallelism: false
         Reducer 3 
             Execution mode: llap
-            Needs Tagging: false
             Reduce Operator Tree:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string)
@@ -833,28 +773,12 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 1780 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
                     compressed: false
-                    GlobalTableId: 1
-#### A masked pattern was here ####
-                    NumFilesPerFileSink: 1
                     Statistics: Num rows: 10 Data size: 1780 Basic stats: COMPLETE Column stats: COMPLETE
-#### A masked pattern was here ####
                     table:
                         input format: org.apache.hadoop.mapred.TextInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                        properties:
-                          columns key,value
-                          columns.types string:string
-                          field.delim ,
-                          line.delim 
-
-                          name default.nzhang_ctas5
-                          serialization.format ,
-                          serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
                         serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
                         name: default.nzhang_ctas5
-                    TotalFiles: 1
-                    GatherStats: true
-                    MultiFileSpray: false
 
   Stage: Stage-2
     Dependency Collection
@@ -873,7 +797,6 @@ STAGE PLANS:
 
   Stage: Stage-3
     Stats-Aggr Operator
-#### A masked pattern was here ####
 
   Stage: Stage-0
     Move Operator

http://git-wip-us.apache.org/repos/asf/hive/blob/e1fa2787/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 7a44cc7..a723365 100644
--- a/ql/src/test/results/clientpositive/spark/ctas.q.out
+++ b/ql/src/test/results/clientpositive/spark/ctas.q.out
@@ -674,9 +674,9 @@ Sort Columns:       	[]
 Storage Desc Params:	 	 
 	field.delim         	,                   
 	serialization.format	,                   
-PREHOOK: query: explain extended create table nzhang_ctas5 row format delimited fields terminated by ',' lines terminated by '\012' stored as textfile as select key, value from src sort by key, value limit 10
+PREHOOK: query: explain create table nzhang_ctas5 row format delimited fields terminated by ',' lines terminated by '\012' stored as textfile as select key, value from src sort by key, value limit 10
 PREHOOK: type: CREATETABLE_AS_SELECT
-POSTHOOK: query: explain extended create table nzhang_ctas5 row format delimited fields terminated by ',' lines terminated by '\012' stored as textfile as select key, value from src sort by key, value limit 10
+POSTHOOK: query: explain create table nzhang_ctas5 row format delimited fields terminated by ',' lines terminated by '\012' stored as textfile as select key, value from src sort by key, value limit 10
 POSTHOOK: type: CREATETABLE_AS_SELECT
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
@@ -697,71 +697,16 @@ STAGE PLANS:
                 TableScan
                   alias: src
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
-                  GatherStats: false
                   Select Operator
                     expressions: key (type: string), value (type: string)
                     outputColumnNames: _col0, _col1
                     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
-                      TopN: 10
                       TopN Hash Memory Usage: 0.1
-                      auto parallelism: false
-            Path -> Alias:
-#### A masked pattern was here ####
-            Path -> Partition:
-#### A masked pattern was here ####
-                Partition
-                  base file name: src
-                  input format: org.apache.hadoop.mapred.TextInputFormat
-                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                  properties:
-                    COLUMN_STATS_ACCURATE {"BASIC_STATS":"true","COLUMN_STATS":{"key":"true","value":"true"}}
-                    bucket_count -1
-                    columns key,value
-                    columns.comments 'default','default'
-                    columns.types string:string
-#### A masked pattern was here ####
-                    name default.src
-                    numFiles 1
-                    numRows 500
-                    rawDataSize 5312
-                    serialization.ddl struct src { string key, string value}
-                    serialization.format 1
-                    serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                    totalSize 5812
-#### A masked pattern was here ####
-                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                
-                    input format: org.apache.hadoop.mapred.TextInputFormat
-                    output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                    properties:
-                      COLUMN_STATS_ACCURATE {"BASIC_STATS":"true","COLUMN_STATS":{"key":"true","value":"true"}}
-                      bucket_count -1
-                      columns key,value
-                      columns.comments 'default','default'
-                      columns.types string:string
-#### A masked pattern was here ####
-                      name default.src
-                      numFiles 1
-                      numRows 500
-                      rawDataSize 5312
-                      serialization.ddl struct src { string key, string value}
-                      serialization.format 1
-                      serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                      totalSize 5812
-#### A masked pattern was here ####
-                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                    name: default.src
-                  name: default.src
-            Truncated Path -> Alias:
-              /src [src]
         Reducer 2 
-            Needs Tagging: false
             Reduce Operator Tree:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string)
@@ -772,15 +717,10 @@ 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
-                    TopN: 10
                     TopN Hash Memory Usage: 0.1
-                    auto parallelism: false
         Reducer 3 
-            Needs Tagging: false
             Reduce Operator Tree:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string)
@@ -791,28 +731,12 @@ STAGE PLANS:
                   Statistics: Num rows: 10 Data size: 100 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
                     compressed: false
-                    GlobalTableId: 1
-#### A masked pattern was here ####
-                    NumFilesPerFileSink: 1
                     Statistics: Num rows: 10 Data size: 100 Basic stats: COMPLETE Column stats: NONE
-#### A masked pattern was here ####
                     table:
                         input format: org.apache.hadoop.mapred.TextInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                        properties:
-                          columns key,value
-                          columns.types string:string
-                          field.delim ,
-                          line.delim 
-
-                          name default.nzhang_ctas5
-                          serialization.format ,
-                          serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
                         serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
                         name: default.nzhang_ctas5
-                    TotalFiles: 1
-                    GatherStats: true
-                    MultiFileSpray: false
 
   Stage: Stage-0
     Move Operator
@@ -834,7 +758,6 @@ STAGE PLANS:
 
   Stage: Stage-2
     Stats-Aggr Operator
-#### A masked pattern was here ####
 
 PREHOOK: query: create table nzhang_ctas5 row format delimited fields terminated by ',' lines terminated by '\012' stored as textfile as select key, value from src sort by key, value limit 10
 PREHOOK: type: CREATETABLE_AS_SELECT

http://git-wip-us.apache.org/repos/asf/hive/blob/e1fa2787/ql/src/test/results/clientpositive/vector_join_part_col_char.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_join_part_col_char.q.out b/ql/src/test/results/clientpositive/vector_join_part_col_char.q.out
index b6631f7..6f6efc6 100644
--- a/ql/src/test/results/clientpositive/vector_join_part_col_char.q.out
+++ b/ql/src/test/results/clientpositive/vector_join_part_col_char.q.out
@@ -134,7 +134,7 @@ STAGE PLANS:
                 sort order: +
                 Map-reduce partition columns: _col2 (type: char(50))
                 Statistics: Num rows: 2 Data size: 203 Basic stats: COMPLETE Column stats: NONE
-                value expressions: _col0 (type: string), _col1 (type: int), _col2 (type: char(5))
+                value expressions: _col0 (type: string), _col1 (type: int)
       Reduce Operator Tree:
         Join Operator
           condition map:
@@ -148,8 +148,8 @@ STAGE PLANS:
             compressed: false
             Statistics: Num rows: 2 Data size: 224 Basic stats: COMPLETE Column stats: NONE
             table:
-                input format: org.apache.hadoop.mapred.TextInputFormat
-                output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
 
   Stage: Stage-0


[10/22] hive git commit: HIVE-14889: Beeline leaks sensitive environment variables of HiveServer2 when you type set; (Vihang Karajgaonkar, reviewed by Sergio Pena)

Posted by se...@apache.org.
HIVE-14889: Beeline leaks sensitive environment variables of HiveServer2 when you type set; (Vihang Karajgaonkar, reviewed by Sergio Pena)


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

Branch: refs/heads/hive-14535
Commit: 15039c05b75ddd955dac0fe66b9aef486a45f1a1
Parents: e1fa278
Author: Sergio Pena <se...@cloudera.com>
Authored: Fri Oct 7 10:05:03 2016 -0500
Committer: Sergio Pena <se...@cloudera.com>
Committed: Fri Oct 7 10:05:03 2016 -0500

----------------------------------------------------------------------
 .../hadoop/hive/ql/processors/SetProcessor.java |  33 +++++-
 .../hive/ql/processors/TestSetProcessor.java    | 114 ++++++++++++++++---
 2 files changed, 129 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/15039c05/ql/src/java/org/apache/hadoop/hive/ql/processors/SetProcessor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/processors/SetProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/processors/SetProcessor.java
index c9d06ba..eab1886 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/processors/SetProcessor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/processors/SetProcessor.java
@@ -54,6 +54,8 @@ public class SetProcessor implements CommandProcessor {
   private static final String prefix = "set: ";
   private static final Set<String> removedConfigs = Sets.newHashSet("hive.mapred.supports.subdirectories","hive.enforce.sorting","hive.enforce.bucketing", "hive.outerjoin.supports.filters");
 
+  private static final String[] PASSWORD_STRINGS = new String[] {"password", "paswd", "pswd"};
+
   public static boolean getBoolean(String value) {
     if (value.equals("on") || value.equals("true")) {
       return true;
@@ -88,16 +90,35 @@ public class SetProcessor implements CommandProcessor {
     }
 
     for (Map.Entry<String, String> entry : mapToSortedMap(System.getenv()).entrySet()) {
+      if(isHidden(entry.getKey())) {
+        continue;
+      }
       ss.out.println(ENV_PREFIX+entry.getKey() + "=" + entry.getValue());
     }
 
     for (Map.Entry<String, String> entry :
       propertiesToSortedMap(System.getProperties()).entrySet() ) {
+      if(isHidden(entry.getKey())) {
+        continue;
+      }
       ss.out.println(SYSTEM_PREFIX+entry.getKey() + "=" + entry.getValue());
     }
 
   }
 
+  /*
+   * Checks if the value contains any of the PASSWORD_STRINGS and if yes
+   * return true
+   */
+  private boolean isHidden(String key) {
+    for(String p : PASSWORD_STRINGS) {
+      if(key.toLowerCase().contains(p)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
   private void dumpOption(String s) {
     SessionState ss = SessionState.get();
 
@@ -249,7 +270,11 @@ public class SetProcessor implements CommandProcessor {
       String propName = varname.substring(SYSTEM_PREFIX.length());
       String result = System.getProperty(propName);
       if (result != null) {
-        ss.out.println(SYSTEM_PREFIX + propName + "=" + result);
+        if(isHidden(propName)) {
+          ss.out.println(SYSTEM_PREFIX + propName + " is a hidden config");
+        } else {
+          ss.out.println(SYSTEM_PREFIX + propName + "=" + result);
+        }
         return createProcessorSuccessResponse();
       } else {
         ss.out.println(propName + " is undefined as a system property");
@@ -258,7 +283,11 @@ public class SetProcessor implements CommandProcessor {
     } else if (varname.indexOf(ENV_PREFIX) == 0) {
       String var = varname.substring(ENV_PREFIX.length());
       if (System.getenv(var) != null) {
-        ss.out.println(ENV_PREFIX + var + "=" + System.getenv(var));
+        if(isHidden(var)) {
+          ss.out.println(ENV_PREFIX + var + " is a hidden config");
+        } else {
+          ss.out.println(ENV_PREFIX + var + "=" + System.getenv(var));
+        }
         return createProcessorSuccessResponse();
       } else {
         ss.out.println(varname + " is undefined as an environmental variable");

http://git-wip-us.apache.org/repos/asf/hive/blob/15039c05/ql/src/test/org/apache/hadoop/hive/ql/processors/TestSetProcessor.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/processors/TestSetProcessor.java b/ql/src/test/org/apache/hadoop/hive/ql/processors/TestSetProcessor.java
index bff643a..4eb4941 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/processors/TestSetProcessor.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/processors/TestSetProcessor.java
@@ -20,35 +20,117 @@ package org.apache.hadoop.hive.ql.processors;
 
 import java.io.ByteArrayOutputStream;
 import java.io.PrintStream;
-
-import junit.framework.Assert;
+import java.lang.reflect.Field;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
 
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.SystemVariables;
 import org.apache.hadoop.hive.ql.session.SessionState;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 public class TestSetProcessor {
 
-  @Test
-  public void testHiddenConfig() throws Exception {
+  private static final String TEST_SYSTEM_PROPERTY = "testSystemPropertyPassword";
+  private static final String TEST_SYSTEM_PROPERTY_VALUE = "testSystemPropertyValue";
+  private static final String TEST_ENV_VAR_PASSWORD_VALUE = "testEnvPasswordValue";
+  private static final String TEST_ENV_VAR_PASSWORD = "testEnvPassword";
+  private ByteArrayOutputStream baos;
+  private static SessionState state;
+  private SetProcessor processor;
+
+  @BeforeClass
+  public static void before() throws Exception {
+    Map<String, String> env = new HashMap<>();
+    env.put(TEST_ENV_VAR_PASSWORD, TEST_ENV_VAR_PASSWORD_VALUE);
+    setEnv(env);
+    System.setProperty(TEST_SYSTEM_PROPERTY, TEST_SYSTEM_PROPERTY_VALUE);
     HiveConf conf = new HiveConf();
     SessionState.start(conf);
-    SessionState state = SessionState.get();
-    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    state.out = new PrintStream(baos);
-    SetProcessor processor = new SetProcessor();
-    processor.run("");
-    state.out.flush();
-    String ret = baos.toString();
-    Assert.assertFalse(ret.contains(HiveConf.ConfVars.METASTOREPWD.varname + "="));
-    Assert.assertFalse(ret.contains(HiveConf.ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PASSWORD.varname + "="));
+    state = SessionState.get();
+  }
 
+  @Before
+  public void setupTest() {
     baos = new ByteArrayOutputStream();
     state.out = new PrintStream(baos);
-    processor.run(HiveConf.ConfVars.METASTOREPWD.varname);
+    processor = new SetProcessor();
+  }
+
+  @Test
+  public void testHiddenConfig() throws Exception {
+    runSetProcessor("");
+    String output = baos.toString();
+    Assert.assertFalse(output.contains(HiveConf.ConfVars.METASTOREPWD.varname + "="));
+    Assert.assertFalse(output.contains(HiveConf.ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PASSWORD.varname + "="));
+  }
+
+  @Test
+  public void testHiddenConfigSetVarName() {
+    runSetProcessor(HiveConf.ConfVars.METASTOREPWD.varname);
+    String output = baos.toString();
+    Assert.assertTrue(output.contains("hidden"));
+  }
+
+  @Test
+  public void testEnvPasswordMask() throws Exception {
+    runSetProcessor("");
+    String output = baos.toString();
+    Assert.assertFalse(output.contains(TEST_ENV_VAR_PASSWORD + "="));
+  }
+
+  @Test
+  public void testEnvPasswordMaskIndividual() throws Exception {
+    runSetProcessor(SystemVariables.ENV_PREFIX + TEST_ENV_VAR_PASSWORD);
+    String output = baos.toString();
+    Assert.assertFalse(output.contains(TEST_ENV_VAR_PASSWORD_VALUE));
+    Assert.assertTrue(output.contains("hidden"));
+  }
+
+  @Test
+  public void testSystemProperty() throws Exception {
+    runSetProcessor("");
+    String output = baos.toString();
+    Assert.assertFalse(output.contains(TEST_SYSTEM_PROPERTY + "="));
+  }
+
+  @Test
+  public void testSystemPropertyIndividual() throws Exception {
+    runSetProcessor(SystemVariables.SYSTEM_PREFIX + TEST_SYSTEM_PROPERTY);
+    String output = baos.toString();
+    Assert.assertFalse(output.contains(TEST_SYSTEM_PROPERTY_VALUE));
+    Assert.assertTrue(output.contains("hidden"));
+  }
+
+  /*
+   * Simulates the set <command>;
+   */
+  private void runSetProcessor(String command) {
+    processor.run(command);
     state.out.flush();
-    ret = new String(baos.toByteArray());
-    Assert.assertTrue(baos.toString().contains("hidden"));
+  }
+
+  /*
+   * Dirty hack to set the environment variables using reflection code. This method is for testing
+   * purposes only and should not be used elsewhere
+   */
+  private final static void setEnv(Map<String, String> newenv) throws Exception {
+    Class[] classes = Collections.class.getDeclaredClasses();
+    Map<String, String> env = System.getenv();
+    for (Class cl : classes) {
+      if ("java.util.Collections$UnmodifiableMap".equals(cl.getName())) {
+        Field field = cl.getDeclaredField("m");
+        field.setAccessible(true);
+        Object obj = field.get(env);
+        Map<String, String> map = (Map<String, String>) obj;
+        map.clear();
+        map.putAll(newenv);
+      }
+    }
   }
 
 }


[06/22] hive git commit: HIVE-14773: NPE aggregating column statistics for date column in partitioned table (Pengcheng Xiong, reviewed by Gopal V)

Posted by se...@apache.org.
HIVE-14773: NPE aggregating column statistics for date column in partitioned table (Pengcheng Xiong, reviewed by Gopal V)


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

Branch: refs/heads/hive-14535
Commit: c53c9be7181fc47bb5422473edbba1ad9ae81042
Parents: 5953988
Author: Pengcheng Xiong <px...@apache.org>
Authored: Wed Oct 5 10:40:53 2016 -0700
Committer: Pengcheng Xiong <px...@apache.org>
Committed: Wed Oct 5 10:41:00 2016 -0700

----------------------------------------------------------------------
 .../hive/metastore/IExtrapolatePartStatus.java  |   1 +
 .../hive/metastore/StatObjectConverter.java     |  29 ++
 .../extrapolate_part_stats_date.q               |  14 +
 .../extrapolate_part_stats_date.q.out           | 302 +++++++++++++++++++
 4 files changed, 346 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/c53c9be7/metastore/src/java/org/apache/hadoop/hive/metastore/IExtrapolatePartStatus.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/IExtrapolatePartStatus.java b/metastore/src/java/org/apache/hadoop/hive/metastore/IExtrapolatePartStatus.java
index 4859cff..d0569fb 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/IExtrapolatePartStatus.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/IExtrapolatePartStatus.java
@@ -39,6 +39,7 @@ public interface IExtrapolatePartStatus {
       put("int", new Integer[] { 0, 1, 6, 7, 12, 15 });
       put("smallint", new Integer[] { 0, 1, 6, 7, 12, 15 });
       put("tinyint", new Integer[] { 0, 1, 6, 7, 12, 15 });
+      put("date", new Integer[] { 0, 1, 6, 7, 12, 15 });
       put("timestamp", new Integer[] { 0, 1, 6, 7, 12, 15 });
       put("long", new Integer[] { 0, 1, 6, 7, 12, 15 });
       put("double", new Integer[] { 2, 3, 6, 7, 13, 15 });

http://git-wip-us.apache.org/repos/asf/hive/blob/c53c9be7/metastore/src/java/org/apache/hadoop/hive/metastore/StatObjectConverter.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/StatObjectConverter.java b/metastore/src/java/org/apache/hadoop/hive/metastore/StatObjectConverter.java
index e119dd8..b259dfa 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/StatObjectConverter.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/StatObjectConverter.java
@@ -579,6 +579,35 @@ public class StatObjectConverter {
         longStats.setNumDVs(lowerBound);
       }
       data.setLongStats(longStats);
+    } else if (colType.equals("date")) {
+      DateColumnStatsData dateStats = new DateColumnStatsData();
+      dateStats.setNumNulls(MetaStoreDirectSql.extractSqlLong(nulls));
+      if (lhigh != null) {
+        dateStats.setHighValue(new Date(MetaStoreDirectSql.extractSqlLong(lhigh)));
+      }
+      if (llow != null) {
+        dateStats.setLowValue(new Date(MetaStoreDirectSql.extractSqlLong(llow)));
+      }
+      long lowerBound = MetaStoreDirectSql.extractSqlLong(dist);
+      long higherBound = MetaStoreDirectSql.extractSqlLong(sumDist);
+      if (useDensityFunctionForNDVEstimation && lhigh != null && llow != null && avgLong != null
+          && MetaStoreDirectSql.extractSqlDouble(avgLong) != 0.0) {
+        // We have estimation, lowerbound and higherbound. We use estimation if
+        // it is between lowerbound and higherbound.
+        long estimation = MetaStoreDirectSql
+            .extractSqlLong((MetaStoreDirectSql.extractSqlLong(lhigh) - MetaStoreDirectSql
+                .extractSqlLong(llow)) / MetaStoreDirectSql.extractSqlDouble(avgLong));
+        if (estimation < lowerBound) {
+          dateStats.setNumDVs(lowerBound);
+        } else if (estimation > higherBound) {
+          dateStats.setNumDVs(higherBound);
+        } else {
+          dateStats.setNumDVs(estimation);
+        }
+      } else {
+        dateStats.setNumDVs(lowerBound);
+      }
+      data.setDateStats(dateStats);
     } else if (colType.equals("double") || colType.equals("float")) {
       DoubleColumnStatsData doubleStats = new DoubleColumnStatsData();
       doubleStats.setNumNulls(MetaStoreDirectSql.extractSqlLong(nulls));

http://git-wip-us.apache.org/repos/asf/hive/blob/c53c9be7/ql/src/test/queries/clientpositive/extrapolate_part_stats_date.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/extrapolate_part_stats_date.q b/ql/src/test/queries/clientpositive/extrapolate_part_stats_date.q
new file mode 100644
index 0000000..1f38a65
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/extrapolate_part_stats_date.q
@@ -0,0 +1,14 @@
+set hive.exec.dynamic.partition.mode=nonstrict;
+set hive.stats.fetch.column.stats=true;
+
+create table date_dim (d_date date) partitioned by (d_date_sk bigint) stored as orc;
+insert into date_dim partition(d_date_sk=2416945) values('1905-04-09');
+insert into date_dim partition(d_date_sk=2416946) values('1905-04-10');
+insert into date_dim partition(d_date_sk=2416947) values('1905-04-11');
+analyze table date_dim partition(d_date_sk) compute statistics for columns;
+
+explain select count(*) from date_dim where d_date > date "1900-01-02" and d_date_sk= 2416945;
+
+insert into date_dim partition(d_date_sk=2416948) values('1905-04-12');
+
+explain extended select d_date from date_dim;

http://git-wip-us.apache.org/repos/asf/hive/blob/c53c9be7/ql/src/test/results/clientpositive/extrapolate_part_stats_date.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/extrapolate_part_stats_date.q.out b/ql/src/test/results/clientpositive/extrapolate_part_stats_date.q.out
new file mode 100644
index 0000000..1dce4df
--- /dev/null
+++ b/ql/src/test/results/clientpositive/extrapolate_part_stats_date.q.out
@@ -0,0 +1,302 @@
+PREHOOK: query: create table date_dim (d_date date) partitioned by (d_date_sk bigint) stored as orc
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@date_dim
+POSTHOOK: query: create table date_dim (d_date date) partitioned by (d_date_sk bigint) stored as orc
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@date_dim
+PREHOOK: query: insert into date_dim partition(d_date_sk=2416945) values('1905-04-09')
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__1
+PREHOOK: Output: default@date_dim@d_date_sk=2416945
+POSTHOOK: query: insert into date_dim partition(d_date_sk=2416945) values('1905-04-09')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__1
+POSTHOOK: Output: default@date_dim@d_date_sk=2416945
+POSTHOOK: Lineage: date_dim PARTITION(d_date_sk=2416945).d_date EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+PREHOOK: query: insert into date_dim partition(d_date_sk=2416946) values('1905-04-10')
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__2
+PREHOOK: Output: default@date_dim@d_date_sk=2416946
+POSTHOOK: query: insert into date_dim partition(d_date_sk=2416946) values('1905-04-10')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__2
+POSTHOOK: Output: default@date_dim@d_date_sk=2416946
+POSTHOOK: Lineage: date_dim PARTITION(d_date_sk=2416946).d_date EXPRESSION [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+PREHOOK: query: insert into date_dim partition(d_date_sk=2416947) values('1905-04-11')
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__3
+PREHOOK: Output: default@date_dim@d_date_sk=2416947
+POSTHOOK: query: insert into date_dim partition(d_date_sk=2416947) values('1905-04-11')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__3
+POSTHOOK: Output: default@date_dim@d_date_sk=2416947
+POSTHOOK: Lineage: date_dim PARTITION(d_date_sk=2416947).d_date EXPRESSION [(values__tmp__table__3)values__tmp__table__3.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+PREHOOK: query: analyze table date_dim partition(d_date_sk) compute statistics for columns
+PREHOOK: type: QUERY
+PREHOOK: Input: default@date_dim
+PREHOOK: Input: default@date_dim@d_date_sk=2416945
+PREHOOK: Input: default@date_dim@d_date_sk=2416946
+PREHOOK: Input: default@date_dim@d_date_sk=2416947
+#### A masked pattern was here ####
+POSTHOOK: query: analyze table date_dim partition(d_date_sk) compute statistics for columns
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@date_dim
+POSTHOOK: Input: default@date_dim@d_date_sk=2416945
+POSTHOOK: Input: default@date_dim@d_date_sk=2416946
+POSTHOOK: Input: default@date_dim@d_date_sk=2416947
+#### A masked pattern was here ####
+PREHOOK: query: explain select count(*) from date_dim where d_date > date "1900-01-02" and d_date_sk= 2416945
+PREHOOK: type: QUERY
+POSTHOOK: query: explain select count(*) from date_dim where d_date > date "1900-01-02" and d_date_sk= 2416945
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: date_dim
+            Statistics: Num rows: 1 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE
+            Filter Operator
+              predicate: (d_date > 1900-01-02) (type: boolean)
+              Statistics: Num rows: 1 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE
+              Select Operator
+                Statistics: Num rows: 1 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE
+                Group By Operator
+                  aggregations: count()
+                  mode: hash
+                  outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                  Reduce Output Operator
+                    sort order: 
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+                    value expressions: _col0 (type: bigint)
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations: count(VALUE._col0)
+          mode: mergepartial
+          outputColumnNames: _col0
+          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+          File Output Operator
+            compressed: false
+            Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+            table:
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: insert into date_dim partition(d_date_sk=2416948) values('1905-04-12')
+PREHOOK: type: QUERY
+PREHOOK: Input: default@values__tmp__table__4
+PREHOOK: Output: default@date_dim@d_date_sk=2416948
+POSTHOOK: query: insert into date_dim partition(d_date_sk=2416948) values('1905-04-12')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@values__tmp__table__4
+POSTHOOK: Output: default@date_dim@d_date_sk=2416948
+POSTHOOK: Lineage: date_dim PARTITION(d_date_sk=2416948).d_date EXPRESSION [(values__tmp__table__4)values__tmp__table__4.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
+PREHOOK: query: explain extended select d_date from date_dim
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended select d_date from date_dim
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Partition Description:
+          Partition
+            input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+            output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+            partition values:
+              d_date_sk 2416945
+            properties:
+              COLUMN_STATS_ACCURATE {"BASIC_STATS":"true","COLUMN_STATS":{"d_date":"true"}}
+              bucket_count -1
+              columns d_date
+              columns.comments 
+              columns.types date
+#### A masked pattern was here ####
+              name default.date_dim
+              numFiles 1
+              numRows 1
+              partition_columns d_date_sk
+              partition_columns.types bigint
+              rawDataSize 56
+              serialization.ddl struct date_dim { date d_date}
+              serialization.format 1
+              serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
+              totalSize 193
+#### A masked pattern was here ####
+            serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+          
+              input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+              output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+              properties:
+                bucket_count -1
+                columns d_date
+                columns.comments 
+                columns.types date
+#### A masked pattern was here ####
+                name default.date_dim
+                partition_columns d_date_sk
+                partition_columns.types bigint
+                serialization.ddl struct date_dim { date d_date}
+                serialization.format 1
+                serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
+#### A masked pattern was here ####
+              serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+              name: default.date_dim
+            name: default.date_dim
+          Partition
+            input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+            output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+            partition values:
+              d_date_sk 2416946
+            properties:
+              COLUMN_STATS_ACCURATE {"BASIC_STATS":"true","COLUMN_STATS":{"d_date":"true"}}
+              bucket_count -1
+              columns d_date
+              columns.comments 
+              columns.types date
+#### A masked pattern was here ####
+              name default.date_dim
+              numFiles 1
+              numRows 1
+              partition_columns d_date_sk
+              partition_columns.types bigint
+              rawDataSize 56
+              serialization.ddl struct date_dim { date d_date}
+              serialization.format 1
+              serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
+              totalSize 193
+#### A masked pattern was here ####
+            serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+          
+              input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+              output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+              properties:
+                bucket_count -1
+                columns d_date
+                columns.comments 
+                columns.types date
+#### A masked pattern was here ####
+                name default.date_dim
+                partition_columns d_date_sk
+                partition_columns.types bigint
+                serialization.ddl struct date_dim { date d_date}
+                serialization.format 1
+                serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
+#### A masked pattern was here ####
+              serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+              name: default.date_dim
+            name: default.date_dim
+          Partition
+            input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+            output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+            partition values:
+              d_date_sk 2416947
+            properties:
+              COLUMN_STATS_ACCURATE {"BASIC_STATS":"true","COLUMN_STATS":{"d_date":"true"}}
+              bucket_count -1
+              columns d_date
+              columns.comments 
+              columns.types date
+#### A masked pattern was here ####
+              name default.date_dim
+              numFiles 1
+              numRows 1
+              partition_columns d_date_sk
+              partition_columns.types bigint
+              rawDataSize 56
+              serialization.ddl struct date_dim { date d_date}
+              serialization.format 1
+              serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
+              totalSize 193
+#### A masked pattern was here ####
+            serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+          
+              input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+              output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+              properties:
+                bucket_count -1
+                columns d_date
+                columns.comments 
+                columns.types date
+#### A masked pattern was here ####
+                name default.date_dim
+                partition_columns d_date_sk
+                partition_columns.types bigint
+                serialization.ddl struct date_dim { date d_date}
+                serialization.format 1
+                serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
+#### A masked pattern was here ####
+              serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+              name: default.date_dim
+            name: default.date_dim
+          Partition
+            input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+            output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+            partition values:
+              d_date_sk 2416948
+            properties:
+              COLUMN_STATS_ACCURATE {"BASIC_STATS":"true"}
+              bucket_count -1
+              columns d_date
+              columns.comments 
+              columns.types date
+#### A masked pattern was here ####
+              name default.date_dim
+              numFiles 1
+              numRows 1
+              partition_columns d_date_sk
+              partition_columns.types bigint
+              rawDataSize 56
+              serialization.ddl struct date_dim { date d_date}
+              serialization.format 1
+              serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
+              totalSize 193
+#### A masked pattern was here ####
+            serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+          
+              input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+              output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+              properties:
+                bucket_count -1
+                columns d_date
+                columns.comments 
+                columns.types date
+#### A masked pattern was here ####
+                name default.date_dim
+                partition_columns d_date_sk
+                partition_columns.types bigint
+                serialization.ddl struct date_dim { date d_date}
+                serialization.format 1
+                serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
+#### A masked pattern was here ####
+              serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+              name: default.date_dim
+            name: default.date_dim
+      Processor Tree:
+        TableScan
+          alias: date_dim
+          Statistics: Num rows: 4 Data size: 224 Basic stats: COMPLETE Column stats: PARTIAL
+          GatherStats: false
+          Select Operator
+            expressions: d_date (type: date)
+            outputColumnNames: _col0
+            Statistics: Num rows: 4 Data size: 224 Basic stats: COMPLETE Column stats: PARTIAL
+            ListSink
+


[17/22] hive git commit: HIVE-14803: S3: Stats gathering for insert queries can be expensive for partitioned dataset (Rajesh Balamohan via Pengcheng Xiong)

Posted by se...@apache.org.
HIVE-14803: S3: Stats gathering for insert queries can be expensive for partitioned dataset (Rajesh Balamohan via Pengcheng Xiong)

Signed-off-by: Pengcheng Xiong <px...@apache.org>


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

Branch: refs/heads/hive-14535
Commit: c52c17b227671ad1cb44b1fe24134c7ccb4a06e1
Parents: 1876723
Author: Pengcheng Xiong <px...@apache.org>
Authored: Mon Oct 10 10:46:38 2016 -0700
Committer: Pengcheng Xiong <px...@apache.org>
Committed: Mon Oct 10 10:46:38 2016 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hive/ql/exec/StatsTask.java   | 134 ++++++++++++-------
 1 file changed, 82 insertions(+), 52 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/c52c17b2/ql/src/java/org/apache/hadoop/hive/ql/exec/StatsTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/StatsTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/StatsTask.java
index 9e528b5..51bafc4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/StatsTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/StatsTask.java
@@ -24,7 +24,13 @@ import java.util.ArrayList;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
 
+import com.google.common.collect.Lists;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.fs.FileStatus;
@@ -143,10 +149,10 @@ public class StatsTask extends Task<StatsWork> implements Serializable {
     StatsAggregator statsAggregator = null;
     int ret = 0;
     StatsCollectionContext scc = null;
-    EnvironmentContext environmentContext = null;
+    final EnvironmentContext environmentContext = new EnvironmentContext();;
     try {
       // Stats setup:
-      Warehouse wh = new Warehouse(conf);
+      final Warehouse wh = new Warehouse(conf);
       if (!getWork().getNoStatsAggregator() && !getWork().isNoScanAnalyzeCommand()) {
         try {
           scc = getContext();
@@ -160,9 +166,8 @@ public class StatsTask extends Task<StatsWork> implements Serializable {
       }
 
       List<Partition> partitions = getPartitionsList(db);
-      boolean atomic = HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_STATS_ATOMIC);
-
-      String tableFullName = table.getDbName() + "." + table.getTableName();
+      final boolean atomic = HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_STATS_ATOMIC);
+      final String tableFullName = table.getDbName() + "." + table.getTableName();
 
       if (partitions == null) {
         org.apache.hadoop.hive.metastore.api.Table tTable = table.getTTable();
@@ -200,7 +205,6 @@ public class StatsTask extends Task<StatsWork> implements Serializable {
           }
           // write table stats to metastore
           if (!getWork().getNoStatsAggregator()) {
-            environmentContext = new EnvironmentContext();
             environmentContext.putToProperties(StatsSetupConst.STATS_GENERATED,
                 StatsSetupConst.TASK);
           }
@@ -212,61 +216,87 @@ public class StatsTask extends Task<StatsWork> implements Serializable {
         }
         LOG.info("Table " + tableFullName + " stats: [" + toString(parameters) + ']');
       } else {
+        int poolSize = conf.getInt(ConfVars.HIVE_MOVE_FILES_THREAD_COUNT.varname, 1);
+        final ExecutorService pool = Executors.newFixedThreadPool(poolSize,
+            new ThreadFactoryBuilder()
+                .setDaemon(true)
+                .setNameFormat("stats-updater-thread-%d")
+                .build());
+        final List<Future<Void>> futures = Lists.newLinkedList();
+        final StatsAggregator statsAgg = statsAggregator;
+
         // Partitioned table:
         // Need to get the old stats of the partition
         // and update the table stats based on the old and new stats.
-        List<Partition> updates = new ArrayList<Partition>();
-        for (Partition partn : partitions) {
-          //
-          // get the old partition stats
-          //
-          org.apache.hadoop.hive.metastore.api.Partition tPart = partn.getTPartition();
-          Map<String, String> parameters = tPart.getParameters();
-          if (work.getTableSpecs() == null && AcidUtils.isAcidTable(table)) {
-            StatsSetupConst.setBasicStatsState(parameters, StatsSetupConst.FALSE);
-          } else if (work.getTableSpecs() != null
-              || (work.getLoadTableDesc() != null && work.getLoadTableDesc().getReplace())
-              || (work.getLoadFileDesc() != null && !work.getLoadFileDesc()
-                  .getDestinationCreateTable().isEmpty())) {
-            StatsSetupConst.setBasicStatsState(parameters, StatsSetupConst.TRUE);
-          }
-          if (!existStats(parameters) && atomic) {
-            continue;
-          }
-
-          // The collectable stats for the aggregator needs to be cleared.
-          // For eg. if a file is being loaded, the old number of rows are not valid
-          if (work.isClearAggregatorStats()) {
-            // we choose to keep the invalid stats and only change the setting.
-            StatsSetupConst.setBasicStatsState(parameters, StatsSetupConst.FALSE);
+        final List<Partition> updates = new ArrayList<Partition>();
+        try {
+          for (final Partition partn : partitions) {
+            futures.add(pool.submit(new Callable<Void>() {
+              @Override
+              public Void call() throws Exception {
+                //
+                // get the old partition stats
+                //
+                org.apache.hadoop.hive.metastore.api.Partition tPart = partn.getTPartition();
+                Map<String, String> parameters = tPart.getParameters();
+                if (work.getTableSpecs() == null && AcidUtils.isAcidTable(table)) {
+                  StatsSetupConst.setBasicStatsState(parameters, StatsSetupConst.FALSE);
+                } else if (work.getTableSpecs() != null
+                    || (work.getLoadTableDesc() != null && work.getLoadTableDesc().getReplace())
+                    || (work.getLoadFileDesc() != null && !work.getLoadFileDesc()
+                    .getDestinationCreateTable().isEmpty())) {
+                  StatsSetupConst.setBasicStatsState(parameters, StatsSetupConst.TRUE);
+                }
+                if (!existStats(parameters) && atomic) {
+                  return null;
+                }
+
+                // The collectable stats for the aggregator needs to be cleared.
+                // For eg. if a file is being loaded, the old number of rows are not valid
+                if (work.isClearAggregatorStats()) {
+                  // we choose to keep the invalid stats and only change the setting.
+                  StatsSetupConst.setBasicStatsState(parameters, StatsSetupConst.FALSE);
+                }
+
+                updateQuickStats(wh, parameters, tPart.getSd());
+                if (StatsSetupConst.areBasicStatsUptoDate(parameters)) {
+                  if (statsAgg != null) {
+                    String prefix = getAggregationPrefix(table, partn);
+                    updateStats(statsAgg, parameters, prefix, atomic);
+                  }
+                  if (!getWork().getNoStatsAggregator()) {
+                    environmentContext.putToProperties(StatsSetupConst.STATS_GENERATED,
+                        StatsSetupConst.TASK);
+                  }
+                }
+                updates.add(new Partition(table, tPart));
+
+                if (conf.getBoolVar(ConfVars.TEZ_EXEC_SUMMARY)) {
+                  console.printInfo("Partition " + tableFullName + partn.getSpec() +
+                      " stats: [" + StatsTask.toString(parameters) + ']');
+                }
+                LOG.info("Partition " + tableFullName + partn.getSpec() +
+                    " stats: [" + StatsTask.toString(parameters) + ']');
+                return null;
+              }
+            }));
           }
+          pool.shutdown();
 
-          updateQuickStats(wh, parameters, tPart.getSd());
-          if (StatsSetupConst.areBasicStatsUptoDate(parameters)) {
-            if (statsAggregator != null) {
-              String prefix = getAggregationPrefix(table, partn);
-              updateStats(statsAggregator, parameters, prefix, atomic);
-            }
-            if (!getWork().getNoStatsAggregator()) {
-              environmentContext = new EnvironmentContext();
-              environmentContext.putToProperties(StatsSetupConst.STATS_GENERATED,
-                  StatsSetupConst.TASK);
-            }
+          for (Future future : futures) {
+            future.get();
           }
-          updates.add(new Partition(table, tPart));
-
-          if (conf.getBoolVar(ConfVars.TEZ_EXEC_SUMMARY)) {
-            console.printInfo("Partition " + tableFullName + partn.getSpec() +
-            " stats: [" + toString(parameters) + ']');
+        } catch (InterruptedException e) {
+          LOG.debug("Cancelling " + futures.size() + " for partition stats update");
+          //cancel other futures
+          for (Future future : futures) {
+            future.cancel(true);
           }
-          LOG.info("Partition " + tableFullName + partn.getSpec() +
-              " stats: [" + toString(parameters) + ']');
         }
         if (!updates.isEmpty()) {
-          db.alterPartitions(tableFullName, updates, environmentContext);
+            db.alterPartitions(tableFullName, updates, environmentContext);
+          }
         }
-      }
-
     } catch (Exception e) {
       console.printInfo("[Warning] could not update stats.",
           "Failed with exception " + e.getMessage() + "\n"
@@ -374,7 +404,7 @@ public class StatsTask extends Task<StatsWork> implements Serializable {
     MetaStoreUtils.populateQuickStats(partfileStatus, parameters);
   }
 
-  private String toString(Map<String, String> parameters) {
+  static String toString(Map<String, String> parameters) {
     StringBuilder builder = new StringBuilder();
     for (String statType : StatsSetupConst.supportedStats) {
       String value = parameters.get(statType);


[07/22] hive git commit: HIVE-14545 : HiveServer2 with http transport mode spends too much time just creating configs (Rajesh Balamohan via Thejas Nair)

Posted by se...@apache.org.
HIVE-14545 : HiveServer2 with http transport mode spends too much time just creating configs (Rajesh Balamohan via Thejas Nair)


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

Branch: refs/heads/hive-14535
Commit: a0bf9d6296760af7ce8ed29d5ccf13683a46d809
Parents: c53c9be
Author: Thejas Nair <th...@hortonworks.com>
Authored: Wed Oct 5 18:34:09 2016 -0700
Committer: Thejas Nair <th...@hortonworks.com>
Committed: Wed Oct 5 18:34:09 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/a0bf9d62/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java b/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java
index b7a1e2d..50449e0 100644
--- a/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java
+++ b/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java
@@ -358,7 +358,7 @@ public class ThriftHttpServlet extends TServlet {
       try {
         AuthMethods authMethod = AuthMethods.getValidAuthMethod(authType);
         PasswdAuthenticationProvider provider =
-            AuthenticationProviderFactory.getAuthenticationProvider(authMethod);
+            AuthenticationProviderFactory.getAuthenticationProvider(authMethod, hiveConf);
         provider.Authenticate(userName, getPassword(request, authType));
 
       } catch (Exception e) {


[22/22] hive git commit: HIVE-14639 : handle SKEWED BY for MM tables (Sergey Shelukhin)

Posted by se...@apache.org.
HIVE-14639 : handle SKEWED BY for MM tables (Sergey Shelukhin)


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

Branch: refs/heads/hive-14535
Commit: b9e815722686f5bcdae40caadae34f84509d86c2
Parents: 7d50a15
Author: Sergey Shelukhin <se...@apache.org>
Authored: Tue Oct 11 16:09:28 2016 -0700
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Tue Oct 11 16:09:28 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hive/ql/exec/FileSinkOperator.java   |  31 +-
 .../apache/hadoop/hive/ql/exec/Utilities.java   |   2 +-
 .../hadoop/hive/ql/io/HiveInputFormat.java      |   2 +
 .../apache/hadoop/hive/ql/metadata/Hive.java    |   6 +-
 ql/src/test/queries/clientpositive/mm_all.q     |  76 ++-
 ql/src/test/queries/clientpositive/mm_current.q |  45 +-
 .../results/clientpositive/llap/mm_all.q.out    | 473 +++++++++++--------
 .../clientpositive/llap/mm_current.q.out        | 215 +++++----
 8 files changed, 473 insertions(+), 377 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/b9e81572/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
index 00115fe..5902036 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
@@ -1189,6 +1189,7 @@ public class FileSinkOperator extends TerminalOperator<FileSinkDesc> implements
         Path specPath = conf.getDirName();
         String unionSuffix = null;
         DynamicPartitionCtx dpCtx = conf.getDynPartCtx();
+        ListBucketingCtx lbCtx = conf.getLbCtx();
         if (conf.isLinkedFileSink() && (dpCtx != null)) {
           specPath = conf.getParentDir();
           Utilities.LOG14535.info("Setting specPath to " + specPath + " for dynparts");
@@ -1197,7 +1198,7 @@ public class FileSinkOperator extends TerminalOperator<FileSinkDesc> implements
         if (!conf.isMmTable()) {
           Utilities.mvFileToFinalPath(specPath, hconf, success, LOG, dpCtx, conf, reporter); // TODO# other callers
         } else {
-          handleMmTable(specPath, unionSuffix, hconf, success, dpCtx, conf, reporter);
+          handleMmTable(specPath, unionSuffix, hconf, success, dpCtx, lbCtx, conf, reporter);
         }
       }
     } catch (IOException e) {
@@ -1207,22 +1208,26 @@ public class FileSinkOperator extends TerminalOperator<FileSinkDesc> implements
   }
 
   private static FileStatus[] getMmDirectoryCandidates(FileSystem fs, Path path,
-      int dpLevels, String unionSuffix, PathFilter filter) throws IOException {
+      DynamicPartitionCtx dpCtx, ListBucketingCtx lbCtx, String unionSuffix, PathFilter filter)
+          throws IOException {
     StringBuilder sb = new StringBuilder(path.toUri().getPath());
-    for (int i = 0; i < dpLevels; i++) {
+    int dpLevels = dpCtx == null ? 0 : dpCtx.getNumDPCols(),
+        lbLevels = lbCtx == null ? 0 : lbCtx.getSkewedColNames().size();
+    for (int i = 0; i < dpLevels + lbLevels; i++) {
       sb.append(Path.SEPARATOR).append("*");
     }
     if (unionSuffix != null) {
       sb.append(Path.SEPARATOR).append(unionSuffix);
     }
     sb.append(Path.SEPARATOR).append("*"); // TODO: we could add exact mm prefix here
+    Utilities.LOG14535.info("Looking for files via: " + sb.toString());
     Path pathPattern = new Path(path, sb.toString());
     return fs.globStatus(pathPattern, filter);
   }
 
   private void handleMmTable(Path specPath, String unionSuffix, Configuration hconf,
-      boolean success, DynamicPartitionCtx dpCtx, FileSinkDesc conf, Reporter reporter)
-          throws IOException, HiveException {
+      boolean success, DynamicPartitionCtx dpCtx, ListBucketingCtx lbCtx, FileSinkDesc conf,
+      Reporter reporter) throws IOException, HiveException {
     FileSystem fs = specPath.getFileSystem(hconf);
     // Manifests would be at the root level, but the results at target level.
     // TODO# special case - doesn't take bucketing into account
@@ -1230,7 +1235,7 @@ public class FileSinkOperator extends TerminalOperator<FileSinkDesc> implements
 
     ValidWriteIds.IdPathFilter filter = new ValidWriteIds.IdPathFilter(conf.getMmWriteId(), true);
     if (!success) {
-      tryDeleteAllMmFiles(fs, specPath, manifestDir, dpCtx, unionSuffix, filter);
+      tryDeleteAllMmFiles(fs, specPath, manifestDir, dpCtx, lbCtx, unionSuffix, filter);
       return;
     }
     FileStatus[] files = HiveStatsUtils.getFileStatusRecurse(manifestDir, 1, fs, filter);
@@ -1240,14 +1245,14 @@ public class FileSinkOperator extends TerminalOperator<FileSinkDesc> implements
       for (FileStatus status : files) {
         Path path = status.getPath();
         if (path.getName().endsWith(MANIFEST_EXTENSION)) {
+          Utilities.LOG14535.info("Reading manifest " + path);
           manifests.add(path);
         }
       }
     }
 
     Utilities.LOG14535.info("Looking for files in: " + specPath);
-    files = getMmDirectoryCandidates(fs, specPath,
-        dpCtx == null ? 0 : dpCtx.getNumDPCols(), unionSuffix, filter);
+    files = getMmDirectoryCandidates(fs, specPath, dpCtx, lbCtx, unionSuffix, filter);
     ArrayList<FileStatus> results = new ArrayList<>();
     if (files != null) {
       for (FileStatus status : files) {
@@ -1307,8 +1312,11 @@ public class FileSinkOperator extends TerminalOperator<FileSinkDesc> implements
     }
 
     if (results.isEmpty()) return;
-    FileStatus[] finalResults = results.toArray(new FileStatus[results.size()]);
 
+    // TODO: see HIVE-14886 - removeTempOrDuplicateFiles is broken for list bucketing,
+    //       so maintain parity here by not calling it at all.
+    if (lbCtx != null) return;
+    FileStatus[] finalResults = results.toArray(new FileStatus[results.size()]);
     List<Path> emptyBuckets = Utilities.removeTempOrDuplicateFiles(
         fs, finalResults, dpCtx, conf, hconf);
     // create empty buckets if necessary
@@ -1318,10 +1326,9 @@ public class FileSinkOperator extends TerminalOperator<FileSinkDesc> implements
   }
 
   private void tryDeleteAllMmFiles(FileSystem fs, Path specPath, Path manifestDir,
-      DynamicPartitionCtx dpCtx, String unionSuffix,
+      DynamicPartitionCtx dpCtx, ListBucketingCtx lbCtx, String unionSuffix,
       ValidWriteIds.IdPathFilter filter) throws IOException {
-    FileStatus[] files = getMmDirectoryCandidates(fs, specPath,
-        dpCtx == null ? 0 : dpCtx.getNumDPCols(), unionSuffix, filter);
+    FileStatus[] files = getMmDirectoryCandidates(fs, specPath, dpCtx, lbCtx, unionSuffix, filter);
     if (files != null) {
       for (FileStatus status : files) {
         Utilities.LOG14535.info("Deleting " + status.getPath() + " on failure");

http://git-wip-us.apache.org/repos/asf/hive/blob/b9e81572/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
index 03abdc1..a2eff8b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
@@ -1599,7 +1599,7 @@ public final class Utilities {
         }
       } else {
         String taskId = getPrefixedTaskIdFromFilename(one.getPath().getName());
-        Utilities.LOG14535.info("removeTempOrDuplicateFiles pondering " + one.getPath() + ", taskId " + taskId/*, new Exception()*/);
+        Utilities.LOG14535.info("removeTempOrDuplicateFiles pondering " + one.getPath() + ", taskId " + taskId);
 
         FileStatus otherFile = taskIdToFile.get(taskId);
         if (otherFile == null) {

http://git-wip-us.apache.org/repos/asf/hive/blob/b9e81572/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java b/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java
index c3e2681..a539799 100755
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java
@@ -399,6 +399,7 @@ public class HiveInputFormat<K extends WritableComparable, V extends Writable>
   private void processForWriteIds(Path dir, JobConf conf,
       ValidWriteIds writeIds, List<Path> finalPaths) throws IOException {
     FileSystem fs = dir.getFileSystem(conf);
+    Utilities.LOG14535.warn("Checking " + dir + " (root) for inputs");
     FileStatus[] files = fs.listStatus(dir); // TODO: batch?
     LinkedList<Path> subdirs = new LinkedList<>();
     for (FileStatus file : files) {
@@ -415,6 +416,7 @@ public class HiveInputFormat<K extends WritableComparable, V extends Writable>
   private void handleNonMmDirChild(FileStatus file, ValidWriteIds writeIds,
       LinkedList<Path> subdirs, List<Path> finalPaths) {
     Path path = file.getPath();
+    Utilities.LOG14535.warn("Checking " + path + " for inputs");
     if (!file.isDirectory()) {
       Utilities.LOG14535.warn("Ignoring a file not in MM directory " + path);
       return;

http://git-wip-us.apache.org/repos/asf/hive/blob/b9e81572/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 73a3b19..84c1e7b 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
@@ -1587,12 +1587,12 @@ public class Hive {
       List<Path> newFiles = null;
       if (mmWriteId != null) {
         Utilities.LOG14535.info("not moving " + loadPath + " to " + newPartPath);
-        assert !isAcid && !replace;
+        assert !isAcid;
         if (areEventsForDmlNeeded(tbl, oldPart)) {
           newFiles = listFilesCreatedByQuery(loadPath, mmWriteId);
         }
-        if (replace) {
-          Path tableDest = tbl.getPath();
+        Utilities.LOG14535.info("maybe deleting stuff from " + oldPartPath + " (new " + newPartPath + ") for replace");
+        if (replace && oldPartPath != null) { // TODO# is this correct? ignore until iow jira
           deleteOldPathForReplace(newPartPath, oldPartPath,
               getConf(), new ValidWriteIds.IdPathFilter(mmWriteId, false));
         }

http://git-wip-us.apache.org/repos/asf/hive/blob/b9e81572/ql/src/test/queries/clientpositive/mm_all.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/mm_all.q b/ql/src/test/queries/clientpositive/mm_all.q
index cc44c19..1f85c48 100644
--- a/ql/src/test/queries/clientpositive/mm_all.q
+++ b/ql/src/test/queries/clientpositive/mm_all.q
@@ -9,8 +9,8 @@ set hive.exec.dynamic.partition.mode=nonstrict;
 -- Force multiple writers when reading
 drop table intermediate;
 create table intermediate(key int) partitioned by (p int) stored as orc;
-insert into table intermediate partition(p='455') select key from src limit 2;
-insert into table intermediate partition(p='456') select key from src limit 2;
+insert into table intermediate partition(p='455') select distinct key from src where key >= 0 order by key desc limit 2;
+insert into table intermediate partition(p='456') select distinct key from src where key is not null order by key asc limit 2;
 
 drop table part_mm;
 create table part_mm(key int) partitioned by (key_mm int) stored as orc tblproperties ('hivecommit'='true');
@@ -18,21 +18,20 @@ explain insert into table part_mm partition(key_mm='455') select key from interm
 insert into table part_mm partition(key_mm='455') select key from intermediate;
 insert into table part_mm partition(key_mm='456') select key from intermediate;
 insert into table part_mm partition(key_mm='455') select key from intermediate;
-select * from part_mm;
+select * from part_mm order by key;
 drop table part_mm;
 
 drop table simple_mm;
 create table simple_mm(key int) stored as orc tblproperties ('hivecommit'='true');
 insert into table simple_mm select key from intermediate;
 insert overwrite table simple_mm select key from intermediate;
-select * from simple_mm;
+select * from simple_mm order by key;
 insert into table simple_mm select key from intermediate;
-select * from simple_mm;
+select * from simple_mm order by key;
 drop table simple_mm;
 
 
--- simple DP (no bucketing, no sorting?)
-drop table dp_no_mm;
+-- simple DP (no bucketing)
 drop table dp_mm;
 
 set hive.exec.dynamic.partition.mode=nonstrict;
@@ -41,18 +40,13 @@ set hive.merge.mapredfiles=false;
 set hive.merge.sparkfiles=false;
 set hive.merge.tezfiles=false;
 
-create table dp_no_mm (key int) partitioned by (key1 string, key2 int) stored as orc;
 create table dp_mm (key int) partitioned by (key1 string, key2 int) stored as orc
   tblproperties ('hivecommit'='true');
 
-insert into table dp_no_mm partition (key1='123', key2) select key, key from intermediate;
-
 insert into table dp_mm partition (key1='123', key2) select key, key from intermediate;
 
-select * from dp_no_mm;
-select * from dp_mm;
+select * from dp_mm order by key;
 
-drop table dp_no_mm;
 drop table dp_mm;
 
 
@@ -108,10 +102,34 @@ select key as p, key from intermediate
 union all 
 select key + 1 as p, key + 1 from intermediate ) temps;
 
-select * from partunion_mm;
+select * from partunion_mm order by id;
 drop table partunion_mm;
 
--- TODO# from here, fix it
+
+
+create table skew_mm(k1 int, k2 int, k4 int) skewed by (k1, k4) on ((0,0),(1,1),(2,2),(3,3))
+ stored as directories tblproperties ('hivecommit'='true');
+
+insert into table skew_mm 
+select key, key, key from intermediate;
+
+select * from skew_mm order by k2;
+drop table skew_mm;
+
+
+create table skew_dp_union_mm(k1 int, k2 int, k4 int) partitioned by (k3 int) 
+skewed by (k1, k4) on ((0,0),(1,1),(2,2),(3,3)) stored as directories tblproperties ('hivecommit'='true');
+
+insert into table skew_dp_union_mm partition (k3)
+select key as i, key as j, key as k, key as l from intermediate
+union all 
+select key +1 as i, key +2 as j, key +3 as k, key +4 as l from intermediate;
+
+
+select * from skew_dp_union_mm order by k2;
+drop table skew_dp_union_mm;
+
+
 
 
 
@@ -122,24 +140,12 @@ drop table partunion_mm;
 
 
 
---drop table partunion_mm;
 --drop table merge_mm;
 --drop table ctas_mm;
---drop table T1;
---drop table T2;
---drop table skew_mm;
 --
 --
 --create table ctas_mm tblproperties ('hivecommit'='true') as select * from src limit 3;
 --
---create table partunion_mm(id_mm int) partitioned by (key_mm int)  tblproperties ('hivecommit'='true');
---
---
---insert into table partunion_mm partition(key_mm)
---select temps.* from (
---select key as key_mm, key from ctas_mm 
---union all 
---select key as key_mm, key from simple_mm ) temps;
 --
 --set hive.merge.mapredfiles=true;
 --set hive.merge.sparkfiles=true;
@@ -158,22 +164,6 @@ drop table partunion_mm;
 --        FROM src;
 --
 --
---set hive.optimize.skewjoin.compiletime = true;
----- the test case is wrong?
---
---CREATE TABLE T1(key STRING, val STRING)
---SKEWED BY (key) ON ((2)) STORED AS TEXTFILE;
---LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
---CREATE TABLE T2(key STRING, val STRING)
---SKEWED BY (key) ON ((3)) STORED AS TEXTFILE;
---LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2;
---
---EXPLAIN
---SELECT a.*, b.* FROM T1 a JOIN T2 b ON a.key = b.key;
---
---create table skew_mm(k1 string, k2 string, k3 string, k4 string) SKEWED BY (key) ON ((2)) tblproperties ('hivecommit'='true');
---INSERT OVERWRITE TABLE skew_mm
---SELECT a.key as k1, a.val as k2, b.key as k3, b.val as k4 FROM T1 a JOIN T2 b ON a.key = b.key;
 --
 ---- TODO load, multi-insert etc
 --

http://git-wip-us.apache.org/repos/asf/hive/blob/b9e81572/ql/src/test/queries/clientpositive/mm_current.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/mm_current.q b/ql/src/test/queries/clientpositive/mm_current.q
index e1fb3d9..ceb7a1a 100644
--- a/ql/src/test/queries/clientpositive/mm_current.q
+++ b/ql/src/test/queries/clientpositive/mm_current.q
@@ -12,26 +12,41 @@ insert into table intermediate partition(p='455') select key from src limit 2;
 insert into table intermediate partition(p='456') select key from src limit 2;
 
 
-create table partunion_no_mm(id int) partitioned by (key int); 
-insert into table partunion_no_mm partition(key)
-select temps.* from (
-select key as p, key from intermediate 
-union all 
-select key + 1 as p, key + 1 from intermediate ) temps;
+set hive.optimize.skewjoin.compiletime = true;
+
+create table skew_mm(k1 int, k2 int, k4 int) skewed by (k1, k4) on ((0,0),(1,1),(2,2),(3,3))
+ stored as directories tblproperties ('hivecommit'='false');
+
+insert into table skew_mm 
+select key, key, key from intermediate;
+
+drop table skew_mm;
+
+
+create table skew_mm(k1 int, k2 int, k4 int) skewed by (k1, k4) on ((0,0),(1,1),(2,2),(3,3))
+ stored as directories tblproperties ('hivecommit'='true');
 
-select * from partunion_no_mm;
-drop table partunion_no_mm;
+insert into table skew_mm 
+select key, key, key from intermediate;
 
+select * from skew_mm;
+drop table skew_mm;
 
-create table partunion_mm(id int) partitioned by (key int) tblproperties ('hivecommit'='true'); 
-insert into table partunion_mm partition(key)
-select temps.* from (
-select key as p, key from intermediate 
+
+
+
+
+create table skew_mm(k1 int, k2 int, k4 int) partitioned by (k3 int) 
+skewed by (k1, k4) on ((0,0),(1,1),(2,2),(3,3)) stored as directories tblproperties ('hivecommit'='true');
+
+insert into table skew_mm partition (k3)
+select key as i, key as j, key as k, key as l from intermediate
 union all 
-select key + 1 as p, key + 1 from intermediate ) temps;
+select key +1 as i, key +2 as j, key +3 as k, key +4 as l from intermediate;
+
 
-select * from partunion_mm;
-drop table partunion_mm;
+select * from skew_mm;
+drop table skew_mm;
 
 
 

http://git-wip-us.apache.org/repos/asf/hive/blob/b9e81572/ql/src/test/results/clientpositive/llap/mm_all.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/mm_all.q.out b/ql/src/test/results/clientpositive/llap/mm_all.q.out
index 0a8bb40..b70ae3c 100644
--- a/ql/src/test/results/clientpositive/llap/mm_all.q.out
+++ b/ql/src/test/results/clientpositive/llap/mm_all.q.out
@@ -12,20 +12,20 @@ POSTHOOK: query: create table intermediate(key int) partitioned by (p int) store
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@intermediate
-PREHOOK: query: insert into table intermediate partition(p='455') select key from src limit 2
+PREHOOK: query: insert into table intermediate partition(p='455') select distinct key from src where key >= 0 order by key desc limit 2
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
 PREHOOK: Output: default@intermediate@p=455
-POSTHOOK: query: insert into table intermediate partition(p='455') select key from src limit 2
+POSTHOOK: query: insert into table intermediate partition(p='455') select distinct key from src where key >= 0 order by key desc limit 2
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@src
 POSTHOOK: Output: default@intermediate@p=455
 POSTHOOK: Lineage: intermediate PARTITION(p=455).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ]
-PREHOOK: query: insert into table intermediate partition(p='456') select key from src limit 2
+PREHOOK: query: insert into table intermediate partition(p='456') select distinct key from src where key is not null order by key asc limit 2
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
 PREHOOK: Output: default@intermediate@p=456
-POSTHOOK: query: insert into table intermediate partition(p='456') select key from src limit 2
+POSTHOOK: query: insert into table intermediate partition(p='456') select distinct key from src where key is not null order by key asc limit 2
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@src
 POSTHOOK: Output: default@intermediate@p=456
@@ -135,30 +135,30 @@ POSTHOOK: Input: default@intermediate@p=455
 POSTHOOK: Input: default@intermediate@p=456
 POSTHOOK: Output: default@part_mm@key_mm=455
 POSTHOOK: Lineage: part_mm PARTITION(key_mm=455).key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
-PREHOOK: query: select * from part_mm
+PREHOOK: query: select * from part_mm order by key
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part_mm
 PREHOOK: Input: default@part_mm@key_mm=455
 PREHOOK: Input: default@part_mm@key_mm=456
 #### A masked pattern was here ####
-POSTHOOK: query: select * from part_mm
+POSTHOOK: query: select * from part_mm order by key
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@part_mm
 POSTHOOK: Input: default@part_mm@key_mm=455
 POSTHOOK: Input: default@part_mm@key_mm=456
 #### A masked pattern was here ####
 0	455
-455	455
 0	455
-455	455
-0	455
-455	455
-0	455
-455	455
 0	456
-455	456
-0	456
-455	456
+10	456
+10	455
+10	455
+97	456
+97	455
+97	455
+98	455
+98	455
+98	456
 PREHOOK: query: drop table part_mm
 PREHOOK: type: DROPTABLE
 PREHOOK: Input: default@part_mm
@@ -205,18 +205,18 @@ POSTHOOK: Input: default@intermediate@p=455
 POSTHOOK: Input: default@intermediate@p=456
 POSTHOOK: Output: default@simple_mm
 POSTHOOK: Lineage: simple_mm.key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
-PREHOOK: query: select * from simple_mm
+PREHOOK: query: select * from simple_mm order by key
 PREHOOK: type: QUERY
 PREHOOK: Input: default@simple_mm
 #### A masked pattern was here ####
-POSTHOOK: query: select * from simple_mm
+POSTHOOK: query: select * from simple_mm order by key
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@simple_mm
 #### A masked pattern was here ####
 0
-455
-0
-455
+10
+97
+98
 PREHOOK: query: insert into table simple_mm select key from intermediate
 PREHOOK: type: QUERY
 PREHOOK: Input: default@intermediate
@@ -230,22 +230,22 @@ POSTHOOK: Input: default@intermediate@p=455
 POSTHOOK: Input: default@intermediate@p=456
 POSTHOOK: Output: default@simple_mm
 POSTHOOK: Lineage: simple_mm.key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
-PREHOOK: query: select * from simple_mm
+PREHOOK: query: select * from simple_mm order by key
 PREHOOK: type: QUERY
 PREHOOK: Input: default@simple_mm
 #### A masked pattern was here ####
-POSTHOOK: query: select * from simple_mm
+POSTHOOK: query: select * from simple_mm order by key
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@simple_mm
 #### A masked pattern was here ####
 0
-455
 0
-455
-0
-455
-0
-455
+10
+10
+97
+97
+98
+98
 PREHOOK: query: drop table simple_mm
 PREHOOK: type: DROPTABLE
 PREHOOK: Input: default@simple_mm
@@ -254,24 +254,12 @@ POSTHOOK: query: drop table simple_mm
 POSTHOOK: type: DROPTABLE
 POSTHOOK: Input: default@simple_mm
 POSTHOOK: Output: default@simple_mm
-PREHOOK: query: -- simple DP (no bucketing, no sorting?)
-drop table dp_no_mm
+PREHOOK: query: -- simple DP (no bucketing)
+drop table dp_mm
 PREHOOK: type: DROPTABLE
-POSTHOOK: query: -- simple DP (no bucketing, no sorting?)
-drop table dp_no_mm
+POSTHOOK: query: -- simple DP (no bucketing)
+drop table dp_mm
 POSTHOOK: type: DROPTABLE
-PREHOOK: query: drop table dp_mm
-PREHOOK: type: DROPTABLE
-POSTHOOK: query: drop table dp_mm
-POSTHOOK: type: DROPTABLE
-PREHOOK: query: create table dp_no_mm (key int) partitioned by (key1 string, key2 int) stored as orc
-PREHOOK: type: CREATETABLE
-PREHOOK: Output: database:default
-PREHOOK: Output: default@dp_no_mm
-POSTHOOK: query: create table dp_no_mm (key int) partitioned by (key1 string, key2 int) stored as orc
-POSTHOOK: type: CREATETABLE
-POSTHOOK: Output: database:default
-POSTHOOK: Output: default@dp_no_mm
 PREHOOK: query: create table dp_mm (key int) partitioned by (key1 string, key2 int) stored as orc
   tblproperties ('hivecommit'='true')
 PREHOOK: type: CREATETABLE
@@ -282,21 +270,6 @@ POSTHOOK: query: create table dp_mm (key int) partitioned by (key1 string, key2
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@dp_mm
-PREHOOK: query: insert into table dp_no_mm partition (key1='123', key2) select key, key from intermediate
-PREHOOK: type: QUERY
-PREHOOK: Input: default@intermediate
-PREHOOK: Input: default@intermediate@p=455
-PREHOOK: Input: default@intermediate@p=456
-PREHOOK: Output: default@dp_no_mm@key1=123
-POSTHOOK: query: insert into table dp_no_mm partition (key1='123', key2) select key, key from intermediate
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@intermediate
-POSTHOOK: Input: default@intermediate@p=455
-POSTHOOK: Input: default@intermediate@p=456
-POSTHOOK: Output: default@dp_no_mm@key1=123/key2=0
-POSTHOOK: Output: default@dp_no_mm@key1=123/key2=455
-POSTHOOK: Lineage: dp_no_mm PARTITION(key1=123,key2=0).key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
-POSTHOOK: Lineage: dp_no_mm PARTITION(key1=123,key2=455).key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 PREHOOK: query: insert into table dp_mm partition (key1='123', key2) select key, key from intermediate
 PREHOOK: type: QUERY
 PREHOOK: Input: default@intermediate
@@ -309,49 +282,33 @@ POSTHOOK: Input: default@intermediate
 POSTHOOK: Input: default@intermediate@p=455
 POSTHOOK: Input: default@intermediate@p=456
 POSTHOOK: Output: default@dp_mm@key1=123/key2=0
-POSTHOOK: Output: default@dp_mm@key1=123/key2=455
+POSTHOOK: Output: default@dp_mm@key1=123/key2=10
+POSTHOOK: Output: default@dp_mm@key1=123/key2=97
+POSTHOOK: Output: default@dp_mm@key1=123/key2=98
 POSTHOOK: Lineage: dp_mm PARTITION(key1=123,key2=0).key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
-POSTHOOK: Lineage: dp_mm PARTITION(key1=123,key2=455).key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
-PREHOOK: query: select * from dp_no_mm
-PREHOOK: type: QUERY
-PREHOOK: Input: default@dp_no_mm
-PREHOOK: Input: default@dp_no_mm@key1=123/key2=0
-PREHOOK: Input: default@dp_no_mm@key1=123/key2=455
-#### A masked pattern was here ####
-POSTHOOK: query: select * from dp_no_mm
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@dp_no_mm
-POSTHOOK: Input: default@dp_no_mm@key1=123/key2=0
-POSTHOOK: Input: default@dp_no_mm@key1=123/key2=455
-#### A masked pattern was here ####
-455	123	455
-455	123	455
-0	123	0
-0	123	0
-PREHOOK: query: select * from dp_mm
+POSTHOOK: Lineage: dp_mm PARTITION(key1=123,key2=10).key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: dp_mm PARTITION(key1=123,key2=97).key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: dp_mm PARTITION(key1=123,key2=98).key SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+PREHOOK: query: select * from dp_mm order by key
 PREHOOK: type: QUERY
 PREHOOK: Input: default@dp_mm
 PREHOOK: Input: default@dp_mm@key1=123/key2=0
-PREHOOK: Input: default@dp_mm@key1=123/key2=455
+PREHOOK: Input: default@dp_mm@key1=123/key2=10
+PREHOOK: Input: default@dp_mm@key1=123/key2=97
+PREHOOK: Input: default@dp_mm@key1=123/key2=98
 #### A masked pattern was here ####
-POSTHOOK: query: select * from dp_mm
+POSTHOOK: query: select * from dp_mm order by key
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@dp_mm
 POSTHOOK: Input: default@dp_mm@key1=123/key2=0
-POSTHOOK: Input: default@dp_mm@key1=123/key2=455
+POSTHOOK: Input: default@dp_mm@key1=123/key2=10
+POSTHOOK: Input: default@dp_mm@key1=123/key2=97
+POSTHOOK: Input: default@dp_mm@key1=123/key2=98
 #### A masked pattern was here ####
-455	123	455
-455	123	455
-0	123	0
 0	123	0
-PREHOOK: query: drop table dp_no_mm
-PREHOOK: type: DROPTABLE
-PREHOOK: Input: default@dp_no_mm
-PREHOOK: Output: default@dp_no_mm
-POSTHOOK: query: drop table dp_no_mm
-POSTHOOK: type: DROPTABLE
-POSTHOOK: Input: default@dp_no_mm
-POSTHOOK: Output: default@dp_no_mm
+10	123	10
+97	123	97
+98	123	98
 PREHOOK: query: drop table dp_mm
 PREHOOK: type: DROPTABLE
 PREHOOK: Input: default@dp_mm
@@ -402,13 +359,13 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@union_mm
 #### A masked pattern was here ####
 0
-0
 1
-1
-455
-455
-456
-456
+10
+11
+97
+98
+98
+99
 PREHOOK: query: insert into table union_mm 
 select p from
 (
@@ -448,20 +405,23 @@ POSTHOOK: Input: default@union_mm
 #### A masked pattern was here ####
 0
 0
-0
-1
 1
 1
 2
-2
-455
-455
-455
-456
-456
-456
-457
-457
+10
+10
+11
+11
+12
+97
+97
+98
+98
+98
+99
+99
+99
+100
 PREHOOK: query: insert into table union_mm
 SELECT p FROM
 (
@@ -516,25 +476,32 @@ POSTHOOK: Input: default@union_mm
 0
 0
 0
-0
-1
 1
 1
 1
 2
 2
-2
-455
-455
-455
-455
-456
-456
-456
-456
-457
-457
-457
+10
+10
+10
+11
+11
+11
+12
+12
+97
+97
+97
+98
+98
+98
+98
+99
+99
+99
+99
+100
+100
 PREHOOK: query: drop table union_mm
 PREHOOK: type: DROPTABLE
 PREHOOK: Input: default@union_mm
@@ -572,36 +539,48 @@ POSTHOOK: Input: default@intermediate@p=455
 POSTHOOK: Input: default@intermediate@p=456
 POSTHOOK: Output: default@partunion_mm@key=0
 POSTHOOK: Output: default@partunion_mm@key=1
-POSTHOOK: Output: default@partunion_mm@key=455
-POSTHOOK: Output: default@partunion_mm@key=456
+POSTHOOK: Output: default@partunion_mm@key=10
+POSTHOOK: Output: default@partunion_mm@key=11
+POSTHOOK: Output: default@partunion_mm@key=97
+POSTHOOK: Output: default@partunion_mm@key=98
+POSTHOOK: Output: default@partunion_mm@key=99
 POSTHOOK: Lineage: partunion_mm PARTITION(key=0).id EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: partunion_mm PARTITION(key=10).id EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: partunion_mm PARTITION(key=11).id EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
 POSTHOOK: Lineage: partunion_mm PARTITION(key=1).id EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
-POSTHOOK: Lineage: partunion_mm PARTITION(key=455).id EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
-POSTHOOK: Lineage: partunion_mm PARTITION(key=456).id EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
-PREHOOK: query: select * from partunion_mm
+POSTHOOK: Lineage: partunion_mm PARTITION(key=97).id EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: partunion_mm PARTITION(key=98).id EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: partunion_mm PARTITION(key=99).id EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+PREHOOK: query: select * from partunion_mm order by id
 PREHOOK: type: QUERY
 PREHOOK: Input: default@partunion_mm
 PREHOOK: Input: default@partunion_mm@key=0
 PREHOOK: Input: default@partunion_mm@key=1
-PREHOOK: Input: default@partunion_mm@key=455
-PREHOOK: Input: default@partunion_mm@key=456
+PREHOOK: Input: default@partunion_mm@key=10
+PREHOOK: Input: default@partunion_mm@key=11
+PREHOOK: Input: default@partunion_mm@key=97
+PREHOOK: Input: default@partunion_mm@key=98
+PREHOOK: Input: default@partunion_mm@key=99
 #### A masked pattern was here ####
-POSTHOOK: query: select * from partunion_mm
+POSTHOOK: query: select * from partunion_mm order by id
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@partunion_mm
 POSTHOOK: Input: default@partunion_mm@key=0
 POSTHOOK: Input: default@partunion_mm@key=1
-POSTHOOK: Input: default@partunion_mm@key=455
-POSTHOOK: Input: default@partunion_mm@key=456
+POSTHOOK: Input: default@partunion_mm@key=10
+POSTHOOK: Input: default@partunion_mm@key=11
+POSTHOOK: Input: default@partunion_mm@key=97
+POSTHOOK: Input: default@partunion_mm@key=98
+POSTHOOK: Input: default@partunion_mm@key=99
 #### A masked pattern was here ####
 0	0
-0	0
-1	1
 1	1
-455	455
-455	455
-456	456
-456	456
+10	10
+11	11
+97	97
+98	98
+98	98
+99	99
 PREHOOK: query: drop table partunion_mm
 PREHOOK: type: DROPTABLE
 PREHOOK: Input: default@partunion_mm
@@ -610,35 +589,164 @@ POSTHOOK: query: drop table partunion_mm
 POSTHOOK: type: DROPTABLE
 POSTHOOK: Input: default@partunion_mm
 POSTHOOK: Output: default@partunion_mm
-PREHOOK: query: -- TODO# from here, fix it
-
-
-
-
--- future
+PREHOOK: query: create table skew_mm(k1 int, k2 int, k4 int) skewed by (k1, k4) on ((0,0),(1,1),(2,2),(3,3))
+ stored as directories tblproperties ('hivecommit'='true')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@skew_mm
+POSTHOOK: query: create table skew_mm(k1 int, k2 int, k4 int) skewed by (k1, k4) on ((0,0),(1,1),(2,2),(3,3))
+ stored as directories tblproperties ('hivecommit'='true')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@skew_mm
+PREHOOK: query: insert into table skew_mm 
+select key, key, key from intermediate
+PREHOOK: type: QUERY
+PREHOOK: Input: default@intermediate
+PREHOOK: Input: default@intermediate@p=455
+PREHOOK: Input: default@intermediate@p=456
+PREHOOK: Output: default@skew_mm
+POSTHOOK: query: insert into table skew_mm 
+select key, key, key from intermediate
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@intermediate
+POSTHOOK: Input: default@intermediate@p=455
+POSTHOOK: Input: default@intermediate@p=456
+POSTHOOK: Output: default@skew_mm
+POSTHOOK: Lineage: skew_mm.k1 SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: skew_mm.k2 SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: skew_mm.k4 SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+PREHOOK: query: select * from skew_mm order by k2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@skew_mm
+#### A masked pattern was here ####
+POSTHOOK: query: select * from skew_mm order by k2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@skew_mm
+#### A masked pattern was here ####
+0	0	0
+10	10	10
+97	97	97
+98	98	98
+PREHOOK: query: drop table skew_mm
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@skew_mm
+PREHOOK: Output: default@skew_mm
+POSTHOOK: query: drop table skew_mm
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@skew_mm
+POSTHOOK: Output: default@skew_mm
+PREHOOK: query: create table skew_dp_union_mm(k1 int, k2 int, k4 int) partitioned by (k3 int) 
+skewed by (k1, k4) on ((0,0),(1,1),(2,2),(3,3)) stored as directories tblproperties ('hivecommit'='true')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@skew_dp_union_mm
+POSTHOOK: query: create table skew_dp_union_mm(k1 int, k2 int, k4 int) partitioned by (k3 int) 
+skewed by (k1, k4) on ((0,0),(1,1),(2,2),(3,3)) stored as directories tblproperties ('hivecommit'='true')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@skew_dp_union_mm
+PREHOOK: query: insert into table skew_dp_union_mm partition (k3)
+select key as i, key as j, key as k, key as l from intermediate
+union all 
+select key +1 as i, key +2 as j, key +3 as k, key +4 as l from intermediate
+PREHOOK: type: QUERY
+PREHOOK: Input: default@intermediate
+PREHOOK: Input: default@intermediate@p=455
+PREHOOK: Input: default@intermediate@p=456
+PREHOOK: Output: default@skew_dp_union_mm
+POSTHOOK: query: insert into table skew_dp_union_mm partition (k3)
+select key as i, key as j, key as k, key as l from intermediate
+union all 
+select key +1 as i, key +2 as j, key +3 as k, key +4 as l from intermediate
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@intermediate
+POSTHOOK: Input: default@intermediate@p=455
+POSTHOOK: Input: default@intermediate@p=456
+POSTHOOK: Output: default@skew_dp_union_mm@k3=0
+POSTHOOK: Output: default@skew_dp_union_mm@k3=10
+POSTHOOK: Output: default@skew_dp_union_mm@k3=101
+POSTHOOK: Output: default@skew_dp_union_mm@k3=102
+POSTHOOK: Output: default@skew_dp_union_mm@k3=14
+POSTHOOK: Output: default@skew_dp_union_mm@k3=4
+POSTHOOK: Output: default@skew_dp_union_mm@k3=97
+POSTHOOK: Output: default@skew_dp_union_mm@k3=98
+POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=0).k1 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=0).k2 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=0).k4 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=101).k1 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=101).k2 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=101).k4 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=102).k1 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=102).k2 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=102).k4 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=10).k1 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=10).k2 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=10).k4 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=14).k1 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=14).k2 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=14).k4 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=4).k1 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=4).k2 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=4).k4 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=97).k1 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=97).k2 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=97).k4 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=98).k1 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=98).k2 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: skew_dp_union_mm PARTITION(k3=98).k4 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+PREHOOK: query: select * from skew_dp_union_mm order by k2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@skew_dp_union_mm
+PREHOOK: Input: default@skew_dp_union_mm@k3=0
+PREHOOK: Input: default@skew_dp_union_mm@k3=10
+PREHOOK: Input: default@skew_dp_union_mm@k3=101
+PREHOOK: Input: default@skew_dp_union_mm@k3=102
+PREHOOK: Input: default@skew_dp_union_mm@k3=14
+PREHOOK: Input: default@skew_dp_union_mm@k3=4
+PREHOOK: Input: default@skew_dp_union_mm@k3=97
+PREHOOK: Input: default@skew_dp_union_mm@k3=98
+#### A masked pattern was here ####
+POSTHOOK: query: select * from skew_dp_union_mm order by k2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@skew_dp_union_mm
+POSTHOOK: Input: default@skew_dp_union_mm@k3=0
+POSTHOOK: Input: default@skew_dp_union_mm@k3=10
+POSTHOOK: Input: default@skew_dp_union_mm@k3=101
+POSTHOOK: Input: default@skew_dp_union_mm@k3=102
+POSTHOOK: Input: default@skew_dp_union_mm@k3=14
+POSTHOOK: Input: default@skew_dp_union_mm@k3=4
+POSTHOOK: Input: default@skew_dp_union_mm@k3=97
+POSTHOOK: Input: default@skew_dp_union_mm@k3=98
+#### A masked pattern was here ####
+0	0	0	0
+1	2	3	4
+10	10	10	10
+11	12	13	14
+97	97	97	97
+98	98	98	98
+98	99	100	101
+99	100	101	102
+PREHOOK: query: drop table skew_dp_union_mm
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@skew_dp_union_mm
+PREHOOK: Output: default@skew_dp_union_mm
+POSTHOOK: query: drop table skew_dp_union_mm
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@skew_dp_union_mm
+POSTHOOK: Output: default@skew_dp_union_mm
+PREHOOK: query: -- future
 
 
 
 
 
---drop table partunion_mm;
 --drop table merge_mm;
 --drop table ctas_mm;
---drop table T1;
---drop table T2;
---drop table skew_mm;
 --
 --
 --create table ctas_mm tblproperties ('hivecommit'='true') as select * from src limit 3;
 --
---create table partunion_mm(id_mm int) partitioned by (key_mm int)  tblproperties ('hivecommit'='true');
---
---
---insert into table partunion_mm partition(key_mm)
---select temps.* from (
---select key as key_mm, key from ctas_mm 
---union all 
---select key as key_mm, key from simple_mm ) temps;
 --
 --set hive.merge.mapredfiles=true;
 --set hive.merge.sparkfiles=true;
@@ -657,22 +765,6 @@ PREHOOK: query: -- TODO# from here, fix it
 --        FROM src;
 --
 --
---set hive.optimize.skewjoin.compiletime = true;
----- the test case is wrong?
---
---CREATE TABLE T1(key STRING, val STRING)
---SKEWED BY (key) ON ((2)) STORED AS TEXTFILE;
---LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
---CREATE TABLE T2(key STRING, val STRING)
---SKEWED BY (key) ON ((3)) STORED AS TEXTFILE;
---LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2;
---
---EXPLAIN
---SELECT a.*, b.* FROM T1 a JOIN T2 b ON a.key = b.key;
---
---create table skew_mm(k1 string, k2 string, k3 string, k4 string) SKEWED BY (key) ON ((2)) tblproperties ('hivecommit'='true');
---INSERT OVERWRITE TABLE skew_mm
---SELECT a.key as k1, a.val as k2, b.key as k3, b.val as k4 FROM T1 a JOIN T2 b ON a.key = b.key;
 --
 ---- TODO load, multi-insert etc
 --
@@ -682,35 +774,18 @@ drop table intermediate
 PREHOOK: type: DROPTABLE
 PREHOOK: Input: default@intermediate
 PREHOOK: Output: default@intermediate
-POSTHOOK: query: -- TODO# from here, fix it
-
-
-
-
--- future
+POSTHOOK: query: -- future
 
 
 
 
 
---drop table partunion_mm;
 --drop table merge_mm;
 --drop table ctas_mm;
---drop table T1;
---drop table T2;
---drop table skew_mm;
 --
 --
 --create table ctas_mm tblproperties ('hivecommit'='true') as select * from src limit 3;
 --
---create table partunion_mm(id_mm int) partitioned by (key_mm int)  tblproperties ('hivecommit'='true');
---
---
---insert into table partunion_mm partition(key_mm)
---select temps.* from (
---select key as key_mm, key from ctas_mm 
---union all 
---select key as key_mm, key from simple_mm ) temps;
 --
 --set hive.merge.mapredfiles=true;
 --set hive.merge.sparkfiles=true;
@@ -729,22 +804,6 @@ POSTHOOK: query: -- TODO# from here, fix it
 --        FROM src;
 --
 --
---set hive.optimize.skewjoin.compiletime = true;
----- the test case is wrong?
---
---CREATE TABLE T1(key STRING, val STRING)
---SKEWED BY (key) ON ((2)) STORED AS TEXTFILE;
---LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
---CREATE TABLE T2(key STRING, val STRING)
---SKEWED BY (key) ON ((3)) STORED AS TEXTFILE;
---LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2;
---
---EXPLAIN
---SELECT a.*, b.* FROM T1 a JOIN T2 b ON a.key = b.key;
---
---create table skew_mm(k1 string, k2 string, k3 string, k4 string) SKEWED BY (key) ON ((2)) tblproperties ('hivecommit'='true');
---INSERT OVERWRITE TABLE skew_mm
---SELECT a.key as k1, a.val as k2, b.key as k3, b.val as k4 FROM T1 a JOIN T2 b ON a.key = b.key;
 --
 ---- TODO load, multi-insert etc
 --

http://git-wip-us.apache.org/repos/asf/hive/blob/b9e81572/ql/src/test/results/clientpositive/llap/mm_current.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/mm_current.q.out b/ql/src/test/results/clientpositive/llap/mm_current.q.out
index 87214ba..d6d31ea 100644
--- a/ql/src/test/results/clientpositive/llap/mm_current.q.out
+++ b/ql/src/test/results/clientpositive/llap/mm_current.q.out
@@ -28,140 +28,163 @@ POSTHOOK: type: QUERY
 POSTHOOK: Input: default@src
 POSTHOOK: Output: default@intermediate@p=456
 POSTHOOK: Lineage: intermediate PARTITION(p=456).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ]
-PREHOOK: query: create table partunion_no_mm(id int) partitioned by (key int)
+PREHOOK: query: create table skew_mm(k1 int, k2 int, k4 int) skewed by (k1, k4) on ((0,0),(1,1),(2,2),(3,3))
+ stored as directories tblproperties ('hivecommit'='false')
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
-PREHOOK: Output: default@partunion_no_mm
-POSTHOOK: query: create table partunion_no_mm(id int) partitioned by (key int)
+PREHOOK: Output: default@skew_mm
+POSTHOOK: query: create table skew_mm(k1 int, k2 int, k4 int) skewed by (k1, k4) on ((0,0),(1,1),(2,2),(3,3))
+ stored as directories tblproperties ('hivecommit'='false')
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
-POSTHOOK: Output: default@partunion_no_mm
-PREHOOK: query: insert into table partunion_no_mm partition(key)
-select temps.* from (
-select key as p, key from intermediate 
-union all 
-select key + 1 as p, key + 1 from intermediate ) temps
+POSTHOOK: Output: default@skew_mm
+PREHOOK: query: insert into table skew_mm 
+select key, key, key from intermediate
 PREHOOK: type: QUERY
 PREHOOK: Input: default@intermediate
 PREHOOK: Input: default@intermediate@p=455
 PREHOOK: Input: default@intermediate@p=456
-PREHOOK: Output: default@partunion_no_mm
-POSTHOOK: query: insert into table partunion_no_mm partition(key)
-select temps.* from (
-select key as p, key from intermediate 
-union all 
-select key + 1 as p, key + 1 from intermediate ) temps
+PREHOOK: Output: default@skew_mm
+POSTHOOK: query: insert into table skew_mm 
+select key, key, key from intermediate
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@intermediate
+POSTHOOK: Input: default@intermediate@p=455
+POSTHOOK: Input: default@intermediate@p=456
+POSTHOOK: Output: default@skew_mm
+POSTHOOK: Lineage: skew_mm.k1 SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: skew_mm.k2 SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: skew_mm.k4 SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+PREHOOK: query: drop table skew_mm
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@skew_mm
+PREHOOK: Output: default@skew_mm
+POSTHOOK: query: drop table skew_mm
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@skew_mm
+POSTHOOK: Output: default@skew_mm
+PREHOOK: query: create table skew_mm(k1 int, k2 int, k4 int) skewed by (k1, k4) on ((0,0),(1,1),(2,2),(3,3))
+ stored as directories tblproperties ('hivecommit'='true')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@skew_mm
+POSTHOOK: query: create table skew_mm(k1 int, k2 int, k4 int) skewed by (k1, k4) on ((0,0),(1,1),(2,2),(3,3))
+ stored as directories tblproperties ('hivecommit'='true')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@skew_mm
+PREHOOK: query: insert into table skew_mm 
+select key, key, key from intermediate
+PREHOOK: type: QUERY
+PREHOOK: Input: default@intermediate
+PREHOOK: Input: default@intermediate@p=455
+PREHOOK: Input: default@intermediate@p=456
+PREHOOK: Output: default@skew_mm
+POSTHOOK: query: insert into table skew_mm 
+select key, key, key from intermediate
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@intermediate
 POSTHOOK: Input: default@intermediate@p=455
 POSTHOOK: Input: default@intermediate@p=456
-POSTHOOK: Output: default@partunion_no_mm@key=0
-POSTHOOK: Output: default@partunion_no_mm@key=1
-POSTHOOK: Output: default@partunion_no_mm@key=455
-POSTHOOK: Output: default@partunion_no_mm@key=456
-POSTHOOK: Lineage: partunion_no_mm PARTITION(key=0).id EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
-POSTHOOK: Lineage: partunion_no_mm PARTITION(key=1).id EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
-POSTHOOK: Lineage: partunion_no_mm PARTITION(key=455).id EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
-POSTHOOK: Lineage: partunion_no_mm PARTITION(key=456).id EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
-PREHOOK: query: select * from partunion_no_mm
+POSTHOOK: Output: default@skew_mm
+POSTHOOK: Lineage: skew_mm.k1 SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: skew_mm.k2 SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: skew_mm.k4 SIMPLE [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+PREHOOK: query: select * from skew_mm
 PREHOOK: type: QUERY
-PREHOOK: Input: default@partunion_no_mm
-PREHOOK: Input: default@partunion_no_mm@key=0
-PREHOOK: Input: default@partunion_no_mm@key=1
-PREHOOK: Input: default@partunion_no_mm@key=455
-PREHOOK: Input: default@partunion_no_mm@key=456
+PREHOOK: Input: default@skew_mm
 #### A masked pattern was here ####
-POSTHOOK: query: select * from partunion_no_mm
+POSTHOOK: query: select * from skew_mm
 POSTHOOK: type: QUERY
-POSTHOOK: Input: default@partunion_no_mm
-POSTHOOK: Input: default@partunion_no_mm@key=0
-POSTHOOK: Input: default@partunion_no_mm@key=1
-POSTHOOK: Input: default@partunion_no_mm@key=455
-POSTHOOK: Input: default@partunion_no_mm@key=456
+POSTHOOK: Input: default@skew_mm
 #### A masked pattern was here ####
-0	0
-0	0
-1	1
-1	1
-455	455
-455	455
-456	456
-456	456
-PREHOOK: query: drop table partunion_no_mm
+455	455	455
+455	455	455
+0	0	0
+0	0	0
+PREHOOK: query: drop table skew_mm
 PREHOOK: type: DROPTABLE
-PREHOOK: Input: default@partunion_no_mm
-PREHOOK: Output: default@partunion_no_mm
-POSTHOOK: query: drop table partunion_no_mm
+PREHOOK: Input: default@skew_mm
+PREHOOK: Output: default@skew_mm
+POSTHOOK: query: drop table skew_mm
 POSTHOOK: type: DROPTABLE
-POSTHOOK: Input: default@partunion_no_mm
-POSTHOOK: Output: default@partunion_no_mm
-PREHOOK: query: create table partunion_mm(id int) partitioned by (key int) tblproperties ('hivecommit'='true')
+POSTHOOK: Input: default@skew_mm
+POSTHOOK: Output: default@skew_mm
+PREHOOK: query: create table skew_mm(k1 int, k2 int, k4 int) partitioned by (k3 int) 
+skewed by (k1, k4) on ((0,0),(1,1),(2,2),(3,3)) stored as directories tblproperties ('hivecommit'='true')
 PREHOOK: type: CREATETABLE
 PREHOOK: Output: database:default
-PREHOOK: Output: default@partunion_mm
-POSTHOOK: query: create table partunion_mm(id int) partitioned by (key int) tblproperties ('hivecommit'='true')
+PREHOOK: Output: default@skew_mm
+POSTHOOK: query: create table skew_mm(k1 int, k2 int, k4 int) partitioned by (k3 int) 
+skewed by (k1, k4) on ((0,0),(1,1),(2,2),(3,3)) stored as directories tblproperties ('hivecommit'='true')
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
-POSTHOOK: Output: default@partunion_mm
-PREHOOK: query: insert into table partunion_mm partition(key)
-select temps.* from (
-select key as p, key from intermediate 
+POSTHOOK: Output: default@skew_mm
+PREHOOK: query: insert into table skew_mm partition (k3)
+select key as i, key as j, key as k, key as l from intermediate
 union all 
-select key + 1 as p, key + 1 from intermediate ) temps
+select key +1 as i, key +2 as j, key +3 as k, key +4 as l from intermediate
 PREHOOK: type: QUERY
 PREHOOK: Input: default@intermediate
 PREHOOK: Input: default@intermediate@p=455
 PREHOOK: Input: default@intermediate@p=456
-PREHOOK: Output: default@partunion_mm
-POSTHOOK: query: insert into table partunion_mm partition(key)
-select temps.* from (
-select key as p, key from intermediate 
+PREHOOK: Output: default@skew_mm
+POSTHOOK: query: insert into table skew_mm partition (k3)
+select key as i, key as j, key as k, key as l from intermediate
 union all 
-select key + 1 as p, key + 1 from intermediate ) temps
+select key +1 as i, key +2 as j, key +3 as k, key +4 as l from intermediate
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@intermediate
 POSTHOOK: Input: default@intermediate@p=455
 POSTHOOK: Input: default@intermediate@p=456
-POSTHOOK: Output: default@partunion_mm@key=0
-POSTHOOK: Output: default@partunion_mm@key=1
-POSTHOOK: Output: default@partunion_mm@key=455
-POSTHOOK: Output: default@partunion_mm@key=456
-POSTHOOK: Lineage: partunion_mm PARTITION(key=0).id EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
-POSTHOOK: Lineage: partunion_mm PARTITION(key=1).id EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
-POSTHOOK: Lineage: partunion_mm PARTITION(key=455).id EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
-POSTHOOK: Lineage: partunion_mm PARTITION(key=456).id EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
-PREHOOK: query: select * from partunion_mm
+POSTHOOK: Output: default@skew_mm@k3=0
+POSTHOOK: Output: default@skew_mm@k3=4
+POSTHOOK: Output: default@skew_mm@k3=455
+POSTHOOK: Output: default@skew_mm@k3=459
+POSTHOOK: Lineage: skew_mm PARTITION(k3=0).k1 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: skew_mm PARTITION(k3=0).k2 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: skew_mm PARTITION(k3=0).k4 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: skew_mm PARTITION(k3=455).k1 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: skew_mm PARTITION(k3=455).k2 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: skew_mm PARTITION(k3=455).k4 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: skew_mm PARTITION(k3=459).k1 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: skew_mm PARTITION(k3=459).k2 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: skew_mm PARTITION(k3=459).k4 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: skew_mm PARTITION(k3=4).k1 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: skew_mm PARTITION(k3=4).k2 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: skew_mm PARTITION(k3=4).k4 EXPRESSION [(intermediate)intermediate.FieldSchema(name:key, type:int, comment:null), ]
+PREHOOK: query: select * from skew_mm
 PREHOOK: type: QUERY
-PREHOOK: Input: default@partunion_mm
-PREHOOK: Input: default@partunion_mm@key=0
-PREHOOK: Input: default@partunion_mm@key=1
-PREHOOK: Input: default@partunion_mm@key=455
-PREHOOK: Input: default@partunion_mm@key=456
+PREHOOK: Input: default@skew_mm
+PREHOOK: Input: default@skew_mm@k3=0
+PREHOOK: Input: default@skew_mm@k3=4
+PREHOOK: Input: default@skew_mm@k3=455
+PREHOOK: Input: default@skew_mm@k3=459
 #### A masked pattern was here ####
-POSTHOOK: query: select * from partunion_mm
+POSTHOOK: query: select * from skew_mm
 POSTHOOK: type: QUERY
-POSTHOOK: Input: default@partunion_mm
-POSTHOOK: Input: default@partunion_mm@key=0
-POSTHOOK: Input: default@partunion_mm@key=1
-POSTHOOK: Input: default@partunion_mm@key=455
-POSTHOOK: Input: default@partunion_mm@key=456
+POSTHOOK: Input: default@skew_mm
+POSTHOOK: Input: default@skew_mm@k3=0
+POSTHOOK: Input: default@skew_mm@k3=4
+POSTHOOK: Input: default@skew_mm@k3=455
+POSTHOOK: Input: default@skew_mm@k3=459
 #### A masked pattern was here ####
-0	0
-0	0
-1	1
-1	1
-455	455
-455	455
-456	456
-456	456
-PREHOOK: query: drop table partunion_mm
+0	0	0	0
+0	0	0	0
+1	2	3	4
+1	2	3	4
+455	455	455	455
+455	455	455	455
+456	457	458	459
+456	457	458	459
+PREHOOK: query: drop table skew_mm
 PREHOOK: type: DROPTABLE
-PREHOOK: Input: default@partunion_mm
-PREHOOK: Output: default@partunion_mm
-POSTHOOK: query: drop table partunion_mm
+PREHOOK: Input: default@skew_mm
+PREHOOK: Output: default@skew_mm
+POSTHOOK: query: drop table skew_mm
 POSTHOOK: type: DROPTABLE
-POSTHOOK: Input: default@partunion_mm
-POSTHOOK: Output: default@partunion_mm
+POSTHOOK: Input: default@skew_mm
+POSTHOOK: Output: default@skew_mm
 PREHOOK: query: drop table intermediate
 PREHOOK: type: DROPTABLE
 PREHOOK: Input: default@intermediate


[08/22] hive git commit: HIVE-14892: Test that explicitly submit jobs via child process are slow (Prasanth Jayachandran reviewed by Siddharth Seth)

Posted by se...@apache.org.
HIVE-14892: Test that explicitly submit jobs via child process are slow (Prasanth Jayachandran reviewed by Siddharth Seth)


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

Branch: refs/heads/hive-14535
Commit: 8e0b19b3c469c1e0d4165cc1e4652093195fc587
Parents: a0bf9d6
Author: Prasanth Jayachandran <pr...@apache.org>
Authored: Wed Oct 5 21:12:08 2016 -0700
Committer: Prasanth Jayachandran <pr...@apache.org>
Committed: Wed Oct 5 21:12:08 2016 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hive/ql/exec/mr/MapRedTask.java  | 17 ++++++++++++++++-
 .../queries/clientpositive/alter_merge_stats_orc.q |  4 ++--
 .../clientpositive/archive_excludeHadoop20.q       |  5 ++---
 .../queries/clientpositive/auto_sortmerge_join_8.q |  4 ++--
 ql/src/test/queries/clientpositive/nonmr_fetch.q   |  4 ++--
 ql/src/test/queries/clientpositive/orc_analyze.q   |  4 ++--
 ql/src/test/queries/clientpositive/sample10.q      |  4 ++--
 .../clientpositive/sample_islocalmode_hook.q       |  4 ++--
 .../sample_islocalmode_hook_use_metadata.q         |  6 +++---
 .../queries/clientpositive/vectorized_parquet.q    |  4 ++--
 10 files changed, 35 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/8e0b19b3/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapRedTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapRedTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapRedTask.java
index 55bab6c..17db852 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapRedTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapRedTask.java
@@ -138,9 +138,24 @@ public class MapRedTask extends ExecDriver implements Serializable {
       runningViaChild = conf.getBoolVar(HiveConf.ConfVars.SUBMITVIACHILD);
 
       if (!runningViaChild) {
+        // since we are running the mapred task in the same jvm, we should update the job conf
+        // in ExecDriver as well to have proper local properties.
+        if (this.isLocalMode()) {
+          // save the original job tracker
+          ctx.setOriginalTracker(ShimLoader.getHadoopShims().getJobLauncherRpcAddress(job));
+          // change it to local
+          ShimLoader.getHadoopShims().setJobLauncherRpcAddress(job, "local");
+        }
         // we are not running this mapred task via child jvm
         // so directly invoke ExecDriver
-        return super.execute(driverContext);
+        int ret = super.execute(driverContext);
+
+        // restore the previous properties for framework name, RM address etc.
+        if (this.isLocalMode()) {
+          // restore the local job tracker back to original
+          ctx.restoreOriginalTracker();
+        }
+        return ret;
       }
 
       // we need to edit the configuration to setup cmdline. clone it first

http://git-wip-us.apache.org/repos/asf/hive/blob/8e0b19b3/ql/src/test/queries/clientpositive/alter_merge_stats_orc.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/alter_merge_stats_orc.q b/ql/src/test/queries/clientpositive/alter_merge_stats_orc.q
index 66a7c10..8d0c7bd 100644
--- a/ql/src/test/queries/clientpositive/alter_merge_stats_orc.q
+++ b/ql/src/test/queries/clientpositive/alter_merge_stats_orc.q
@@ -1,5 +1,5 @@
-set hive.exec.submitviachild=true;
-set hive.exec.submit.local.task.via.child=true;
+set hive.exec.submitviachild=false;
+set hive.exec.submit.local.task.via.child=false;
 create table src_orc_merge_test_stat(key int, value string) stored as orc;
 
 insert overwrite table src_orc_merge_test_stat select * from src;

http://git-wip-us.apache.org/repos/asf/hive/blob/8e0b19b3/ql/src/test/queries/clientpositive/archive_excludeHadoop20.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/archive_excludeHadoop20.q b/ql/src/test/queries/clientpositive/archive_excludeHadoop20.q
index 37d5cf9..e961641 100644
--- a/ql/src/test/queries/clientpositive/archive_excludeHadoop20.q
+++ b/ql/src/test/queries/clientpositive/archive_excludeHadoop20.q
@@ -1,8 +1,7 @@
 set hive.mapred.mode=nonstrict;
 set hive.archive.enabled = true;
-;
-set hive.exec.submitviachild=true;
-set hive.exec.submit.local.task.via.child=true;
+set hive.exec.submitviachild=false;
+set hive.exec.submit.local.task.via.child=false;
 
 drop table tstsrc;
 drop table tstsrcpart;

http://git-wip-us.apache.org/repos/asf/hive/blob/8e0b19b3/ql/src/test/queries/clientpositive/auto_sortmerge_join_8.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/auto_sortmerge_join_8.q b/ql/src/test/queries/clientpositive/auto_sortmerge_join_8.q
index 1bd5da4..cc2ba28 100644
--- a/ql/src/test/queries/clientpositive/auto_sortmerge_join_8.q
+++ b/ql/src/test/queries/clientpositive/auto_sortmerge_join_8.q
@@ -1,6 +1,6 @@
 set hive.mapred.mode=nonstrict;
-set hive.exec.submitviachild=true;
-set hive.exec.submit.local.task.via.child=true;
+set hive.exec.submitviachild=false;
+set hive.exec.submit.local.task.via.child=false;
 
 -- small 2 part, 2 bucket & big 2 part, 4 bucket
 CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;

http://git-wip-us.apache.org/repos/asf/hive/blob/8e0b19b3/ql/src/test/queries/clientpositive/nonmr_fetch.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/nonmr_fetch.q b/ql/src/test/queries/clientpositive/nonmr_fetch.q
index a446217..dffdc2c 100644
--- a/ql/src/test/queries/clientpositive/nonmr_fetch.q
+++ b/ql/src/test/queries/clientpositive/nonmr_fetch.q
@@ -1,7 +1,7 @@
 set hive.mapred.mode=nonstrict;
 set hive.fetch.task.conversion=minimal;
-set hive.exec.submitviachild=true;
-set hive.exec.submit.local.task.via.child=true;
+set hive.exec.submitviachild=false;
+set hive.exec.submit.local.task.via.child=false;
 
 -- backward compatible (minimal)
 explain select * from src limit 10;

http://git-wip-us.apache.org/repos/asf/hive/blob/8e0b19b3/ql/src/test/queries/clientpositive/orc_analyze.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/orc_analyze.q b/ql/src/test/queries/clientpositive/orc_analyze.q
index 4cbe1e6..3f9a592 100644
--- a/ql/src/test/queries/clientpositive/orc_analyze.q
+++ b/ql/src/test/queries/clientpositive/orc_analyze.q
@@ -1,6 +1,6 @@
 set hive.mapred.mode=nonstrict;
-set hive.exec.submitviachild=true;
-set hive.exec.submit.local.task.via.child=true;
+set hive.exec.submitviachild=false;
+set hive.exec.submit.local.task.via.child=false;
 
 CREATE TABLE orc_create_people_staging (
   id int,

http://git-wip-us.apache.org/repos/asf/hive/blob/8e0b19b3/ql/src/test/queries/clientpositive/sample10.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/sample10.q b/ql/src/test/queries/clientpositive/sample10.q
index b410644..a5a8c8f 100644
--- a/ql/src/test/queries/clientpositive/sample10.q
+++ b/ql/src/test/queries/clientpositive/sample10.q
@@ -1,6 +1,6 @@
 set hive.mapred.mode=nonstrict;
-set hive.exec.submitviachild=true;
-set hive.exec.submit.local.task.via.child=true;
+set hive.exec.submitviachild=false;
+set hive.exec.submit.local.task.via.child=false;
 set hive.exec.dynamic.partition=true;
 set hive.exec.dynamic.partition.mode=nonstrict;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/8e0b19b3/ql/src/test/queries/clientpositive/sample_islocalmode_hook.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/sample_islocalmode_hook.q b/ql/src/test/queries/clientpositive/sample_islocalmode_hook.q
index 3f986e0..08f8cdf 100644
--- a/ql/src/test/queries/clientpositive/sample_islocalmode_hook.q
+++ b/ql/src/test/queries/clientpositive/sample_islocalmode_hook.q
@@ -1,6 +1,6 @@
 set hive.mapred.mode=nonstrict;
-set hive.exec.submitviachild=true;
-set hive.exec.submit.local.task.via.child=true;
+set hive.exec.submitviachild=false;
+set hive.exec.submit.local.task.via.child=false;
 set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
 set mapred.max.split.size=300;
 set mapred.min.split.size=300;

http://git-wip-us.apache.org/repos/asf/hive/blob/8e0b19b3/ql/src/test/queries/clientpositive/sample_islocalmode_hook_use_metadata.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/sample_islocalmode_hook_use_metadata.q b/ql/src/test/queries/clientpositive/sample_islocalmode_hook_use_metadata.q
index ac915b5..b523f06 100644
--- a/ql/src/test/queries/clientpositive/sample_islocalmode_hook_use_metadata.q
+++ b/ql/src/test/queries/clientpositive/sample_islocalmode_hook_use_metadata.q
@@ -1,6 +1,6 @@
 set hive.mapred.mode=nonstrict;
-set hive.exec.submitviachild=true;
-set hive.exec.submit.local.task.via.child=true;
+set hive.exec.submitviachild=false;
+set hive.exec.submit.local.task.via.child=false;
 set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
 set mapred.max.split.size=300;
 set mapred.min.split.size=300;
@@ -45,4 +45,4 @@ select count(1) from sih_src tablesample(1 percent);
 
 explain select count(1) from sih_src tablesample(10 rows);
 
-select count(1) from sih_src tablesample(10 rows);
\ No newline at end of file
+select count(1) from sih_src tablesample(10 rows);

http://git-wip-us.apache.org/repos/asf/hive/blob/8e0b19b3/ql/src/test/queries/clientpositive/vectorized_parquet.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/vectorized_parquet.q b/ql/src/test/queries/clientpositive/vectorized_parquet.q
index a49ca63..da138e0 100644
--- a/ql/src/test/queries/clientpositive/vectorized_parquet.q
+++ b/ql/src/test/queries/clientpositive/vectorized_parquet.q
@@ -1,6 +1,6 @@
 set hive.explain.user=false;
-set hive.exec.submitviachild=true;
-set hive.exec.submit.local.task.via.child=true;
+set hive.exec.submitviachild=false;
+set hive.exec.submit.local.task.via.child=false;
 
 create table if not exists alltypes_parquet (
   cint int, 


[04/22] hive git commit: HIVE-14873: Add UDF for extraction of 'day of week' (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/59539885/ql/src/test/results/clientpositive/vectorized_date_funcs.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vectorized_date_funcs.q.out b/ql/src/test/results/clientpositive/vectorized_date_funcs.q.out
index 299d3bc..68a2a4d 100644
--- a/ql/src/test/results/clientpositive/vectorized_date_funcs.q.out
+++ b/ql/src/test/results/clientpositive/vectorized_date_funcs.q.out
@@ -209,6 +209,7 @@ PREHOOK: query: EXPLAIN SELECT
   month(fl_time),
   day(fl_time),
   dayofmonth(fl_time),
+  dayofweek(fl_time),
   weekofyear(fl_time),
   date(fl_time),
   to_date(fl_time),
@@ -223,6 +224,7 @@ POSTHOOK: query: EXPLAIN SELECT
   month(fl_time),
   day(fl_time),
   dayofmonth(fl_time),
+  dayofweek(fl_time),
   weekofyear(fl_time),
   date(fl_time),
   to_date(fl_time),
@@ -243,8 +245,8 @@ STAGE PLANS:
             alias: date_udf_flight_orc
             Statistics: Num rows: 137 Data size: 13152 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              expressions: to_unix_timestamp(fl_time) (type: bigint), year(fl_time) (type: int), month(fl_time) (type: int), day(fl_time) (type: int), dayofmonth(fl_time) (type: int), weekofyear(fl_time) (type: int), CAST( fl_time AS DATE) (type: date), to_date(fl_time) (type: date), date_add(fl_time, 2) (type: date), date_sub(fl_time, 2) (type: date), datediff(fl_time, '2000-01-01') (type: int)
-              outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
+              expressions: to_unix_timestamp(fl_time) (type: bigint), year(fl_time) (type: int), month(fl_time) (type: int), day(fl_time) (type: int), dayofmonth(fl_time) (type: int), dayofweek(fl_time) (type: int), weekofyear(fl_time) (type: int), CAST( fl_time AS DATE) (type: date), to_date(fl_time) (type: date), date_add(fl_time, 2) (type: date), date_sub(fl_time, 2) (type: date), datediff(fl_time, '2000-01-01') (type: int)
+              outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
               Statistics: Num rows: 137 Data size: 13152 Basic stats: COMPLETE Column stats: NONE
               File Output Operator
                 compressed: false
@@ -267,6 +269,7 @@ PREHOOK: query: SELECT
   month(fl_time),
   day(fl_time),
   dayofmonth(fl_time),
+  dayofweek(fl_time),
   weekofyear(fl_time),
   date(fl_time),
   to_date(fl_time),
@@ -283,6 +286,7 @@ POSTHOOK: query: SELECT
   month(fl_time),
   day(fl_time),
   dayofmonth(fl_time),
+  dayofweek(fl_time),
   weekofyear(fl_time),
   date(fl_time),
   to_date(fl_time),
@@ -293,149 +297,150 @@ FROM date_udf_flight_orc
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@date_udf_flight_orc
 #### A masked pattern was here ####
-1287583200	2010	10	20	20	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
-1287583200	2010	10	20	20	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
-1287583200	2010	10	20	20	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
-1287583200	2010	10	20	20	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
-1287583200	2010	10	20	20	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
-1287583200	2010	10	20	20	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
-1287583200	2010	10	20	20	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
-1287583200	2010	10	20	20	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
-1287583200	2010	10	20	20	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
-1287583200	2010	10	20	20	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
-1287583200	2010	10	20	20	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
-1287669600	2010	10	21	21	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
-1287669600	2010	10	21	21	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
-1287669600	2010	10	21	21	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
-1287669600	2010	10	21	21	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
-1287669600	2010	10	21	21	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
-1287669600	2010	10	21	21	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
-1287669600	2010	10	21	21	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
-1287669600	2010	10	21	21	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
-1287669600	2010	10	21	21	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
-1287669600	2010	10	21	21	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
-1287669600	2010	10	21	21	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
-1287669600	2010	10	21	21	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
-1287756000	2010	10	22	22	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
-1287756000	2010	10	22	22	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
-1287756000	2010	10	22	22	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
-1287756000	2010	10	22	22	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
-1287756000	2010	10	22	22	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
-1287756000	2010	10	22	22	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
-1287756000	2010	10	22	22	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
-1287756000	2010	10	22	22	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
-1287756000	2010	10	22	22	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
-1287756000	2010	10	22	22	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
-1287756000	2010	10	22	22	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
-1287842400	2010	10	23	23	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
-1287842400	2010	10	23	23	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
-1287842400	2010	10	23	23	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
-1287842400	2010	10	23	23	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
-1287842400	2010	10	23	23	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
-1287842400	2010	10	23	23	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
-1287842400	2010	10	23	23	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
-1287842400	2010	10	23	23	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
-1287842400	2010	10	23	23	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
-1287842400	2010	10	23	23	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
-1287842400	2010	10	23	23	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
-1287842400	2010	10	23	23	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
-1287928800	2010	10	24	24	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
-1287928800	2010	10	24	24	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
-1287928800	2010	10	24	24	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
-1287928800	2010	10	24	24	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
-1287928800	2010	10	24	24	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
-1287928800	2010	10	24	24	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
-1287928800	2010	10	24	24	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
-1287928800	2010	10	24	24	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
-1287928800	2010	10	24	24	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
-1287928800	2010	10	24	24	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
-1287928800	2010	10	24	24	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
-1287928800	2010	10	24	24	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
-1288015200	2010	10	25	25	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
-1288015200	2010	10	25	25	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
-1288015200	2010	10	25	25	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
-1288015200	2010	10	25	25	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
-1288015200	2010	10	25	25	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
-1288015200	2010	10	25	25	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
-1288015200	2010	10	25	25	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
-1288015200	2010	10	25	25	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
-1288015200	2010	10	25	25	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
-1288015200	2010	10	25	25	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
-1288015200	2010	10	25	25	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
-1288015200	2010	10	25	25	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
-1288101600	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288101600	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288101600	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288101600	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288101600	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288101600	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288101600	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288101600	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288101600	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288101600	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288101600	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288101600	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288101600	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288188000	2010	10	27	27	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
-1288188000	2010	10	27	27	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
-1288188000	2010	10	27	27	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
-1288188000	2010	10	27	27	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
-1288188000	2010	10	27	27	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
-1288188000	2010	10	27	27	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
-1288188000	2010	10	27	27	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
-1288188000	2010	10	27	27	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
-1288188000	2010	10	27	27	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
-1288188000	2010	10	27	27	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
-1288188000	2010	10	27	27	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
-1288274400	2010	10	28	28	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
-1288274400	2010	10	28	28	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
-1288274400	2010	10	28	28	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
-1288274400	2010	10	28	28	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
-1288274400	2010	10	28	28	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
-1288274400	2010	10	28	28	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
-1288274400	2010	10	28	28	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
-1288274400	2010	10	28	28	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
-1288274400	2010	10	28	28	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
-1288274400	2010	10	28	28	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
-1288274400	2010	10	28	28	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
-1288274400	2010	10	28	28	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
-1288360800	2010	10	29	29	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
-1288360800	2010	10	29	29	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
-1288360800	2010	10	29	29	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
-1288360800	2010	10	29	29	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
-1288360800	2010	10	29	29	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
-1288360800	2010	10	29	29	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
-1288360800	2010	10	29	29	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
-1288360800	2010	10	29	29	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
-1288360800	2010	10	29	29	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
-1288360800	2010	10	29	29	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
-1288360800	2010	10	29	29	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
-1288360800	2010	10	29	29	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
-1288447200	2010	10	30	30	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
-1288447200	2010	10	30	30	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
-1288447200	2010	10	30	30	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
-1288447200	2010	10	30	30	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
-1288447200	2010	10	30	30	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
-1288447200	2010	10	30	30	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
-1288447200	2010	10	30	30	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
-1288447200	2010	10	30	30	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
-1288447200	2010	10	30	30	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
-1288447200	2010	10	30	30	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
-1288447200	2010	10	30	30	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
-1288533600	2010	10	31	31	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
-1288533600	2010	10	31	31	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
-1288533600	2010	10	31	31	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
-1288533600	2010	10	31	31	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
-1288533600	2010	10	31	31	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
-1288533600	2010	10	31	31	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
-1288533600	2010	10	31	31	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
-1288533600	2010	10	31	31	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
+1287583200	2010	10	20	20	4	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
+1287583200	2010	10	20	20	4	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
+1287583200	2010	10	20	20	4	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
+1287583200	2010	10	20	20	4	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
+1287583200	2010	10	20	20	4	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
+1287583200	2010	10	20	20	4	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
+1287583200	2010	10	20	20	4	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
+1287583200	2010	10	20	20	4	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
+1287583200	2010	10	20	20	4	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
+1287583200	2010	10	20	20	4	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
+1287583200	2010	10	20	20	4	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
+1287669600	2010	10	21	21	5	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
+1287669600	2010	10	21	21	5	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
+1287669600	2010	10	21	21	5	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
+1287669600	2010	10	21	21	5	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
+1287669600	2010	10	21	21	5	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
+1287669600	2010	10	21	21	5	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
+1287669600	2010	10	21	21	5	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
+1287669600	2010	10	21	21	5	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
+1287669600	2010	10	21	21	5	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
+1287669600	2010	10	21	21	5	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
+1287669600	2010	10	21	21	5	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
+1287669600	2010	10	21	21	5	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
+1287756000	2010	10	22	22	6	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
+1287756000	2010	10	22	22	6	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
+1287756000	2010	10	22	22	6	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
+1287756000	2010	10	22	22	6	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
+1287756000	2010	10	22	22	6	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
+1287756000	2010	10	22	22	6	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
+1287756000	2010	10	22	22	6	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
+1287756000	2010	10	22	22	6	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
+1287756000	2010	10	22	22	6	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
+1287756000	2010	10	22	22	6	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
+1287756000	2010	10	22	22	6	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
+1287842400	2010	10	23	23	7	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
+1287842400	2010	10	23	23	7	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
+1287842400	2010	10	23	23	7	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
+1287842400	2010	10	23	23	7	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
+1287842400	2010	10	23	23	7	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
+1287842400	2010	10	23	23	7	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
+1287842400	2010	10	23	23	7	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
+1287842400	2010	10	23	23	7	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
+1287842400	2010	10	23	23	7	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
+1287842400	2010	10	23	23	7	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
+1287842400	2010	10	23	23	7	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
+1287842400	2010	10	23	23	7	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
+1287928800	2010	10	24	24	1	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
+1287928800	2010	10	24	24	1	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
+1287928800	2010	10	24	24	1	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
+1287928800	2010	10	24	24	1	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
+1287928800	2010	10	24	24	1	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
+1287928800	2010	10	24	24	1	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
+1287928800	2010	10	24	24	1	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
+1287928800	2010	10	24	24	1	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
+1287928800	2010	10	24	24	1	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
+1287928800	2010	10	24	24	1	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
+1287928800	2010	10	24	24	1	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
+1287928800	2010	10	24	24	1	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
+1288015200	2010	10	25	25	2	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
+1288015200	2010	10	25	25	2	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
+1288015200	2010	10	25	25	2	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
+1288015200	2010	10	25	25	2	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
+1288015200	2010	10	25	25	2	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
+1288015200	2010	10	25	25	2	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
+1288015200	2010	10	25	25	2	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
+1288015200	2010	10	25	25	2	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
+1288015200	2010	10	25	25	2	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
+1288015200	2010	10	25	25	2	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
+1288015200	2010	10	25	25	2	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
+1288015200	2010	10	25	25	2	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
+1288101600	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288101600	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288101600	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288101600	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288101600	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288101600	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288101600	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288101600	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288101600	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288101600	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288101600	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288101600	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288101600	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288188000	2010	10	27	27	4	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
+1288188000	2010	10	27	27	4	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
+1288188000	2010	10	27	27	4	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
+1288188000	2010	10	27	27	4	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
+1288188000	2010	10	27	27	4	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
+1288188000	2010	10	27	27	4	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
+1288188000	2010	10	27	27	4	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
+1288188000	2010	10	27	27	4	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
+1288188000	2010	10	27	27	4	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
+1288188000	2010	10	27	27	4	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
+1288188000	2010	10	27	27	4	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
+1288274400	2010	10	28	28	5	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
+1288274400	2010	10	28	28	5	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
+1288274400	2010	10	28	28	5	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
+1288274400	2010	10	28	28	5	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
+1288274400	2010	10	28	28	5	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
+1288274400	2010	10	28	28	5	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
+1288274400	2010	10	28	28	5	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
+1288274400	2010	10	28	28	5	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
+1288274400	2010	10	28	28	5	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
+1288274400	2010	10	28	28	5	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
+1288274400	2010	10	28	28	5	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
+1288274400	2010	10	28	28	5	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
+1288360800	2010	10	29	29	6	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
+1288360800	2010	10	29	29	6	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
+1288360800	2010	10	29	29	6	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
+1288360800	2010	10	29	29	6	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
+1288360800	2010	10	29	29	6	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
+1288360800	2010	10	29	29	6	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
+1288360800	2010	10	29	29	6	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
+1288360800	2010	10	29	29	6	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
+1288360800	2010	10	29	29	6	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
+1288360800	2010	10	29	29	6	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
+1288360800	2010	10	29	29	6	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
+1288360800	2010	10	29	29	6	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
+1288447200	2010	10	30	30	7	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
+1288447200	2010	10	30	30	7	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
+1288447200	2010	10	30	30	7	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
+1288447200	2010	10	30	30	7	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
+1288447200	2010	10	30	30	7	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
+1288447200	2010	10	30	30	7	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
+1288447200	2010	10	30	30	7	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
+1288447200	2010	10	30	30	7	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
+1288447200	2010	10	30	30	7	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
+1288447200	2010	10	30	30	7	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
+1288447200	2010	10	30	30	7	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
+1288533600	2010	10	31	31	1	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
+1288533600	2010	10	31	31	1	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
+1288533600	2010	10	31	31	1	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
+1288533600	2010	10	31	31	1	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
+1288533600	2010	10	31	31	1	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
+1288533600	2010	10	31	31	1	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
+1288533600	2010	10	31	31	1	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
+1288533600	2010	10	31	31	1	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
 PREHOOK: query: EXPLAIN SELECT
   to_unix_timestamp(fl_date),
   year(fl_date),
   month(fl_date),
   day(fl_date),
   dayofmonth(fl_date),
+  dayofweek(fl_date),
   weekofyear(fl_date),
   date(fl_date),
   to_date(fl_date),
@@ -450,6 +455,7 @@ POSTHOOK: query: EXPLAIN SELECT
   month(fl_date),
   day(fl_date),
   dayofmonth(fl_date),
+  dayofweek(fl_date),
   weekofyear(fl_date),
   date(fl_date),
   to_date(fl_date),
@@ -470,8 +476,8 @@ STAGE PLANS:
             alias: date_udf_flight_orc
             Statistics: Num rows: 137 Data size: 13152 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              expressions: to_unix_timestamp(fl_date) (type: bigint), year(fl_date) (type: int), month(fl_date) (type: int), day(fl_date) (type: int), dayofmonth(fl_date) (type: int), weekofyear(fl_date) (type: int), fl_date (type: date), to_date(fl_date) (type: date), date_add(fl_date, 2) (type: date), date_sub(fl_date, 2) (type: date), datediff(fl_date, '2000-01-01') (type: int)
-              outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
+              expressions: to_unix_timestamp(fl_date) (type: bigint), year(fl_date) (type: int), month(fl_date) (type: int), day(fl_date) (type: int), dayofmonth(fl_date) (type: int), dayofweek(fl_date) (type: int), weekofyear(fl_date) (type: int), fl_date (type: date), to_date(fl_date) (type: date), date_add(fl_date, 2) (type: date), date_sub(fl_date, 2) (type: date), datediff(fl_date, '2000-01-01') (type: int)
+              outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
               Statistics: Num rows: 137 Data size: 13152 Basic stats: COMPLETE Column stats: NONE
               File Output Operator
                 compressed: false
@@ -494,6 +500,7 @@ PREHOOK: query: SELECT
   month(fl_date),
   day(fl_date),
   dayofmonth(fl_date),
+  dayofweek(fl_date),
   weekofyear(fl_date),
   date(fl_date),
   to_date(fl_date),
@@ -510,6 +517,7 @@ POSTHOOK: query: SELECT
   month(fl_date),
   day(fl_date),
   dayofmonth(fl_date),
+  dayofweek(fl_date),
   weekofyear(fl_date),
   date(fl_date),
   to_date(fl_date),
@@ -520,148 +528,149 @@ FROM date_udf_flight_orc
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@date_udf_flight_orc
 #### A masked pattern was here ####
-1287558000	2010	10	20	20	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
-1287558000	2010	10	20	20	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
-1287558000	2010	10	20	20	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
-1287558000	2010	10	20	20	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
-1287558000	2010	10	20	20	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
-1287558000	2010	10	20	20	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
-1287558000	2010	10	20	20	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
-1287558000	2010	10	20	20	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
-1287558000	2010	10	20	20	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
-1287558000	2010	10	20	20	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
-1287558000	2010	10	20	20	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
-1287644400	2010	10	21	21	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
-1287644400	2010	10	21	21	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
-1287644400	2010	10	21	21	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
-1287644400	2010	10	21	21	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
-1287644400	2010	10	21	21	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
-1287644400	2010	10	21	21	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
-1287644400	2010	10	21	21	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
-1287644400	2010	10	21	21	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
-1287644400	2010	10	21	21	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
-1287644400	2010	10	21	21	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
-1287644400	2010	10	21	21	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
-1287644400	2010	10	21	21	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
-1287730800	2010	10	22	22	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
-1287730800	2010	10	22	22	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
-1287730800	2010	10	22	22	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
-1287730800	2010	10	22	22	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
-1287730800	2010	10	22	22	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
-1287730800	2010	10	22	22	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
-1287730800	2010	10	22	22	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
-1287730800	2010	10	22	22	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
-1287730800	2010	10	22	22	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
-1287730800	2010	10	22	22	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
-1287730800	2010	10	22	22	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
-1287817200	2010	10	23	23	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
-1287817200	2010	10	23	23	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
-1287817200	2010	10	23	23	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
-1287817200	2010	10	23	23	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
-1287817200	2010	10	23	23	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
-1287817200	2010	10	23	23	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
-1287817200	2010	10	23	23	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
-1287817200	2010	10	23	23	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
-1287817200	2010	10	23	23	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
-1287817200	2010	10	23	23	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
-1287817200	2010	10	23	23	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
-1287817200	2010	10	23	23	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
-1287903600	2010	10	24	24	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
-1287903600	2010	10	24	24	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
-1287903600	2010	10	24	24	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
-1287903600	2010	10	24	24	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
-1287903600	2010	10	24	24	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
-1287903600	2010	10	24	24	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
-1287903600	2010	10	24	24	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
-1287903600	2010	10	24	24	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
-1287903600	2010	10	24	24	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
-1287903600	2010	10	24	24	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
-1287903600	2010	10	24	24	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
-1287903600	2010	10	24	24	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
-1287990000	2010	10	25	25	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
-1287990000	2010	10	25	25	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
-1287990000	2010	10	25	25	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
-1287990000	2010	10	25	25	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
-1287990000	2010	10	25	25	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
-1287990000	2010	10	25	25	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
-1287990000	2010	10	25	25	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
-1287990000	2010	10	25	25	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
-1287990000	2010	10	25	25	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
-1287990000	2010	10	25	25	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
-1287990000	2010	10	25	25	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
-1287990000	2010	10	25	25	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
-1288076400	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288076400	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288076400	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288076400	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288076400	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288076400	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288076400	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288076400	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288076400	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288076400	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288076400	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288076400	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288076400	2010	10	26	26	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
-1288162800	2010	10	27	27	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
-1288162800	2010	10	27	27	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
-1288162800	2010	10	27	27	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
-1288162800	2010	10	27	27	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
-1288162800	2010	10	27	27	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
-1288162800	2010	10	27	27	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
-1288162800	2010	10	27	27	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
-1288162800	2010	10	27	27	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
-1288162800	2010	10	27	27	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
-1288162800	2010	10	27	27	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
-1288162800	2010	10	27	27	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
-1288249200	2010	10	28	28	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
-1288249200	2010	10	28	28	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
-1288249200	2010	10	28	28	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
-1288249200	2010	10	28	28	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
-1288249200	2010	10	28	28	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
-1288249200	2010	10	28	28	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
-1288249200	2010	10	28	28	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
-1288249200	2010	10	28	28	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
-1288249200	2010	10	28	28	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
-1288249200	2010	10	28	28	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
-1288249200	2010	10	28	28	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
-1288249200	2010	10	28	28	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
-1288335600	2010	10	29	29	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
-1288335600	2010	10	29	29	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
-1288335600	2010	10	29	29	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
-1288335600	2010	10	29	29	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
-1288335600	2010	10	29	29	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
-1288335600	2010	10	29	29	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
-1288335600	2010	10	29	29	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
-1288335600	2010	10	29	29	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
-1288335600	2010	10	29	29	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
-1288335600	2010	10	29	29	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
-1288335600	2010	10	29	29	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
-1288335600	2010	10	29	29	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
-1288422000	2010	10	30	30	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
-1288422000	2010	10	30	30	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
-1288422000	2010	10	30	30	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
-1288422000	2010	10	30	30	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
-1288422000	2010	10	30	30	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
-1288422000	2010	10	30	30	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
-1288422000	2010	10	30	30	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
-1288422000	2010	10	30	30	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
-1288422000	2010	10	30	30	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
-1288422000	2010	10	30	30	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
-1288422000	2010	10	30	30	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
-1288508400	2010	10	31	31	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
-1288508400	2010	10	31	31	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
-1288508400	2010	10	31	31	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
-1288508400	2010	10	31	31	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
-1288508400	2010	10	31	31	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
-1288508400	2010	10	31	31	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
-1288508400	2010	10	31	31	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
-1288508400	2010	10	31	31	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
+1287558000	2010	10	20	20	4	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
+1287558000	2010	10	20	20	4	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
+1287558000	2010	10	20	20	4	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
+1287558000	2010	10	20	20	4	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
+1287558000	2010	10	20	20	4	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
+1287558000	2010	10	20	20	4	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
+1287558000	2010	10	20	20	4	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
+1287558000	2010	10	20	20	4	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
+1287558000	2010	10	20	20	4	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
+1287558000	2010	10	20	20	4	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
+1287558000	2010	10	20	20	4	42	2010-10-20	2010-10-20	2010-10-22	2010-10-18	3945
+1287644400	2010	10	21	21	5	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
+1287644400	2010	10	21	21	5	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
+1287644400	2010	10	21	21	5	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
+1287644400	2010	10	21	21	5	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
+1287644400	2010	10	21	21	5	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
+1287644400	2010	10	21	21	5	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
+1287644400	2010	10	21	21	5	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
+1287644400	2010	10	21	21	5	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
+1287644400	2010	10	21	21	5	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
+1287644400	2010	10	21	21	5	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
+1287644400	2010	10	21	21	5	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
+1287644400	2010	10	21	21	5	42	2010-10-21	2010-10-21	2010-10-23	2010-10-19	3946
+1287730800	2010	10	22	22	6	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
+1287730800	2010	10	22	22	6	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
+1287730800	2010	10	22	22	6	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
+1287730800	2010	10	22	22	6	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
+1287730800	2010	10	22	22	6	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
+1287730800	2010	10	22	22	6	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
+1287730800	2010	10	22	22	6	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
+1287730800	2010	10	22	22	6	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
+1287730800	2010	10	22	22	6	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
+1287730800	2010	10	22	22	6	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
+1287730800	2010	10	22	22	6	42	2010-10-22	2010-10-22	2010-10-24	2010-10-20	3947
+1287817200	2010	10	23	23	7	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
+1287817200	2010	10	23	23	7	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
+1287817200	2010	10	23	23	7	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
+1287817200	2010	10	23	23	7	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
+1287817200	2010	10	23	23	7	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
+1287817200	2010	10	23	23	7	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
+1287817200	2010	10	23	23	7	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
+1287817200	2010	10	23	23	7	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
+1287817200	2010	10	23	23	7	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
+1287817200	2010	10	23	23	7	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
+1287817200	2010	10	23	23	7	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
+1287817200	2010	10	23	23	7	42	2010-10-23	2010-10-23	2010-10-25	2010-10-21	3948
+1287903600	2010	10	24	24	1	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
+1287903600	2010	10	24	24	1	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
+1287903600	2010	10	24	24	1	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
+1287903600	2010	10	24	24	1	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
+1287903600	2010	10	24	24	1	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
+1287903600	2010	10	24	24	1	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
+1287903600	2010	10	24	24	1	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
+1287903600	2010	10	24	24	1	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
+1287903600	2010	10	24	24	1	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
+1287903600	2010	10	24	24	1	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
+1287903600	2010	10	24	24	1	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
+1287903600	2010	10	24	24	1	42	2010-10-24	2010-10-24	2010-10-26	2010-10-22	3949
+1287990000	2010	10	25	25	2	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
+1287990000	2010	10	25	25	2	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
+1287990000	2010	10	25	25	2	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
+1287990000	2010	10	25	25	2	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
+1287990000	2010	10	25	25	2	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
+1287990000	2010	10	25	25	2	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
+1287990000	2010	10	25	25	2	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
+1287990000	2010	10	25	25	2	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
+1287990000	2010	10	25	25	2	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
+1287990000	2010	10	25	25	2	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
+1287990000	2010	10	25	25	2	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
+1287990000	2010	10	25	25	2	43	2010-10-25	2010-10-25	2010-10-27	2010-10-23	3950
+1288076400	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288076400	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288076400	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288076400	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288076400	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288076400	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288076400	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288076400	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288076400	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288076400	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288076400	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288076400	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288076400	2010	10	26	26	3	43	2010-10-26	2010-10-26	2010-10-28	2010-10-24	3951
+1288162800	2010	10	27	27	4	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
+1288162800	2010	10	27	27	4	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
+1288162800	2010	10	27	27	4	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
+1288162800	2010	10	27	27	4	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
+1288162800	2010	10	27	27	4	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
+1288162800	2010	10	27	27	4	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
+1288162800	2010	10	27	27	4	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
+1288162800	2010	10	27	27	4	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
+1288162800	2010	10	27	27	4	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
+1288162800	2010	10	27	27	4	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
+1288162800	2010	10	27	27	4	43	2010-10-27	2010-10-27	2010-10-29	2010-10-25	3952
+1288249200	2010	10	28	28	5	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
+1288249200	2010	10	28	28	5	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
+1288249200	2010	10	28	28	5	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
+1288249200	2010	10	28	28	5	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
+1288249200	2010	10	28	28	5	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
+1288249200	2010	10	28	28	5	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
+1288249200	2010	10	28	28	5	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
+1288249200	2010	10	28	28	5	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
+1288249200	2010	10	28	28	5	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
+1288249200	2010	10	28	28	5	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
+1288249200	2010	10	28	28	5	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
+1288249200	2010	10	28	28	5	43	2010-10-28	2010-10-28	2010-10-30	2010-10-26	3953
+1288335600	2010	10	29	29	6	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
+1288335600	2010	10	29	29	6	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
+1288335600	2010	10	29	29	6	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
+1288335600	2010	10	29	29	6	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
+1288335600	2010	10	29	29	6	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
+1288335600	2010	10	29	29	6	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
+1288335600	2010	10	29	29	6	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
+1288335600	2010	10	29	29	6	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
+1288335600	2010	10	29	29	6	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
+1288335600	2010	10	29	29	6	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
+1288335600	2010	10	29	29	6	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
+1288335600	2010	10	29	29	6	43	2010-10-29	2010-10-29	2010-10-31	2010-10-27	3954
+1288422000	2010	10	30	30	7	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
+1288422000	2010	10	30	30	7	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
+1288422000	2010	10	30	30	7	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
+1288422000	2010	10	30	30	7	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
+1288422000	2010	10	30	30	7	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
+1288422000	2010	10	30	30	7	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
+1288422000	2010	10	30	30	7	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
+1288422000	2010	10	30	30	7	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
+1288422000	2010	10	30	30	7	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
+1288422000	2010	10	30	30	7	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
+1288422000	2010	10	30	30	7	43	2010-10-30	2010-10-30	2010-11-01	2010-10-28	3955
+1288508400	2010	10	31	31	1	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
+1288508400	2010	10	31	31	1	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
+1288508400	2010	10	31	31	1	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
+1288508400	2010	10	31	31	1	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
+1288508400	2010	10	31	31	1	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
+1288508400	2010	10	31	31	1	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
+1288508400	2010	10	31	31	1	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
+1288508400	2010	10	31	31	1	43	2010-10-31	2010-10-31	2010-11-02	2010-10-29	3956
 PREHOOK: query: EXPLAIN SELECT
   year(fl_time) = year(fl_date),
   month(fl_time) = month(fl_date),
   day(fl_time) = day(fl_date),
   dayofmonth(fl_time) = dayofmonth(fl_date),
+  dayofweek(fl_time) = dayofweek(fl_date),
   weekofyear(fl_time) = weekofyear(fl_date),
   date(fl_time) = date(fl_date),
   to_date(fl_time) = to_date(fl_date),
@@ -675,6 +684,7 @@ POSTHOOK: query: EXPLAIN SELECT
   month(fl_time) = month(fl_date),
   day(fl_time) = day(fl_date),
   dayofmonth(fl_time) = dayofmonth(fl_date),
+  dayofweek(fl_time) = dayofweek(fl_date),
   weekofyear(fl_time) = weekofyear(fl_date),
   date(fl_time) = date(fl_date),
   to_date(fl_time) = to_date(fl_date),
@@ -695,8 +705,8 @@ STAGE PLANS:
             alias: date_udf_flight_orc
             Statistics: Num rows: 137 Data size: 13152 Basic stats: COMPLETE Column stats: NONE
             Select Operator
-              expressions: (year(fl_time) = year(fl_date)) (type: boolean), (month(fl_time) = month(fl_date)) (type: boolean), (day(fl_time) = day(fl_date)) (type: boolean), (dayofmonth(fl_time) = dayofmonth(fl_date)) (type: boolean), (weekofyear(fl_time) = weekofyear(fl_date)) (type: boolean), (CAST( fl_time AS DATE) = fl_date) (type: boolean), (to_date(fl_time) = to_date(fl_date)) (type: boolean), (date_add(fl_time, 2) = date_add(fl_date, 2)) (type: boolean), (date_sub(fl_time, 2) = date_sub(fl_date, 2)) (type: boolean), (datediff(fl_time, '2000-01-01') = datediff(fl_date, '2000-01-01')) (type: boolean)
-              outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
+              expressions: (year(fl_time) = year(fl_date)) (type: boolean), (month(fl_time) = month(fl_date)) (type: boolean), (day(fl_time) = day(fl_date)) (type: boolean), (dayofmonth(fl_time) = dayofmonth(fl_date)) (type: boolean), (dayofweek(fl_time) = dayofweek(fl_date)) (type: boolean), (weekofyear(fl_time) = weekofyear(fl_date)) (type: boolean), (CAST( fl_time AS DATE) = fl_date) (type: boolean), (to_date(fl_time) = to_date(fl_date)) (type: boolean), (date_add(fl_time, 2) = date_add(fl_date, 2)) (type: boolean), (date_sub(fl_time, 2) = date_sub(fl_date, 2)) (type: boolean), (datediff(fl_time, '2000-01-01') = datediff(fl_date, '2000-01-01')) (type: boolean)
+              outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
               Statistics: Num rows: 137 Data size: 13152 Basic stats: COMPLETE Column stats: NONE
               File Output Operator
                 compressed: false
@@ -719,6 +729,7 @@ SELECT
   month(fl_time) = month(fl_date),
   day(fl_time) = day(fl_date),
   dayofmonth(fl_time) = dayofmonth(fl_date),
+  dayofweek(fl_time) = dayofweek(fl_date),
   weekofyear(fl_time) = weekofyear(fl_date),
   date(fl_time) = date(fl_date),
   to_date(fl_time) = to_date(fl_date),
@@ -735,6 +746,7 @@ SELECT
   month(fl_time) = month(fl_date),
   day(fl_time) = day(fl_date),
   dayofmonth(fl_time) = dayofmonth(fl_date),
+  dayofweek(fl_time) = dayofweek(fl_date),
   weekofyear(fl_time) = weekofyear(fl_date),
   date(fl_time) = date(fl_date),
   to_date(fl_time) = to_date(fl_date),
@@ -745,143 +757,143 @@ FROM date_udf_flight_orc
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@date_udf_flight_orc
 #### A masked pattern was here ####
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
-true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
+true	true	true	true	true	true	true	true	true	true	true
 PREHOOK: query: EXPLAIN SELECT 
   fl_date, 
   to_date(date_add(fl_date, 2)), 


[13/22] hive git commit: HIVE-14146: Column comments with "\n" character "corrupts" table metadata (Peter Vary, reviewed by Aihua Xu)

Posted by se...@apache.org.
HIVE-14146: Column comments with "\n" character "corrupts" table metadata (Peter Vary, reviewed by Aihua Xu)


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

Branch: refs/heads/hive-14535
Commit: d729b45446dd4eef5a06c71ee4c4be9ee37070d9
Parents: 2435e70
Author: Aihua Xu <ai...@apache.org>
Authored: Mon Oct 10 10:16:43 2016 -0400
Committer: Aihua Xu <ai...@apache.org>
Committed: Mon Oct 10 10:16:43 2016 -0400

----------------------------------------------------------------------
 .../hive/common/util/HiveStringUtils.java       |  23 +-
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |  41 ++--
 .../formatting/MetaDataFormatUtils.java         | 156 +++++++++++---
 .../formatting/TextMetaDataFormatter.java       |   5 +-
 .../queries/clientpositive/escape_comments.q    |  20 ++
 .../alter_view_as_select_with_partition.q.out   |  10 +-
 .../alter_table_invalidate_column_stats.q.out   |  24 +--
 .../clientpositive/alter_view_as_select.q.out   |  14 +-
 .../columnstats_part_coltype.q.out              |  16 +-
 .../results/clientpositive/create_like.q.out    |  46 +++-
 .../results/clientpositive/create_view.q.out    |  72 +++----
 .../create_view_partitioned.q.out               |  20 +-
 .../clientpositive/create_view_translate.q.out  |   8 +-
 .../create_with_constraints.q.out               |  36 ++--
 .../describe_comment_indent.q.out               |  15 +-
 .../clientpositive/escape_comments.q.out        | 213 +++++++++++++++++++
 16 files changed, 558 insertions(+), 161 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/d729b454/common/src/java/org/apache/hive/common/util/HiveStringUtils.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hive/common/util/HiveStringUtils.java b/common/src/java/org/apache/hive/common/util/HiveStringUtils.java
index 72c3fa9..507e369 100644
--- a/common/src/java/org/apache/hive/common/util/HiveStringUtils.java
+++ b/common/src/java/org/apache/hive/common/util/HiveStringUtils.java
@@ -77,6 +77,15 @@ public class HiveStringUtils {
       }).with(
         new LookupTranslator(EntityArrays.JAVA_CTRL_CHARS_ESCAPE()));
 
+  private static final CharSequenceTranslator ESCAPE_HIVE_COMMAND =
+      new LookupTranslator(
+        new String[][] {
+          {"'", "\\'"},
+          {";", "\\;"},
+          {"\\", "\\\\"},
+      }).with(
+        new LookupTranslator(EntityArrays.JAVA_CTRL_CHARS_ESCAPE()));
+
   /**
    * Maintain a String pool to reduce memory.
    */
@@ -622,7 +631,19 @@ public class HiveStringUtils {
    */
   public static String escapeJava(String str) {
     return ESCAPE_JAVA.translate(str);
-}
+  }
+
+  /**
+   * Escape non-unicode characters, and ', and ;
+   * Like StringEscapeUtil.escapeJava() will escape
+   * unicode characters as well but in some cases it's not desired.
+   *
+   * @param str Original string
+   * @return Escaped string
+   */
+  public static String escapeHiveCommand(String str) {
+    return ESCAPE_HIVE_COMMAND.translate(str);
+  }
 
   /**
    * Unescape commas in the string using the default escape char

http://git-wip-us.apache.org/repos/asf/hive/blob/d729b454/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 cea8ce8..362ea32 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
@@ -50,7 +50,6 @@ import java.util.SortedSet;
 import java.util.TreeMap;
 import java.util.TreeSet;
 
-import org.apache.commons.lang.StringEscapeUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
@@ -232,6 +231,7 @@ import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.tools.HadoopArchives;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.hive.common.util.AnnotationUtils;
+import org.apache.hive.common.util.HiveStringUtils;
 import org.apache.hive.common.util.ReflectionUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -2052,7 +2052,8 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
     createDb_str.append("CREATE DATABASE `").append(database.getName()).append("`\n");
     if (database.getDescription() != null) {
       createDb_str.append("COMMENT\n  '");
-      createDb_str.append(escapeHiveCommand(database.getDescription())).append("'\n");
+      createDb_str.append(
+          HiveStringUtils.escapeHiveCommand(database.getDescription())).append("'\n");
     }
     createDb_str.append("LOCATION\n  '");
     createDb_str.append(database.getLocationUri()).append("'\n");
@@ -2150,7 +2151,8 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
       for (FieldSchema col : cols) {
         String columnDesc = "  `" + col.getName() + "` " + col.getType();
         if (col.getComment() != null) {
-          columnDesc = columnDesc + " COMMENT '" + escapeHiveCommand(col.getComment()) + "'";
+          columnDesc = columnDesc + " COMMENT '"
+              + HiveStringUtils.escapeHiveCommand(col.getComment()) + "'";
         }
         columns.add(columnDesc);
       }
@@ -2161,7 +2163,8 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
       String tabComment = tbl.getProperty("comment");
       if (tabComment != null) {
         duplicateProps.add("comment");
-        tbl_comment = "COMMENT '" + escapeHiveCommand(tabComment) + "'";
+        tbl_comment = "COMMENT '"
+            + HiveStringUtils.escapeHiveCommand(tabComment) + "'";
       }
 
       // Partitions
@@ -2173,8 +2176,8 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
         for (FieldSchema partKey : partKeys) {
           String partColDesc = "  `" + partKey.getName() + "` " + partKey.getType();
           if (partKey.getComment() != null) {
-            partColDesc = partColDesc + " COMMENT '" +
-                escapeHiveCommand(partKey.getComment()) + "'";
+            partColDesc = partColDesc + " COMMENT '"
+                + HiveStringUtils.escapeHiveCommand(partKey.getComment()) + "'";
           }
           partCols.add(partColDesc);
         }
@@ -2217,7 +2220,8 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
       SerDeInfo serdeInfo = sd.getSerdeInfo();
       Map<String, String> serdeParams = serdeInfo.getParameters();
       tbl_row_format.append("ROW FORMAT SERDE \n");
-      tbl_row_format.append("  '" + escapeHiveCommand(serdeInfo.getSerializationLib()) + "' \n");
+      tbl_row_format.append("  '"
+          + HiveStringUtils.escapeHiveCommand(serdeInfo.getSerializationLib()) + "' \n");
       if (tbl.getStorageHandler() == null) {
         // If serialization.format property has the default value, it will not to be included in
         // SERDE properties
@@ -2228,20 +2232,21 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
         if (!serdeParams.isEmpty()) {
           appendSerdeParams(tbl_row_format, serdeParams).append(" \n");
         }
-        tbl_row_format.append("STORED AS INPUTFORMAT \n  '" +
-            escapeHiveCommand(sd.getInputFormat()) + "' \n");
-        tbl_row_format.append("OUTPUTFORMAT \n  '" +
-            escapeHiveCommand(sd.getOutputFormat()) + "'");
+        tbl_row_format.append("STORED AS INPUTFORMAT \n  '"
+            + HiveStringUtils.escapeHiveCommand(sd.getInputFormat()) + "' \n");
+        tbl_row_format.append("OUTPUTFORMAT \n  '"
+            + HiveStringUtils.escapeHiveCommand(sd.getOutputFormat()) + "'");
       } else {
         duplicateProps.add(META_TABLE_STORAGE);
-        tbl_row_format.append("STORED BY \n  '" + escapeHiveCommand(tbl.getParameters().get(
+        tbl_row_format.append("STORED BY \n  '"
+            + HiveStringUtils.escapeHiveCommand(tbl.getParameters().get(
             META_TABLE_STORAGE)) + "' \n");
         // SerDe Properties
         if (!serdeParams.isEmpty()) {
           appendSerdeParams(tbl_row_format, serdeInfo.getParameters());
         }
       }
-      String tbl_location = "  '" + escapeHiveCommand(sd.getLocation()) + "'";
+      String tbl_location = "  '" + HiveStringUtils.escapeHiveCommand(sd.getLocation()) + "'";
 
       // Table properties
       duplicateProps.addAll(Arrays.asList(StatsSetupConst.TABLE_PARAMS_STATS_KEYS));
@@ -2277,7 +2282,7 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
       for (String key : properties.keySet()) {
         if (properties.get(key) != null && (exclude == null || !exclude.contains(key))) {
           realProps.add("  '" + key + "'='" +
-              escapeHiveCommand(StringEscapeUtils.escapeJava(properties.get(key))) + "'");
+              HiveStringUtils.escapeHiveCommand(properties.get(key)) + "'");
         }
       }
       prop_string += StringUtils.join(realProps, ", \n");
@@ -2291,7 +2296,7 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
     List<String> serdeCols = new ArrayList<String>();
     for (Entry<String, String> entry : serdeParam.entrySet()) {
       serdeCols.add("  '" + entry.getKey() + "'='"
-          + escapeHiveCommand(StringEscapeUtils.escapeJava(entry.getValue())) + "'");
+          + HiveStringUtils.escapeHiveCommand(entry.getValue()) + "'");
     }
     builder.append(StringUtils.join(serdeCols, ", \n")).append(')');
     return builder;
@@ -2318,6 +2323,10 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
 
     indexes = db.getIndexes(tbl.getDbName(), tbl.getTableName(), (short) -1);
 
+    // In case the query is served by HiveServer2, don't pad it with spaces,
+    // as HiveServer2 output is consumed by JDBC/ODBC clients.
+    boolean isOutputPadded = !SessionState.get().isHiveServerQuery();
+
     // write the results in the file
     DataOutputStream outStream = getOutputStream(showIndexes.getResFile());
     try {
@@ -2330,7 +2339,7 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
 
       for (Index index : indexes)
       {
-        outStream.write(MetaDataFormatUtils.getAllColumnsInformation(index).getBytes(StandardCharsets.UTF_8));
+        outStream.write(MetaDataFormatUtils.getIndexInformation(index, isOutputPadded).getBytes(StandardCharsets.UTF_8));
       }
     } catch (FileNotFoundException e) {
       LOG.info("show indexes: " + stringifyException(e));

http://git-wip-us.apache.org/repos/asf/hive/blob/d729b454/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatUtils.java
index ba4f6a7..c850e43 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/MetaDataFormatUtils.java
@@ -111,7 +111,6 @@ public final class MetaDataFormatUtils {
    * @param printHeader - if header should be included
    * @param isOutputPadded - make it more human readable by setting indentation
    *        with spaces. Turned off for use by HiveServer2
-   * @param showParColsSep - show partition column separator
    * @return string with formatted column information
    */
   public static String getAllColumnsInformation(List<FieldSchema> cols,
@@ -233,11 +232,11 @@ public final class MetaDataFormatUtils {
         appendColumnStatsNoFormatting(colBuffer, "", "", "", "", "", "", "", "");
       }
     }
-    colBuffer.append(comment == null ? "" : comment);
+    colBuffer.append(comment == null ? "" : HiveStringUtils.escapeJava(comment));
     colBuffer.append(LINE_DELIM);
   }
 
-  public static String getAllColumnsInformation(Index index) {
+  public static String getIndexInformation(Index index, boolean isOutputPadded) {
     StringBuilder indexInfo = new StringBuilder(DEFAULT_STRINGBUILDER_SIZE);
 
     List<String> indexColumns = new ArrayList<String>();
@@ -268,9 +267,10 @@ public final class MetaDataFormatUtils {
     IndexType indexType = HiveIndex.getIndexTypeByClassName(indexHandlerClass);
     indexColumns.add(indexType.getName());
 
-    indexColumns.add(index.getParameters().get("comment"));
+    String comment = index.getParameters().get("comment");
+    indexColumns.add(comment == null ? null : HiveStringUtils.escapeJava(comment));
 
-    formatOutput(indexColumns.toArray(new String[0]), indexInfo);
+    formatOutput(indexColumns.toArray(new String[0]), indexInfo, isOutputPadded);
 
     return indexInfo.toString();
   }
@@ -354,12 +354,12 @@ public final class MetaDataFormatUtils {
     return tableInfo.toString();
   }
 
-  public static String getTableInformation(Table table) {
+  public static String getTableInformation(Table table, boolean isOutputPadded) {
     StringBuilder tableInfo = new StringBuilder(DEFAULT_STRINGBUILDER_SIZE);
 
     // Table Metadata
     tableInfo.append(LINE_DELIM).append("# Detailed Table Information").append(LINE_DELIM);
-    getTableMetaDataInformation(tableInfo, table);
+    getTableMetaDataInformation(tableInfo, table, isOutputPadded);
 
     // Storage information.
     tableInfo.append(LINE_DELIM).append("# Storage Information").append(LINE_DELIM);
@@ -428,7 +428,8 @@ public final class MetaDataFormatUtils {
     }
   }
 
-  private static void getTableMetaDataInformation(StringBuilder tableInfo, Table  tbl) {
+  private static void getTableMetaDataInformation(StringBuilder tableInfo, Table  tbl,
+      boolean isOutputPadded) {
     formatOutput("Database:", tbl.getDbName(), tableInfo);
     formatOutput("Owner:", tbl.getOwner(), tableInfo);
     formatOutput("CreateTime:", formatDate(tbl.getTTable().getCreateTime()), tableInfo);
@@ -441,7 +442,7 @@ public final class MetaDataFormatUtils {
 
     if (tbl.getParameters().size() > 0) {
       tableInfo.append("Table Parameters:").append(LINE_DELIM);
-      displayAllParameters(tbl.getParameters(), tableInfo, false);
+      displayAllParameters(tbl.getParameters(), tableInfo, false, isOutputPadded);
     }
   }
 
@@ -465,7 +466,7 @@ public final class MetaDataFormatUtils {
    * including unicode.
    */
   private static void displayAllParameters(Map<String, String> params, StringBuilder tableInfo) {
-    displayAllParameters(params, tableInfo, true);
+    displayAllParameters(params, tableInfo, true, false);
   }
 
   /**
@@ -473,15 +474,16 @@ public final class MetaDataFormatUtils {
    * including unicode if escapeUnicode is true; otherwise the characters other
    * than unicode will be escaped.
    */
-
-  private static void displayAllParameters(Map<String, String> params, StringBuilder tableInfo, boolean escapeUnicode) {
+  private static void displayAllParameters(Map<String, String> params, StringBuilder tableInfo,
+      boolean escapeUnicode, boolean isOutputPadded) {
     List<String> keys = new ArrayList<String>(params.keySet());
     Collections.sort(keys);
     for (String key : keys) {
       tableInfo.append(FIELD_DELIM); // Ensures all params are indented.
       formatOutput(key,
-          escapeUnicode ? StringEscapeUtils.escapeJava(params.get(key)) : HiveStringUtils.escapeJava(params.get(key)),
-          tableInfo);
+          escapeUnicode ? StringEscapeUtils.escapeJava(params.get(key))
+              : HiveStringUtils.escapeJava(params.get(key)),
+          tableInfo, isOutputPadded);
     }
   }
 
@@ -550,21 +552,74 @@ public final class MetaDataFormatUtils {
     return "UNKNOWN";
   }
 
-  private static void formatOutput(String[] fields, StringBuilder tableInfo) {
-    for (String field : fields) {
-      if (field == null) {
-        tableInfo.append(FIELD_DELIM);
-        continue;
+  /**
+   * Prints a row with the given fields into the builder
+   * The last field could be a multiline field, and the extra lines should be padded
+   * @param fields The fields to print
+   * @param tableInfo The target builder
+   * @param isLastLinePadded Is the last field could be printed in multiple lines, if contains
+   *                         newlines?
+   */
+  private static void formatOutput(String[] fields, StringBuilder tableInfo,
+      boolean isLastLinePadded) {
+    int[] paddings = new int[fields.length-1];
+    if (fields.length > 1) {
+      for (int i = 0; i < fields.length - 1; i++) {
+        if (fields[i] == null) {
+          tableInfo.append(FIELD_DELIM);
+          continue;
+        }
+        tableInfo.append(String.format("%-" + ALIGNMENT + "s", fields[i])).append(FIELD_DELIM);
+        paddings[i] = ALIGNMENT > fields[i].length() ? ALIGNMENT : fields[i].length();
       }
-      tableInfo.append(String.format("%-" + ALIGNMENT + "s", field)).append(FIELD_DELIM);
     }
-    tableInfo.append(LINE_DELIM);
+    if (fields.length > 0) {
+      String value = fields[fields.length-1];
+      String unescapedValue =
+          (isLastLinePadded && value != null) ? value.replaceAll("\\\\n|\\\\r|\\\\r\\\\n","\n")
+              :value;
+      indentMultilineValue(unescapedValue, tableInfo, paddings, false);
+    } else {
+      tableInfo.append(LINE_DELIM);
+    }
+  }
+
+  /**
+   * Prints a row the given fields to a formatted line
+   * @param fields The fields to print
+   * @param tableInfo The target builder
+   */
+  private static void formatOutput(String[] fields, StringBuilder tableInfo) {
+    formatOutput(fields, tableInfo, false);
   }
 
-  private static void formatOutput(String name, String value,
-      StringBuilder tableInfo) {
+  /**
+   * Prints the name value pair, and if the value contains newlines, it add one more empty field
+   * before the two values (Assumes, the name value pair is already indented with it)
+   * @param name The field name to print
+   * @param value The value to print - might contain newlines
+   * @param tableInfo The target builder
+   */
+  private static void formatOutput(String name, String value, StringBuilder tableInfo) {
     tableInfo.append(String.format("%-" + ALIGNMENT + "s", name)).append(FIELD_DELIM);
-    tableInfo.append(String.format("%-" + ALIGNMENT + "s", value)).append(LINE_DELIM);
+    int colNameLength = ALIGNMENT > name.length() ? ALIGNMENT : name.length();
+    indentMultilineValue(value, tableInfo, new int[] {0, colNameLength}, true);
+  }
+
+  /**
+   * Prints the name value pair
+   * It the output is padded then unescape the value, so it could be printed in multiple lines.
+   * In this case it assumes the pair is already indented with a field delimiter
+   * @param name The field name to print
+   * @param value The value t print
+   * @param tableInfo The target builder
+   * @param isOutputPadded Should the value printed as a padded string?
+   */
+  private static void formatOutput(String name, String value, StringBuilder tableInfo,
+      boolean isOutputPadded) {
+    String unescapedValue =
+        (isOutputPadded && value != null) ? value.replaceAll("\\\\n|\\\\r|\\\\r\\\\n","\n"):value;
+    formatOutput(name, unescapedValue, tableInfo);
   }
 
   private static void formatWithIndentation(String colName, String colType, String colComment,
@@ -613,17 +668,51 @@ public final class MetaDataFormatUtils {
       }
     }
 
-    // comment indent processing for multi-line comments
-    // comments should be indented the same amount on each line
-    // if the first line comment starts indented by k,
-    // the following line comments should also be indented by k
-    String[] commentSegments = colComment.split("\n|\r|\r\n");
-    tableInfo.append(String.format("%-" + ALIGNMENT + "s", commentSegments[0])).append(LINE_DELIM);
     int colNameLength = ALIGNMENT > colName.length() ? ALIGNMENT : colName.length();
     int colTypeLength = ALIGNMENT > colType.length() ? ALIGNMENT : colType.length();
-    for (int i = 1; i < commentSegments.length; i++) {
-      tableInfo.append(String.format("%" + colNameLength + "s" + FIELD_DELIM + "%"
-          + colTypeLength + "s" + FIELD_DELIM + "%s", "", "", commentSegments[i])).append(LINE_DELIM);
+    indentMultilineValue(colComment, tableInfo, new int[]{colNameLength, colTypeLength}, false);
+  }
+
+  /**
+   * comment indent processing for multi-line values
+   * values should be indented the same amount on each line
+   * if the first line comment starts indented by k,
+   * the following line comments should also be indented by k
+   * @param value the value to write
+   * @param tableInfo the buffer to write to
+   * @param columnWidths the widths of the previous columns
+   * @param printNull print null as a string, or do not print anything
+   */
+  private static void indentMultilineValue(String value, StringBuilder tableInfo,
+      int[] columnWidths, boolean printNull) {
+    if (value==null) {
+      if (printNull) {
+        tableInfo.append(String.format("%-" + ALIGNMENT + "s", value));
+      }
+      tableInfo.append(LINE_DELIM);
+    } else {
+      String[] valueSegments = value.split("\n|\r|\r\n");
+      tableInfo.append(String.format("%-" + ALIGNMENT + "s", valueSegments[0])).append(LINE_DELIM);
+      for (int i = 1; i < valueSegments.length; i++) {
+        printPadding(tableInfo, columnWidths);
+        tableInfo.append(String.format("%-" + ALIGNMENT + "s", valueSegments[i]))
+            .append(LINE_DELIM);
+      }
+    }
+  }
+
+  /**
+   * Print the rigth padding, with the given column widths
+   * @param tableInfo The buffer to write to
+   * @param columnWidths The column widths
+   */
+  private static void printPadding(StringBuilder tableInfo, int[] columnWidths) {
+    for (int columnWidth : columnWidths) {
+      if (columnWidth == 0) {
+        tableInfo.append(FIELD_DELIM);
+      } else {
+        tableInfo.append(String.format("%" + columnWidth + "s" + FIELD_DELIM, ""));
+      }
     }
   }
 
@@ -665,6 +754,7 @@ public final class MetaDataFormatUtils {
     formatOutput(ShowIndexesDesc.getSchema().split("#")[0].split(","), indexCols);
     return indexCols.toString();
   }
+
   public static MetaDataFormatter getFormatter(HiveConf conf) {
     if ("json".equals(conf.get(HiveConf.ConfVars.HIVE_DDL_OUTPUT_FORMAT.varname, "text"))) {
       return new JsonMetaDataFormatter();

http://git-wip-us.apache.org/repos/asf/hive/blob/d729b454/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java
index 47d67b1..b990bda 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/formatting/TextMetaDataFormatter.java
@@ -27,6 +27,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.hive.common.util.HiveStringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.fs.FileStatus;
@@ -139,7 +140,7 @@ class TextMetaDataFormatter implements MetaDataFormatter {
           if (part != null) {
             output = MetaDataFormatUtils.getPartitionInformation(part);
           } else {
-            output = MetaDataFormatUtils.getTableInformation(tbl);
+            output = MetaDataFormatUtils.getTableInformation(tbl, isOutputPadded);
           }
           outStream.write(output.getBytes("UTF-8"));
 
@@ -460,7 +461,7 @@ class TextMetaDataFormatter implements MetaDataFormatter {
       outStream.write(database.getBytes("UTF-8"));
       outStream.write(separator);
       if (comment != null) {
-        outStream.write(comment.getBytes("UTF-8"));
+        outStream.write(HiveStringUtils.escapeJava(comment).getBytes("UTF-8"));
       }
       outStream.write(separator);
       if (location != null) {

http://git-wip-us.apache.org/repos/asf/hive/blob/d729b454/ql/src/test/queries/clientpositive/escape_comments.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/escape_comments.q b/ql/src/test/queries/clientpositive/escape_comments.q
new file mode 100644
index 0000000..8c38690
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/escape_comments.q
@@ -0,0 +1,20 @@
+create database escape_comments_db comment 'a\nb';
+use escape_comments_db;
+create table escape_comments_tbl1
+(col1 string comment 'a\nb\'\;') comment 'a\nb'
+partitioned by (p1 string comment 'a\nb');
+create view escape_comments_view1 (col1 comment 'a\nb') comment 'a\nb'
+as select col1 from escape_comments_tbl1;
+create index index2 on table escape_comments_tbl1(col1) as 'COMPACT' with deferred rebuild comment 'a\nb';
+
+describe database extended escape_comments_db;
+describe database escape_comments_db;
+show create table escape_comments_tbl1;
+describe formatted escape_comments_tbl1;
+describe pretty escape_comments_tbl1;
+describe escape_comments_tbl1;
+show create table escape_comments_view1;
+describe formatted escape_comments_view1;
+show formatted index on escape_comments_tbl1;
+
+drop database escape_comments_db cascade;

http://git-wip-us.apache.org/repos/asf/hive/blob/d729b454/ql/src/test/results/clientnegative/alter_view_as_select_with_partition.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/alter_view_as_select_with_partition.q.out b/ql/src/test/results/clientnegative/alter_view_as_select_with_partition.q.out
index 9b84227..4e43819 100644
--- a/ql/src/test/results/clientnegative/alter_view_as_select_with_partition.q.out
+++ b/ql/src/test/results/clientnegative/alter_view_as_select_with_partition.q.out
@@ -63,11 +63,11 @@ Bucket Columns:     	[]
 Sort Columns:       	[]                  	 
 	 	 
 # View Information	 	 
-View Original Text: 	SELECT key, value	 
-FROM src	 	 
-WHERE key=86	 	 
+View Original Text: 	SELECT key, value   	 
+	                    	FROM src            
+	                    	WHERE key=86        
 View Expanded Text: 	SELECT `src`.`key`, `src`.`value`	 
-FROM `default`.`src`	 	 
-WHERE `src`.`key`=86	 	 
+	                    	FROM `default`.`src`
+	                    	WHERE `src`.`key`=86
 FAILED: SemanticException [Error 10217]: Cannot replace a view with CREATE VIEW or REPLACE VIEW or ALTER VIEW AS SELECT if the view has partitions
  The following view has partition, it could not be replaced: default.testViewPart

http://git-wip-us.apache.org/repos/asf/hive/blob/d729b454/ql/src/test/results/clientpositive/alter_table_invalidate_column_stats.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/alter_table_invalidate_column_stats.q.out b/ql/src/test/results/clientpositive/alter_table_invalidate_column_stats.q.out
index f3c10ee..85d7dc4 100644
--- a/ql/src/test/results/clientpositive/alter_table_invalidate_column_stats.q.out
+++ b/ql/src/test/results/clientpositive/alter_table_invalidate_column_stats.q.out
@@ -296,7 +296,7 @@ PREHOOK: Input: statsdb1@testpart1
 POSTHOOK: query: describe formatted statsdb1.testpart1 partition (part = 'part3') col1
 POSTHOOK: type: DESCTABLE
 POSTHOOK: Input: statsdb1@testpart1
-# col_name            	data_type           	comment             		 	 	 	 	 	 	 
+# col_name            	data_type           	comment             	 	 	 	 	 	 	 	 
 	 	 	 	 	 	 	 	 	 	 
 col1                	int                 	from deserializer   	 	 	 	 	 	 	 	 
 PREHOOK: query: describe formatted statsdb1.testpart1 partition (part = 'part3') col2
@@ -305,7 +305,7 @@ PREHOOK: Input: statsdb1@testpart1
 POSTHOOK: query: describe formatted statsdb1.testpart1 partition (part = 'part3') col2
 POSTHOOK: type: DESCTABLE
 POSTHOOK: Input: statsdb1@testpart1
-# col_name            	data_type           	comment             		 	 	 	 	 	 	 
+# col_name            	data_type           	comment             	 	 	 	 	 	 	 	 
 	 	 	 	 	 	 	 	 	 	 
 col2                	string              	from deserializer   	 	 	 	 	 	 	 	 
 PREHOOK: query: describe formatted statsdb1.testpart1 partition (part = 'part3') col3
@@ -314,7 +314,7 @@ PREHOOK: Input: statsdb1@testpart1
 POSTHOOK: query: describe formatted statsdb1.testpart1 partition (part = 'part3') col3
 POSTHOOK: type: DESCTABLE
 POSTHOOK: Input: statsdb1@testpart1
-# col_name            	data_type           	comment             		 	 	 	 	 	 	 
+# col_name            	data_type           	comment             	 	 	 	 	 	 	 	 
 	 	 	 	 	 	 	 	 	 	 
 col3                	string              	from deserializer   	 	 	 	 	 	 	 	 
 PREHOOK: query: alter table statsdb1.testpart1 replace columns (col1 int, col2 string, col4 string) cascade
@@ -353,7 +353,7 @@ PREHOOK: Input: statsdb1@testpart1
 POSTHOOK: query: describe formatted statsdb1.testpart1 partition (part = 'part1') col4
 POSTHOOK: type: DESCTABLE
 POSTHOOK: Input: statsdb1@testpart1
-# col_name            	data_type           	comment             		 	 	 	 	 	 	 
+# col_name            	data_type           	comment             	 	 	 	 	 	 	 	 
 	 	 	 	 	 	 	 	 	 	 
 col4                	string              	from deserializer   	 	 	 	 	 	 	 	 
 PREHOOK: query: alter table statsdb1.testpart1 change column col1 col1 string
@@ -382,7 +382,7 @@ PREHOOK: Input: statsdb1@testpart1
 POSTHOOK: query: describe formatted statsdb1.testpart1 partition (part = 'part1') col1
 POSTHOOK: type: DESCTABLE
 POSTHOOK: Input: statsdb1@testpart1
-# col_name            	data_type           	comment             		 	 	 	 	 	 	 
+# col_name            	data_type           	comment             	 	 	 	 	 	 	 	 
 	 	 	 	 	 	 	 	 	 	 
 col1                	string              	from deserializer   	 	 	 	 	 	 	 	 
 PREHOOK: query: describe formatted statsdb1.testpart1 partition (part = 'part1') col2
@@ -400,7 +400,7 @@ PREHOOK: Input: statsdb1@testpart1
 POSTHOOK: query: describe formatted statsdb1.testpart1 partition (part = 'part1') col4
 POSTHOOK: type: DESCTABLE
 POSTHOOK: Input: statsdb1@testpart1
-# col_name            	data_type           	comment             		 	 	 	 	 	 	 
+# col_name            	data_type           	comment             	 	 	 	 	 	 	 	 
 	 	 	 	 	 	 	 	 	 	 
 col4                	string              	from deserializer   	 	 	 	 	 	 	 	 
 PREHOOK: query: alter table statsdb1.testpart1 rename to statsdb2.testpart2
@@ -762,7 +762,7 @@ PREHOOK: Input: statsdb1@testpart1
 POSTHOOK: query: describe formatted statsdb1.testpart1 partition (part = 'part3') col1
 POSTHOOK: type: DESCTABLE
 POSTHOOK: Input: statsdb1@testpart1
-# col_name            	data_type           	comment             		 	 	 	 	 	 	 
+# col_name            	data_type           	comment             	 	 	 	 	 	 	 	 
 	 	 	 	 	 	 	 	 	 	 
 col1                	int                 	from deserializer   	 	 	 	 	 	 	 	 
 PREHOOK: query: describe formatted statsdb1.testpart1 partition (part = 'part3') col2
@@ -771,7 +771,7 @@ PREHOOK: Input: statsdb1@testpart1
 POSTHOOK: query: describe formatted statsdb1.testpart1 partition (part = 'part3') col2
 POSTHOOK: type: DESCTABLE
 POSTHOOK: Input: statsdb1@testpart1
-# col_name            	data_type           	comment             		 	 	 	 	 	 	 
+# col_name            	data_type           	comment             	 	 	 	 	 	 	 	 
 	 	 	 	 	 	 	 	 	 	 
 col2                	string              	from deserializer   	 	 	 	 	 	 	 	 
 PREHOOK: query: describe formatted statsdb1.testpart1 partition (part = 'part3') col3
@@ -780,7 +780,7 @@ PREHOOK: Input: statsdb1@testpart1
 POSTHOOK: query: describe formatted statsdb1.testpart1 partition (part = 'part3') col3
 POSTHOOK: type: DESCTABLE
 POSTHOOK: Input: statsdb1@testpart1
-# col_name            	data_type           	comment             		 	 	 	 	 	 	 
+# col_name            	data_type           	comment             	 	 	 	 	 	 	 	 
 	 	 	 	 	 	 	 	 	 	 
 col3                	string              	from deserializer   	 	 	 	 	 	 	 	 
 PREHOOK: query: alter table statsdb1.testpart1 replace columns (col1 int, col2 string, col4 string) cascade
@@ -819,7 +819,7 @@ PREHOOK: Input: statsdb1@testpart1
 POSTHOOK: query: describe formatted statsdb1.testpart1 partition (part = 'part1') col4
 POSTHOOK: type: DESCTABLE
 POSTHOOK: Input: statsdb1@testpart1
-# col_name            	data_type           	comment             		 	 	 	 	 	 	 
+# col_name            	data_type           	comment             	 	 	 	 	 	 	 	 
 	 	 	 	 	 	 	 	 	 	 
 col4                	string              	from deserializer   	 	 	 	 	 	 	 	 
 PREHOOK: query: alter table statsdb1.testpart1 change column col1 col1 string
@@ -848,7 +848,7 @@ PREHOOK: Input: statsdb1@testpart1
 POSTHOOK: query: describe formatted statsdb1.testpart1 partition (part = 'part1') col1
 POSTHOOK: type: DESCTABLE
 POSTHOOK: Input: statsdb1@testpart1
-# col_name            	data_type           	comment             		 	 	 	 	 	 	 
+# col_name            	data_type           	comment             	 	 	 	 	 	 	 	 
 	 	 	 	 	 	 	 	 	 	 
 col1                	string              	from deserializer   	 	 	 	 	 	 	 	 
 PREHOOK: query: describe formatted statsdb1.testpart1 partition (part = 'part1') col2
@@ -866,7 +866,7 @@ PREHOOK: Input: statsdb1@testpart1
 POSTHOOK: query: describe formatted statsdb1.testpart1 partition (part = 'part1') col4
 POSTHOOK: type: DESCTABLE
 POSTHOOK: Input: statsdb1@testpart1
-# col_name            	data_type           	comment             		 	 	 	 	 	 	 
+# col_name            	data_type           	comment             	 	 	 	 	 	 	 	 
 	 	 	 	 	 	 	 	 	 	 
 col4                	string              	from deserializer   	 	 	 	 	 	 	 	 
 PREHOOK: query: alter table statsdb1.testpart1 rename to statsdb2.testpart2

http://git-wip-us.apache.org/repos/asf/hive/blob/d729b454/ql/src/test/results/clientpositive/alter_view_as_select.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/alter_view_as_select.q.out b/ql/src/test/results/clientpositive/alter_view_as_select.q.out
index 2d82395..dc1814e 100644
--- a/ql/src/test/results/clientpositive/alter_view_as_select.q.out
+++ b/ql/src/test/results/clientpositive/alter_view_as_select.q.out
@@ -134,14 +134,14 @@ Bucket Columns:     	[]
 Sort Columns:       	[]                  	 
 	 	 
 # View Information	 	 
-View Original Text: 	SELECT * FROM src	 
-WHERE key > 80 AND key < 100	 	 
-ORDER BY key, value	 	 
-LIMIT 10	 	 
+View Original Text: 	SELECT * FROM src   	 
+	                    	WHERE key > 80 AND key < 100
+	                    	ORDER BY key, value 
+	                    	LIMIT 10            
 View Expanded Text: 	SELECT `src`.`key`, `src`.`value` FROM `default`.`src`	 
-WHERE `src`.`key` > 80 AND `src`.`key` < 100	 	 
-ORDER BY `src`.`key`, `src`.`value`	 	 
-LIMIT 10	 	 
+	                    	WHERE `src`.`key` > 80 AND `src`.`key` < 100
+	                    	ORDER BY `src`.`key`, `src`.`value`
+	                    	LIMIT 10            
 PREHOOK: query: DROP VIEW tv.testView
 PREHOOK: type: DROPVIEW
 PREHOOK: Input: tv@testview

http://git-wip-us.apache.org/repos/asf/hive/blob/d729b454/ql/src/test/results/clientpositive/columnstats_part_coltype.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/columnstats_part_coltype.q.out b/ql/src/test/results/clientpositive/columnstats_part_coltype.q.out
index 84232b6..d52f020 100644
--- a/ql/src/test/results/clientpositive/columnstats_part_coltype.q.out
+++ b/ql/src/test/results/clientpositive/columnstats_part_coltype.q.out
@@ -96,7 +96,7 @@ PREHOOK: Input: default@partcolstats
 POSTHOOK: query: describe formatted partcolstats partition (ds=date '2015-04-02', hr=2, part='partB') key
 POSTHOOK: type: DESCTABLE
 POSTHOOK: Input: default@partcolstats
-# col_name            	data_type           	comment             		 	 	 	 	 	 	 
+# col_name            	data_type           	comment             	 	 	 	 	 	 	 	 
 	 	 	 	 	 	 	 	 	 	 
 key                 	int                 	from deserializer   	 	 	 	 	 	 	 	 
 PREHOOK: query: describe formatted partcolstats partition (ds=date '2015-04-02', hr=2, part='partB') value
@@ -105,7 +105,7 @@ PREHOOK: Input: default@partcolstats
 POSTHOOK: query: describe formatted partcolstats partition (ds=date '2015-04-02', hr=2, part='partB') value
 POSTHOOK: type: DESCTABLE
 POSTHOOK: Input: default@partcolstats
-# col_name            	data_type           	comment             		 	 	 	 	 	 	 
+# col_name            	data_type           	comment             	 	 	 	 	 	 	 	 
 	 	 	 	 	 	 	 	 	 	 
 value               	string              	from deserializer   	 	 	 	 	 	 	 	 
 PREHOOK: query: analyze table partcolstats partition (ds=date '2015-04-02', hr=2, part) compute statistics for columns
@@ -144,7 +144,7 @@ PREHOOK: Input: default@partcolstats
 POSTHOOK: query: describe formatted partcolstats partition (ds=date '2015-04-02', hr=3, part='partA') key
 POSTHOOK: type: DESCTABLE
 POSTHOOK: Input: default@partcolstats
-# col_name            	data_type           	comment             		 	 	 	 	 	 	 
+# col_name            	data_type           	comment             	 	 	 	 	 	 	 	 
 	 	 	 	 	 	 	 	 	 	 
 key                 	int                 	from deserializer   	 	 	 	 	 	 	 	 
 PREHOOK: query: describe formatted partcolstats partition (ds=date '2015-04-02', hr=3, part='partA') value
@@ -153,7 +153,7 @@ PREHOOK: Input: default@partcolstats
 POSTHOOK: query: describe formatted partcolstats partition (ds=date '2015-04-02', hr=3, part='partA') value
 POSTHOOK: type: DESCTABLE
 POSTHOOK: Input: default@partcolstats
-# col_name            	data_type           	comment             		 	 	 	 	 	 	 
+# col_name            	data_type           	comment             	 	 	 	 	 	 	 	 
 	 	 	 	 	 	 	 	 	 	 
 value               	string              	from deserializer   	 	 	 	 	 	 	 	 
 PREHOOK: query: analyze table partcolstats partition (ds=date '2015-04-02', hr, part) compute statistics for columns
@@ -194,7 +194,7 @@ PREHOOK: Input: default@partcolstats
 POSTHOOK: query: describe formatted partcolstats partition (ds=date '2015-04-03', hr=3, part='partA') key
 POSTHOOK: type: DESCTABLE
 POSTHOOK: Input: default@partcolstats
-# col_name            	data_type           	comment             		 	 	 	 	 	 	 
+# col_name            	data_type           	comment             	 	 	 	 	 	 	 	 
 	 	 	 	 	 	 	 	 	 	 
 key                 	int                 	from deserializer   	 	 	 	 	 	 	 	 
 PREHOOK: query: describe formatted partcolstats partition (ds=date '2015-04-03', hr=3, part='partA') value
@@ -203,7 +203,7 @@ PREHOOK: Input: default@partcolstats
 POSTHOOK: query: describe formatted partcolstats partition (ds=date '2015-04-03', hr=3, part='partA') value
 POSTHOOK: type: DESCTABLE
 POSTHOOK: Input: default@partcolstats
-# col_name            	data_type           	comment             		 	 	 	 	 	 	 
+# col_name            	data_type           	comment             	 	 	 	 	 	 	 	 
 	 	 	 	 	 	 	 	 	 	 
 value               	string              	from deserializer   	 	 	 	 	 	 	 	 
 PREHOOK: query: describe formatted partcolstats partition (ds=date '2015-04-03', hr=3, part='partB') key
@@ -212,7 +212,7 @@ PREHOOK: Input: default@partcolstats
 POSTHOOK: query: describe formatted partcolstats partition (ds=date '2015-04-03', hr=3, part='partB') key
 POSTHOOK: type: DESCTABLE
 POSTHOOK: Input: default@partcolstats
-# col_name            	data_type           	comment             		 	 	 	 	 	 	 
+# col_name            	data_type           	comment             	 	 	 	 	 	 	 	 
 	 	 	 	 	 	 	 	 	 	 
 key                 	int                 	from deserializer   	 	 	 	 	 	 	 	 
 PREHOOK: query: describe formatted partcolstats partition (ds=date '2015-04-03', hr=3, part='partB') value
@@ -221,7 +221,7 @@ PREHOOK: Input: default@partcolstats
 POSTHOOK: query: describe formatted partcolstats partition (ds=date '2015-04-03', hr=3, part='partB') value
 POSTHOOK: type: DESCTABLE
 POSTHOOK: Input: default@partcolstats
-# col_name            	data_type           	comment             		 	 	 	 	 	 	 
+# col_name            	data_type           	comment             	 	 	 	 	 	 	 	 
 	 	 	 	 	 	 	 	 	 	 
 value               	string              	from deserializer   	 	 	 	 	 	 	 	 
 PREHOOK: query: analyze table partcolstats partition (ds, hr, part) compute statistics for columns

http://git-wip-us.apache.org/repos/asf/hive/blob/d729b454/ql/src/test/results/clientpositive/create_like.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/create_like.q.out b/ql/src/test/results/clientpositive/create_like.q.out
index 0111c94..58d9879 100644
--- a/ql/src/test/results/clientpositive/create_like.q.out
+++ b/ql/src/test/results/clientpositive/create_like.q.out
@@ -354,7 +354,28 @@ Retention:          	0
 #### A masked pattern was here ####
 Table Type:         	MANAGED_TABLE       	 
 Table Parameters:	 	 
-	avro.schema.literal 	{\n  \"namespace\": \"testing.hive.avro.serde\",\n  \"name\": \"doctors\",\n  \"type\": \"record\",\n  \"fields\": [\n    {\n      \"name\":\"number\",\n      \"type\":\"int\",\n      \"doc\":\"Order of playing the role\"\n    },\n    {\n      \"name\":\"first_name\",\n      \"type\":\"string\",\n      \"doc\":\"first name of actor playing role\"\n    },\n    {\n      \"name\":\"last_name\",\n      \"type\":\"string\",\n      \"doc\":\"last name of actor playing role\"\n    }\n  ]\n}
+	avro.schema.literal 	{                   
+	                    	  \"namespace\": \"testing.hive.avro.serde\",
+	                    	  \"name\": \"doctors\",
+	                    	  \"type\": \"record\",
+	                    	  \"fields\": [     
+	                    	    {               
+	                    	      \"name\":\"number\",
+	                    	      \"type\":\"int\",
+	                    	      \"doc\":\"Order of playing the role\"
+	                    	    },              
+	                    	    {               
+	                    	      \"name\":\"first_name\",
+	                    	      \"type\":\"string\",
+	                    	      \"doc\":\"first name of actor playing role\"
+	                    	    },              
+	                    	    {               
+	                    	      \"name\":\"last_name\",
+	                    	      \"type\":\"string\",
+	                    	      \"doc\":\"last name of actor playing role\"
+	                    	    }               
+	                    	  ]                 
+	                    	}                   
 	k1                  	v1                  
 	k2                  	v2                  
 #### A masked pattern was here ####
@@ -402,7 +423,28 @@ Retention:          	0
 Table Type:         	MANAGED_TABLE       	 
 Table Parameters:	 	 
 	COLUMN_STATS_ACCURATE	{\"BASIC_STATS\":\"true\"}
-	avro.schema.literal 	{\n  \"namespace\": \"testing.hive.avro.serde\",\n  \"name\": \"doctors\",\n  \"type\": \"record\",\n  \"fields\": [\n    {\n      \"name\":\"number\",\n      \"type\":\"int\",\n      \"doc\":\"Order of playing the role\"\n    },\n    {\n      \"name\":\"first_name\",\n      \"type\":\"string\",\n      \"doc\":\"first name of actor playing role\"\n    },\n    {\n      \"name\":\"last_name\",\n      \"type\":\"string\",\n      \"doc\":\"last name of actor playing role\"\n    }\n  ]\n}
+	avro.schema.literal 	{                   
+	                    	  \"namespace\": \"testing.hive.avro.serde\",
+	                    	  \"name\": \"doctors\",
+	                    	  \"type\": \"record\",
+	                    	  \"fields\": [     
+	                    	    {               
+	                    	      \"name\":\"number\",
+	                    	      \"type\":\"int\",
+	                    	      \"doc\":\"Order of playing the role\"
+	                    	    },              
+	                    	    {               
+	                    	      \"name\":\"first_name\",
+	                    	      \"type\":\"string\",
+	                    	      \"doc\":\"first name of actor playing role\"
+	                    	    },              
+	                    	    {               
+	                    	      \"name\":\"last_name\",
+	                    	      \"type\":\"string\",
+	                    	      \"doc\":\"last name of actor playing role\"
+	                    	    }               
+	                    	  ]                 
+	                    	}                   
 	numFiles            	0                   
 	numRows             	0                   
 	rawDataSize         	0                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d729b454/ql/src/test/results/clientpositive/create_view.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/create_view.q.out b/ql/src/test/results/clientpositive/create_view.q.out
index 7281185..12457b4 100644
--- a/ql/src/test/results/clientpositive/create_view.q.out
+++ b/ql/src/test/results/clientpositive/create_view.q.out
@@ -775,9 +775,9 @@ Sort Columns:       	[]
 	 	 
 # View Information	 	 
 View Original Text: 	SELECT test_translate('abc', 'a', 'b')	 
-FROM table1	 	 
+	                    	FROM table1         
 View Expanded Text: 	SELECT `_c0` AS `c` FROM (SELECT `test_translate`('abc', 'a', 'b')	 
-FROM `default`.`table1`) `default.view8`	 	 
+	                    	FROM `default`.`table1`) `default.view8`
 PREHOOK: query: SELECT * FROM view8
 PREHOOK: type: QUERY
 PREHOOK: Input: default@table1
@@ -855,9 +855,9 @@ Sort Columns:       	[]
 	 	 
 # View Information	 	 
 View Original Text: 	SELECT test_max(length(value))	 
-FROM src	 	 
+	                    	FROM src            
 View Expanded Text: 	SELECT `_c0` AS `m` FROM (SELECT `test_max`(length(`src`.`value`))	 
-FROM `default`.`src`) `default.view9`	 	 
+	                    	FROM `default`.`src`) `default.view9`
 PREHOOK: query: SELECT * FROM view9
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
@@ -933,9 +933,9 @@ Sort Columns:       	[]
 	 	 
 # View Information	 	 
 View Original Text: 	SELECT test_max(length(value))	 
-FROM src	 	 
+	                    	FROM src            
 View Expanded Text: 	SELECT `_c0` AS `m` FROM (SELECT `test_max`(length(`src`.`value`))	 
-FROM `default`.`src`) `default.view9`	 	 
+	                    	FROM `default`.`src`) `default.view9`
 PREHOOK: query: SELECT * FROM view9
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
@@ -1077,9 +1077,9 @@ Sort Columns:       	[]
 	 	 
 # View Information	 	 
 View Original Text: 	SELECT test_explode(array(1,2,3)) AS (boom)	 
-FROM table1	 	 
+	                    	FROM table1         
 View Expanded Text: 	SELECT `test_explode`(array(1,2,3)) AS (`boom`)	 
-FROM `default`.`table1`	 	 
+	                    	FROM `default`.`table1`
 PREHOOK: query: SELECT * FROM view11
 PREHOOK: type: QUERY
 PREHOOK: Input: default@table1
@@ -1233,10 +1233,10 @@ Bucket Columns:     	[]
 Sort Columns:       	[]                  	 
 	 	 
 # View Information	 	 
-View Original Text: 	SELECT s.key	 
-FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 5 ON key) s	 	 
-View Expanded Text: 	SELECT `s`.`key`	 
-FROM `default`.`srcbucket` TABLESAMPLE (BUCKET 1 OUT OF 5 ON `key`) `s`	 	 
+View Original Text: 	SELECT s.key        	 
+	                    	FROM srcbucket TABLESAMPLE (BUCKET 1 OUT OF 5 ON key) s
+View Expanded Text: 	SELECT `s`.`key`    	 
+	                    	FROM `default`.`srcbucket` TABLESAMPLE (BUCKET 1 OUT OF 5 ON `key`) `s`
 PREHOOK: query: SELECT * FROM view13
 ORDER BY key LIMIT 12
 PREHOOK: type: QUERY
@@ -1355,25 +1355,25 @@ Sort Columns:       	[]
 	 	 
 # View Information	 	 
 View Original Text: 	SELECT unionsrc1.key as k1, unionsrc1.value as v1,	 
-       unionsrc2.key as k2, unionsrc2.value as v2	 	 
-FROM (select 'tst1' as key, cast(count(1) as string) as value from src s1	 	 
-                         UNION  ALL	 	 
-      select s2.key as key, s2.value as value from src s2 where s2.key < 10) unionsrc1	 	 
-JOIN	 	 
-     (select 'tst1' as key, cast(count(1) as string) as value from src s3	 	 
-                         UNION  ALL	 	 
-      select s4.key as key, s4.value as value from src s4 where s4.key < 10) unionsrc2	 	 
-ON (unionsrc1.key = unionsrc2.key)	 	 
+	                    	       unionsrc2.key as k2, unionsrc2.value as v2
+	                    	FROM (select 'tst1' as key, cast(count(1) as string) as value from src s1
+	                    	                         UNION  ALL
+	                    	      select s2.key as key, s2.value as value from src s2 where s2.key < 10) unionsrc1
+	                    	JOIN                
+	                    	     (select 'tst1' as key, cast(count(1) as string) as value from src s3
+	                    	                         UNION  ALL
+	                    	      select s4.key as key, s4.value as value from src s4 where s4.key < 10) unionsrc2
+	                    	ON (unionsrc1.key = unionsrc2.key)
 View Expanded Text: 	SELECT `unionsrc1`.`key` as `k1`, `unionsrc1`.`value` as `v1`,	 
-       `unionsrc2`.`key` as `k2`, `unionsrc2`.`value` as `v2`	 	 
-FROM (select 'tst1' as `key`, cast(count(1) as string) as `value` from `default`.`src` `s1`	 	 
-                         UNION  ALL	 	 
-      select `s2`.`key` as `key`, `s2`.`value` as `value` from `default`.`src` `s2` where `s2`.`key` < 10) `unionsrc1`	 	 
-JOIN	 	 
-     (select 'tst1' as `key`, cast(count(1) as string) as `value` from `default`.`src` `s3`	 	 
-                         UNION  ALL	 	 
-      select `s4`.`key` as `key`, `s4`.`value` as `value` from `default`.`src` `s4` where `s4`.`key` < 10) `unionsrc2`	 	 
-ON (`unionsrc1`.`key` = `unionsrc2`.`key`)	 	 
+	                    	       `unionsrc2`.`key` as `k2`, `unionsrc2`.`value` as `v2`
+	                    	FROM (select 'tst1' as `key`, cast(count(1) as string) as `value` from `default`.`src` `s1`
+	                    	                         UNION  ALL
+	                    	      select `s2`.`key` as `key`, `s2`.`value` as `value` from `default`.`src` `s2` where `s2`.`key` < 10) `unionsrc1`
+	                    	JOIN                
+	                    	     (select 'tst1' as `key`, cast(count(1) as string) as `value` from `default`.`src` `s3`
+	                    	                         UNION  ALL
+	                    	      select `s4`.`key` as `key`, `s4`.`value` as `value` from `default`.`src` `s4` where `s4`.`key` < 10) `unionsrc2`
+	                    	ON (`unionsrc1`.`key` = `unionsrc2`.`key`)
 PREHOOK: query: SELECT * FROM view14
 ORDER BY k1
 PREHOOK: type: QUERY
@@ -1471,11 +1471,11 @@ Sort Columns:       	[]
 	 	 
 # View Information	 	 
 View Original Text: 	SELECT key,COUNT(value) AS value_count	 
-FROM src	 	 
-GROUP BY key	 	 
+	                    	FROM src            
+	                    	GROUP BY key        
 View Expanded Text: 	SELECT `src`.`key`,COUNT(`src`.`value`) AS `value_count`	 
-FROM `default`.`src`	 	 
-GROUP BY `src`.`key`	 	 
+	                    	FROM `default`.`src`
+	                    	GROUP BY `src`.`key`
 PREHOOK: query: SELECT * FROM view15
 ORDER BY value_count DESC, key
 LIMIT 10
@@ -1556,9 +1556,9 @@ Sort Columns:       	[]
 	 	 
 # View Information	 	 
 View Original Text: 	SELECT DISTINCT value	 
-FROM src	 	 
+	                    	FROM src            
 View Expanded Text: 	SELECT DISTINCT `src`.`value`	 
-FROM `default`.`src`	 	 
+	                    	FROM `default`.`src`
 PREHOOK: query: SELECT * FROM view16
 ORDER BY value
 LIMIT 10

http://git-wip-us.apache.org/repos/asf/hive/blob/d729b454/ql/src/test/results/clientpositive/create_view_partitioned.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/create_view_partitioned.q.out b/ql/src/test/results/clientpositive/create_view_partitioned.q.out
index 15d777a..4373303 100644
--- a/ql/src/test/results/clientpositive/create_view_partitioned.q.out
+++ b/ql/src/test/results/clientpositive/create_view_partitioned.q.out
@@ -86,12 +86,12 @@ Bucket Columns:     	[]
 Sort Columns:       	[]                  	 
 	 	 
 # View Information	 	 
-View Original Text: 	SELECT key, value	 
-FROM src	 	 
-WHERE key=86	 	 
+View Original Text: 	SELECT key, value   	 
+	                    	FROM src            
+	                    	WHERE key=86        
 View Expanded Text: 	SELECT `src`.`key`, `src`.`value`	 
-FROM `default`.`src`	 	 
-WHERE `src`.`key`=86	 	 
+	                    	FROM `default`.`src`
+	                    	WHERE `src`.`key`=86
 PREHOOK: query: SELECT * FROM vp1
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
@@ -406,12 +406,12 @@ Bucket Columns:     	[]
 Sort Columns:       	[]                  	 
 	 	 
 # View Information	 	 
-View Original Text: 	SELECT key, value	 
-FROM src	 	 
-WHERE key=86	 	 
+View Original Text: 	SELECT key, value   	 
+	                    	FROM src            
+	                    	WHERE key=86        
 View Expanded Text: 	SELECT `key` AS `k`, `value` AS `v` FROM (SELECT `src`.`key`, `src`.`value`	 
-FROM `default`.`src`	 	 
-WHERE `src`.`key`=86) `default.vp3`	 	 
+	                    	FROM `default`.`src`
+	                    	WHERE `src`.`key`=86) `default.vp3`
 PREHOOK: query: ALTER VIEW vp3
 ADD PARTITION (v='val_86')
 PREHOOK: type: ALTERTABLE_ADDPARTS

http://git-wip-us.apache.org/repos/asf/hive/blob/d729b454/ql/src/test/results/clientpositive/create_view_translate.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/create_view_translate.q.out b/ql/src/test/results/clientpositive/create_view_translate.q.out
index 2789f8f..43b9062 100644
--- a/ql/src/test/results/clientpositive/create_view_translate.q.out
+++ b/ql/src/test/results/clientpositive/create_view_translate.q.out
@@ -90,11 +90,11 @@ Sort Columns:       	[]
 	 	 
 # View Information	 	 
 View Original Text: 	select key, value from (	 
-  select key, value from src	 	 
-) a	 	 
+	                    	  select key, value from src
+	                    	) a                 
 View Expanded Text: 	select `a`.`key`, `a`.`value` from (	 
-  select `src`.`key`, `src`.`value` from `default`.`src`	 	 
-) `a`	 	 
+	                    	  select `src`.`key`, `src`.`value` from `default`.`src`
+	                    	) `a`               
 PREHOOK: query: drop view v
 PREHOOK: type: DROPVIEW
 PREHOOK: Input: default@v

http://git-wip-us.apache.org/repos/asf/hive/blob/d729b454/ql/src/test/results/clientpositive/create_with_constraints.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/create_with_constraints.q.out b/ql/src/test/results/clientpositive/create_with_constraints.q.out
index b040e43..056ca38 100644
--- a/ql/src/test/results/clientpositive/create_with_constraints.q.out
+++ b/ql/src/test/results/clientpositive/create_with_constraints.q.out
@@ -259,7 +259,7 @@ Storage Desc Params:
 # Primary Key	 	 
 Table:              	default.table1      	 
 Constraint Name:    	#### A masked pattern was here ####	 
-Column Names:       	a                   	
+Column Names:       	a                   	 
 PREHOOK: query: DESCRIBE FORMATTED table2
 PREHOOK: type: DESCTABLE
 PREHOOK: Input: default@table2
@@ -301,7 +301,7 @@ Storage Desc Params:
 # Primary Key	 	 
 Table:              	default.table2      	 
 Constraint Name:    	pk1                 	 
-Column Names:       	a                   	
+Column Names:       	a                   	 
 PREHOOK: query: DESCRIBE FORMATTED table3
 PREHOOK: type: DESCTABLE
 PREHOOK: Input: default@table3
@@ -342,7 +342,7 @@ Storage Desc Params:
 # Primary Key	 	 
 Table:              	default.table3      	 
 Constraint Name:    	#### A masked pattern was here ####	 
-Column Names:       	x                   	
+Column Names:       	x                   	 
 	 	 
 # Foreign Keys	 	 
 Table:              	default.table3      	 
@@ -390,7 +390,7 @@ Storage Desc Params:
 # Primary Key	 	 
 Table:              	default.table4      	 
 Constraint Name:    	#### A masked pattern was here ####	 
-Column Names:       	x                   	
+Column Names:       	x                   	 
 	 	 
 # Foreign Keys	 	 
 Table:              	default.table4      	 
@@ -440,7 +440,7 @@ Storage Desc Params:
 # Primary Key	 	 
 Table:              	default.table5      	 
 Constraint Name:    	#### A masked pattern was here ####	 
-Column Names:       	x                   	
+Column Names:       	x                   	 
 	 	 
 # Foreign Keys	 	 
 Table:              	default.table5      	 
@@ -488,7 +488,7 @@ Storage Desc Params:
 # Primary Key	 	 
 Table:              	default.table6      	 
 Constraint Name:    	#### A masked pattern was here ####	 
-Column Names:       	x                   	
+Column Names:       	x                   	 
 	 	 
 # Foreign Keys	 	 
 Table:              	default.table6      	 
@@ -539,7 +539,7 @@ Storage Desc Params:
 # Primary Key	 	 
 Table:              	default.table7      	 
 Constraint Name:    	#### A masked pattern was here ####	 
-Column Names:       	a                   	
+Column Names:       	a                   	 
 PREHOOK: query: DESCRIBE FORMATTED table8
 PREHOOK: type: DESCTABLE
 PREHOOK: Input: default@table8
@@ -581,7 +581,7 @@ Storage Desc Params:
 # Primary Key	 	 
 Table:              	default.table8      	 
 Constraint Name:    	pk8                 	 
-Column Names:       	a                   	
+Column Names:       	a                   	 
 PREHOOK: query: DESCRIBE FORMATTED table9
 PREHOOK: type: DESCTABLE
 PREHOOK: Input: default@table9
@@ -665,7 +665,7 @@ Storage Desc Params:
 # Primary Key	 	 
 Table:              	default.table10     	 
 Constraint Name:    	pk10                	 
-Column Names:       	a                   	
+Column Names:       	a                   	 
 	 	 
 # Foreign Keys	 	 
 Table:              	default.table10     	 
@@ -715,7 +715,7 @@ Storage Desc Params:
 # Primary Key	 	 
 Table:              	default.table11     	 
 Constraint Name:    	pk11                	 
-Column Names:       	a                   	
+Column Names:       	a                   	 
 	 	 
 # Foreign Keys	 	 
 Table:              	default.table11     	 
@@ -844,7 +844,7 @@ Storage Desc Params:
 # Primary Key	 	 
 Table:              	default.table3      	 
 Constraint Name:    	#### A masked pattern was here ####	 
-Column Names:       	x                   	
+Column Names:       	x                   	 
 PREHOOK: query: DESCRIBE FORMATTED table6
 PREHOOK: type: DESCTABLE
 PREHOOK: Input: default@table6
@@ -886,7 +886,7 @@ Storage Desc Params:
 # Primary Key	 	 
 Table:              	default.table6      	 
 Constraint Name:    	#### A masked pattern was here ####	 
-Column Names:       	x                   	
+Column Names:       	x                   	 
 PREHOOK: query: ALTER TABLE table2 ADD CONSTRAINT pkt2 primary key (a) disable novalidate
 PREHOOK: type: ALTERTABLE_ADDCONSTRAINT
 POSTHOOK: query: ALTER TABLE table2 ADD CONSTRAINT pkt2 primary key (a) disable novalidate
@@ -940,7 +940,7 @@ Storage Desc Params:
 # Primary Key	 	 
 Table:              	default.table2      	 
 Constraint Name:    	pkt2                	 
-Column Names:       	a                   	
+Column Names:       	a                   	 
 PREHOOK: query: DESCRIBE FORMATTED table3
 PREHOOK: type: DESCTABLE
 PREHOOK: Input: default@table3
@@ -981,7 +981,7 @@ Storage Desc Params:
 # Primary Key	 	 
 Table:              	default.table3      	 
 Constraint Name:    	#### A masked pattern was here ####	 
-Column Names:       	x                   	
+Column Names:       	x                   	 
 	 	 
 # Foreign Keys	 	 
 Table:              	default.table3      	 
@@ -1029,7 +1029,7 @@ Storage Desc Params:
 # Primary Key	 	 
 Table:              	default.table6      	 
 Constraint Name:    	#### A masked pattern was here ####	 
-Column Names:       	x                   	
+Column Names:       	x                   	 
 	 	 
 # Foreign Keys	 	 
 Table:              	default.table6      	 
@@ -1117,7 +1117,7 @@ Storage Desc Params:
 # Primary Key	 	 
 Table:              	dbconstraint.table2 	 
 Constraint Name:    	pk1                 	 
-Column Names:       	a                   	
+Column Names:       	a                   	 
 PREHOOK: query: ALTER TABLE dbconstraint.table2 DROP CONSTRAINT pk1
 PREHOOK: type: ALTERTABLE_DROPCONSTRAINT
 POSTHOOK: query: ALTER TABLE dbconstraint.table2 DROP CONSTRAINT pk1
@@ -1212,7 +1212,7 @@ Storage Desc Params:
 # Primary Key	 	 
 Table:              	dbconstraint.table2 	 
 Constraint Name:    	pk1                 	 
-Column Names:       	a                   	
+Column Names:       	a                   	 
 PREHOOK: query: ALTER TABLE dbconstraint.table2  ADD CONSTRAINT fkx FOREIGN KEY (b) REFERENCES table1(a)  DISABLE NOVALIDATE
 PREHOOK: type: ALTERTABLE_ADDCONSTRAINT
 POSTHOOK: query: ALTER TABLE dbconstraint.table2  ADD CONSTRAINT fkx FOREIGN KEY (b) REFERENCES table1(a)  DISABLE NOVALIDATE
@@ -1258,7 +1258,7 @@ Storage Desc Params:
 # Primary Key	 	 
 Table:              	dbconstraint.table2 	 
 Constraint Name:    	pk1                 	 
-Column Names:       	a                   	
+Column Names:       	a                   	 
 	 	 
 # Foreign Keys	 	 
 Table:              	dbconstraint.table2 	 

http://git-wip-us.apache.org/repos/asf/hive/blob/d729b454/ql/src/test/results/clientpositive/describe_comment_indent.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/describe_comment_indent.q.out b/ql/src/test/results/clientpositive/describe_comment_indent.q.out
index 5a01de1..5b41fb8 100644
--- a/ql/src/test/results/clientpositive/describe_comment_indent.q.out
+++ b/ql/src/test/results/clientpositive/describe_comment_indent.q.out
@@ -34,10 +34,10 @@ POSTHOOK: type: DESCTABLE
 POSTHOOK: Input: default@test_table
 col1                	int                 	col1 one line comment
 col2                	string              	col2                
-                    	                    	two lines comment
+                    	                    	two lines comment   
 col3                	string              	col3                
-                    	                    	three lines
-                    	                    	comment
+                    	                    	three lines         
+                    	                    	comment             
 PREHOOK: query: DESCRIBE FORMATTED test_table
 PREHOOK: type: DESCTABLE
 PREHOOK: Input: default@test_table
@@ -48,10 +48,10 @@ POSTHOOK: Input: default@test_table
 	 	 
 col1                	int                 	col1 one line comment
 col2                	string              	col2                
-                    	                    	two lines comment
+                    	                    	two lines comment   
 col3                	string              	col3                
-                    	                    	three lines
-                    	                    	comment
+                    	                    	three lines         
+                    	                    	comment             
 	 	 
 # Detailed Table Information	 	 
 Database:           	default             	 
@@ -61,7 +61,8 @@ Retention:          	0
 Table Type:         	MANAGED_TABLE       	 
 Table Parameters:	 	 
 	COLUMN_STATS_ACCURATE	{\"BASIC_STATS\":\"true\"}
-	comment             	table comment\ntwo lines
+	comment             	table comment       
+	                    	two lines           
 	numFiles            	0                   
 	numRows             	0                   
 	rawDataSize         	0                   

http://git-wip-us.apache.org/repos/asf/hive/blob/d729b454/ql/src/test/results/clientpositive/escape_comments.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/escape_comments.q.out b/ql/src/test/results/clientpositive/escape_comments.q.out
new file mode 100644
index 0000000..0b8c5c5
--- /dev/null
+++ b/ql/src/test/results/clientpositive/escape_comments.q.out
@@ -0,0 +1,213 @@
+PREHOOK: query: create database escape_comments_db comment 'a\nb'
+PREHOOK: type: CREATEDATABASE
+PREHOOK: Output: database:escape_comments_db
+POSTHOOK: query: create database escape_comments_db comment 'a\nb'
+POSTHOOK: type: CREATEDATABASE
+POSTHOOK: Output: database:escape_comments_db
+PREHOOK: query: use escape_comments_db
+PREHOOK: type: SWITCHDATABASE
+PREHOOK: Input: database:escape_comments_db
+POSTHOOK: query: use escape_comments_db
+POSTHOOK: type: SWITCHDATABASE
+POSTHOOK: Input: database:escape_comments_db
+PREHOOK: query: create table escape_comments_tbl1
+(col1 string comment 'a\nb\';') comment 'a\nb'
+partitioned by (p1 string comment 'a\nb')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:escape_comments_db
+PREHOOK: Output: escape_comments_db@escape_comments_tbl1
+POSTHOOK: query: create table escape_comments_tbl1
+(col1 string comment 'a\nb\';') comment 'a\nb'
+partitioned by (p1 string comment 'a\nb')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:escape_comments_db
+POSTHOOK: Output: escape_comments_db@escape_comments_tbl1
+PREHOOK: query: create view escape_comments_view1 (col1 comment 'a\nb') comment 'a\nb'
+as select col1 from escape_comments_tbl1
+PREHOOK: type: CREATEVIEW
+PREHOOK: Input: escape_comments_db@escape_comments_tbl1
+PREHOOK: Output: database:escape_comments_db
+PREHOOK: Output: escape_comments_db@escape_comments_view1
+POSTHOOK: query: create view escape_comments_view1 (col1 comment 'a\nb') comment 'a\nb'
+as select col1 from escape_comments_tbl1
+POSTHOOK: type: CREATEVIEW
+POSTHOOK: Input: escape_comments_db@escape_comments_tbl1
+POSTHOOK: Output: database:escape_comments_db
+POSTHOOK: Output: escape_comments_db@escape_comments_view1
+PREHOOK: query: create index index2 on table escape_comments_tbl1(col1) as 'COMPACT' with deferred rebuild comment 'a\nb'
+PREHOOK: type: CREATEINDEX
+PREHOOK: Input: escape_comments_db@escape_comments_tbl1
+POSTHOOK: query: create index index2 on table escape_comments_tbl1(col1) as 'COMPACT' with deferred rebuild comment 'a\nb'
+POSTHOOK: type: CREATEINDEX
+POSTHOOK: Input: escape_comments_db@escape_comments_tbl1
+POSTHOOK: Output: escape_comments_db@escape_comments_db__escape_comments_tbl1_index2__
+PREHOOK: query: describe database extended escape_comments_db
+PREHOOK: type: DESCDATABASE
+PREHOOK: Input: database:escape_comments_db
+POSTHOOK: query: describe database extended escape_comments_db
+POSTHOOK: type: DESCDATABASE
+POSTHOOK: Input: database:escape_comments_db
+escape_comments_db	a\nb	location/in/test	hive_test_user	USER	
+PREHOOK: query: describe database escape_comments_db
+PREHOOK: type: DESCDATABASE
+PREHOOK: Input: database:escape_comments_db
+POSTHOOK: query: describe database escape_comments_db
+POSTHOOK: type: DESCDATABASE
+POSTHOOK: Input: database:escape_comments_db
+escape_comments_db	a\nb	location/in/test	hive_test_user	USER	
+PREHOOK: query: show create table escape_comments_tbl1
+PREHOOK: type: SHOW_CREATETABLE
+PREHOOK: Input: escape_comments_db@escape_comments_tbl1
+POSTHOOK: query: show create table escape_comments_tbl1
+POSTHOOK: type: SHOW_CREATETABLE
+POSTHOOK: Input: escape_comments_db@escape_comments_tbl1
+CREATE TABLE `escape_comments_tbl1`(
+  `col1` string COMMENT 'a\nb\'\;')
+COMMENT 'a\nb'
+PARTITIONED BY ( 
+  `p1` string COMMENT 'a\nb')
+ROW FORMAT SERDE 
+  'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' 
+STORED AS INPUTFORMAT 
+  'org.apache.hadoop.mapred.TextInputFormat' 
+OUTPUTFORMAT 
+  'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat'
+LOCATION
+#### A masked pattern was here ####
+TBLPROPERTIES (
+#### A masked pattern was here ####
+PREHOOK: query: describe formatted escape_comments_tbl1
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: escape_comments_db@escape_comments_tbl1
+POSTHOOK: query: describe formatted escape_comments_tbl1
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: escape_comments_db@escape_comments_tbl1
+# col_name            	data_type           	comment             
+	 	 
+col1                	string              	a                   
+                    	                    	b';                 
+	 	 
+# Partition Information	 	 
+# col_name            	data_type           	comment             
+	 	 
+p1                  	string              	a                   
+                    	                    	b                   
+	 	 
+# Detailed Table Information	 	 
+Database:           	escape_comments_db  	 
+#### A masked pattern was here ####
+Retention:          	0                   	 
+#### A masked pattern was here ####
+Table Type:         	MANAGED_TABLE       	 
+Table Parameters:	 	 
+	comment             	a                   
+	                    	b                   
+#### A masked pattern was here ####
+	 	 
+# Storage Information	 	 
+SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
+InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
+OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
+Compressed:         	No                  	 
+Num Buckets:        	-1                  	 
+Bucket Columns:     	[]                  	 
+Sort Columns:       	[]                  	 
+Storage Desc Params:	 	 
+	serialization.format	1                   
+PREHOOK: query: describe pretty escape_comments_tbl1
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: escape_comments_db@escape_comments_tbl1
+POSTHOOK: query: describe pretty escape_comments_tbl1
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: escape_comments_db@escape_comments_tbl1
+col_name 	data_type     	comment
+		
+col1     	string        	a
+         	              	b';
+p1       	string        	a
+         	              	b
+	 	 
+# Partition Information	 	 
+col_name 	data_type     	comment
+		
+p1       	string        	a
+         	              	b
+PREHOOK: query: describe escape_comments_tbl1
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: escape_comments_db@escape_comments_tbl1
+POSTHOOK: query: describe escape_comments_tbl1
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: escape_comments_db@escape_comments_tbl1
+col1                	string              	a                   
+                    	                    	b';                 
+p1                  	string              	a                   
+                    	                    	b                   
+	 	 
+# Partition Information	 	 
+# col_name            	data_type           	comment             
+	 	 
+p1                  	string              	a                   
+                    	                    	b                   
+PREHOOK: query: show create table escape_comments_view1
+PREHOOK: type: SHOW_CREATETABLE
+PREHOOK: Input: escape_comments_db@escape_comments_view1
+POSTHOOK: query: show create table escape_comments_view1
+POSTHOOK: type: SHOW_CREATETABLE
+POSTHOOK: Input: escape_comments_db@escape_comments_view1
+CREATE VIEW `escape_comments_view1` AS SELECT `col1` AS `col1` FROM (select `escape_comments_tbl1`.`col1` from `escape_comments_db`.`escape_comments_tbl1`) `escape_comments_db.escape_comments_view1`
+PREHOOK: query: describe formatted escape_comments_view1
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: escape_comments_db@escape_comments_view1
+POSTHOOK: query: describe formatted escape_comments_view1
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: escape_comments_db@escape_comments_view1
+# col_name            	data_type           	comment             
+	 	 
+col1                	string              	a                   
+                    	                    	b                   
+	 	 
+# Detailed Table Information	 	 
+Database:           	escape_comments_db  	 
+#### A masked pattern was here ####
+Retention:          	0                   	 
+Table Type:         	VIRTUAL_VIEW        	 
+Table Parameters:	 	 
+	comment             	a                   
+	                    	b                   
+#### A masked pattern was here ####
+	 	 
+# Storage Information	 	 
+SerDe Library:      	null                	 
+InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
+OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
+Compressed:         	No                  	 
+Num Buckets:        	-1                  	 
+Bucket Columns:     	[]                  	 
+Sort Columns:       	[]                  	 
+	 	 
+# View Information	 	 
+View Original Text: 	select col1 from escape_comments_tbl1	 
+View Expanded Text: 	SELECT `col1` AS `col1` FROM (select `escape_comments_tbl1`.`col1` from `escape_comments_db`.`escape_comments_tbl1`) `escape_comments_db.escape_comments_view1`	 
+PREHOOK: query: show formatted index on escape_comments_tbl1
+PREHOOK: type: SHOWINDEXES
+POSTHOOK: query: show formatted index on escape_comments_tbl1
+POSTHOOK: type: SHOWINDEXES
+idx_name            	tab_name            	col_names           	idx_tab_name        	idx_type            	comment             
+	 	 	 	 	 
+	 	 	 	 	 
+index2              	escape_comments_tbl1	col1                	escape_comments_db__escape_comments_tbl1_index2__	compact             	a                   
+                    	                    	                    	                                                 	                    	b                   
+PREHOOK: query: drop database escape_comments_db cascade
+PREHOOK: type: DROPDATABASE
+PREHOOK: Input: database:escape_comments_db
+PREHOOK: Output: database:escape_comments_db
+PREHOOK: Output: escape_comments_db@escape_comments_db__escape_comments_tbl1_index2__
+PREHOOK: Output: escape_comments_db@escape_comments_tbl1
+PREHOOK: Output: escape_comments_db@escape_comments_view1
+POSTHOOK: query: drop database escape_comments_db cascade
+POSTHOOK: type: DROPDATABASE
+POSTHOOK: Input: database:escape_comments_db
+POSTHOOK: Output: database:escape_comments_db
+POSTHOOK: Output: escape_comments_db@escape_comments_db__escape_comments_tbl1_index2__
+POSTHOOK: Output: escape_comments_db@escape_comments_tbl1
+POSTHOOK: Output: escape_comments_db@escape_comments_view1


[19/22] hive git commit: HIVE-14762 : Add logging while removing scratch space (Ashutosh Chauhan via Pengcheng Xiong)

Posted by se...@apache.org.
HIVE-14762 : Add logging while removing scratch space (Ashutosh Chauhan via Pengcheng Xiong)

Signed-off-by: Ashutosh Chauhan <ha...@apache.org>


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

Branch: refs/heads/hive-14535
Commit: f4a857ca17348cff8671e685a13ebc01b5114c0f
Parents: 17afd9b
Author: Ashutosh Chauhan <ha...@apache.org>
Authored: Wed Sep 14 16:41:07 2016 -0700
Committer: Ashutosh Chauhan <ha...@apache.org>
Committed: Tue Oct 11 14:22:54 2016 -0700

----------------------------------------------------------------------
 ql/src/java/org/apache/hadoop/hive/ql/Context.java | 7 +++++--
 1 file changed, 5 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/f4a857ca/ql/src/java/org/apache/hadoop/hive/ql/Context.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/Context.java b/ql/src/java/org/apache/hadoop/hive/ql/Context.java
index ceb257c..838d73e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/Context.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/Context.java
@@ -99,7 +99,7 @@ public class Context {
   protected int tryCount = 0;
   private TokenRewriteStream tokenRewriteStream;
 
-  private String executionId;
+  private final String executionId;
 
   // List of Locks for this query
   protected List<HiveLock> hiveLocks;
@@ -127,7 +127,7 @@ public class Context {
   private final String stagingDir;
 
   private Heartbeater heartbeater;
-  
+
   private boolean skipTableMasking;
 
   public Context(Configuration conf) throws IOException {
@@ -392,6 +392,7 @@ public class Context {
       try {
         Path p = entry.getValue();
         FileSystem fs = p.getFileSystem(conf);
+        LOG.debug("Deleting scratch dir: {}",  p);
         fs.delete(p, true);
         fs.cancelDeleteOnExit(p);
       } catch (Exception e) {
@@ -533,6 +534,7 @@ public class Context {
     if (resDir != null) {
       try {
         FileSystem fs = resDir.getFileSystem(conf);
+        LOG.debug("Deleting result dir: {}",  resDir);
         fs.delete(resDir, true);
       } catch (IOException e) {
         LOG.info("Context clear error: " + StringUtils.stringifyException(e));
@@ -542,6 +544,7 @@ public class Context {
     if (resFile != null) {
       try {
         FileSystem fs = resFile.getFileSystem(conf);
+        LOG.debug("Deleting result file: {}",  resFile);
         fs.delete(resFile, false);
       } catch (IOException e) {
         LOG.info("Context clear error: " + StringUtils.stringifyException(e));


[21/22] hive git commit: HIVE-14671 : merge master into hive-14535 (Sergey Shelukhin)

Posted by se...@apache.org.
HIVE-14671 : merge master into hive-14535 (Sergey Shelukhin)


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

Branch: refs/heads/hive-14535
Commit: 7d50a1515434e1abf50970965649c3cc9a6ebb08
Parents: 0ce24b9 9feed2f
Author: Sergey Shelukhin <se...@apache.org>
Authored: Tue Oct 11 16:08:03 2016 -0700
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Tue Oct 11 16:08:03 2016 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/conf/HiveConf.java   |    5 +-
 .../hive/common/util/HiveStringUtils.java       |   23 +-
 data/scripts/q_test_cleanup_compare.sql         |    1 +
 data/scripts/q_test_cleanup_contrib.sql         |    2 +
 data/scripts/q_test_cleanup_for_encryption.sql  |    1 -
 data/scripts/q_test_cleanup_src.sql             |    1 +
 data/scripts/q_test_cleanup_tez.sql             |    4 +
 data/scripts/q_test_init_compare.sql            |   26 +
 data/scripts/q_test_init_contrib.sql            |   29 +
 data/scripts/q_test_init_for_encryption.sql     |    5 -
 data/scripts/q_test_init_src.sql                |    5 +
 data/scripts/q_test_init_src_with_stats.sql     |    9 +
 data/scripts/q_test_init_tez.sql                |   78 +
 .../apache/hadoop/hive/ql/TestMTQueries.java    |    3 +-
 .../hadoop/hive/cli/control/CliConfigs.java     |   36 +-
 .../org/apache/hadoop/hive/ql/QTestUtil.java    |    6 +-
 .../hive/metastore/IExtrapolatePartStatus.java  |    1 +
 .../hive/metastore/StatObjectConverter.java     |   29 +
 .../java/org/apache/hadoop/hive/ql/Context.java |    7 +-
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java |   41 +-
 .../hadoop/hive/ql/exec/FunctionRegistry.java   |   18 +-
 .../hadoop/hive/ql/exec/mr/MapRedTask.java      |   17 +-
 .../expressions/VectorUDFDayOfWeekDate.java     |   39 +
 .../expressions/VectorUDFDayOfWeekString.java   |   61 +
 .../VectorUDFDayOfWeekTimestamp.java            |   39 +
 .../formatting/MetaDataFormatUtils.java         |  156 +-
 .../formatting/TextMetaDataFormatter.java       |    5 +-
 .../hive/ql/optimizer/GenMRTableScan1.java      |    4 +-
 .../hive/ql/optimizer/physical/Vectorizer.java  |    2 +
 .../hadoop/hive/ql/parse/FromClauseParser.g     |    2 +-
 .../org/apache/hadoop/hive/ql/parse/HiveLexer.g |    1 +
 .../apache/hadoop/hive/ql/parse/HiveParser.g    |  104 +-
 .../hadoop/hive/ql/parse/IdentifiersParser.g    |    5 +-
 .../hadoop/hive/ql/processors/SetProcessor.java |   33 +-
 .../hadoop/hive/ql/session/SessionState.java    |   11 +-
 .../apache/hadoop/hive/ql/udf/UDFDayOfWeek.java |  105 +
 .../hive/ql/processors/TestSetProcessor.java    |  114 +-
 .../ptf_negative_AmbiguousWindowDefn.q          |    6 +-
 .../clientnegative/ptf_negative_NoWindowDefn.q  |    4 +-
 .../clientnegative/windowing_after_orderby.q    |    7 +
 .../test/queries/clientpositive/acid_mapjoin.q  |    3 +-
 .../clientpositive/alter_merge_stats_orc.q      |    4 +-
 .../clientpositive/archive_excludeHadoop20.q    |    5 +-
 .../clientpositive/auto_sortmerge_join_8.q      |    4 +-
 ql/src/test/queries/clientpositive/ctas.q       |    2 +-
 .../test/queries/clientpositive/ctas_hadoop20.q |   63 -
 .../queries/clientpositive/escape_comments.q    |   20 +
 .../queries/clientpositive/explainanalyze_2.q   |  162 -
 ql/src/test/queries/clientpositive/extract.q    |    3 +
 .../extrapolate_part_stats_date.q               |   14 +
 .../test/queries/clientpositive/nonmr_fetch.q   |    4 +-
 .../test/queries/clientpositive/orc_analyze.q   |    4 +-
 ql/src/test/queries/clientpositive/sample10.q   |    4 +-
 .../clientpositive/sample_islocalmode_hook.q    |    4 +-
 .../sample_islocalmode_hook_use_metadata.q      |    6 +-
 .../test/queries/clientpositive/union_paren.q   |   54 +
 .../clientpositive/vectorized_date_funcs.q      |    8 +-
 .../queries/clientpositive/vectorized_parquet.q |    4 +-
 .../alter_view_as_select_with_partition.q.out   |   10 +-
 .../create_or_replace_view6.q.out               |    2 +-
 .../results/clientnegative/unionClusterBy.q.out |    2 +-
 .../clientnegative/unionDistributeBy.q.out      |    2 +-
 .../results/clientnegative/unionLimit.q.out     |    2 +-
 .../results/clientnegative/unionOrderBy.q.out   |    2 +-
 .../results/clientnegative/unionSortBy.q.out    |    2 +-
 .../windowing_after_orderby.q.out               |    9 +
 .../results/clientpositive/acid_mapjoin.q.out   |   30 +-
 .../alter_table_invalidate_column_stats.q.out   |   24 +-
 .../clientpositive/alter_view_as_select.q.out   |   14 +-
 .../clientpositive/cbo_rp_lineage2.q.out        |    2 +-
 .../columnstats_part_coltype.q.out              |   16 +-
 .../results/clientpositive/complex_alias.q.out  |    4 +-
 .../clientpositive/constant_prop_1.q.out        |   12 +-
 .../results/clientpositive/create_like.q.out    |   46 +-
 .../results/clientpositive/create_view.q.out    |   72 +-
 .../create_view_partitioned.q.out               |   20 +-
 .../clientpositive/create_view_translate.q.out  |    8 +-
 .../create_with_constraints.q.out               |   36 +-
 ql/src/test/results/clientpositive/ctas.q.out   |  118 +-
 .../results/clientpositive/ctas_hadoop20.q.out  |  945 -----
 .../describe_comment_indent.q.out               |   15 +-
 .../clientpositive/escape_comments.q.out        |  213 +
 .../test/results/clientpositive/extract.q.out   |   11 +
 .../extrapolate_part_stats_date.q.out           |  302 ++
 .../results/clientpositive/input_part7.q.out    |    4 +-
 .../test/results/clientpositive/llap/ctas.q.out |   81 +-
 .../clientpositive/llap/explainuser_2.q.out     |  442 +-
 .../llap/vectorized_date_funcs.q.out            |  846 ++--
 .../clientpositive/optimize_nullscan.q.out      |    8 +-
 .../results/clientpositive/show_functions.q.out |    1 +
 .../results/clientpositive/spark/ctas.q.out     |   81 +-
 .../clientpositive/tez/explainanalyze_2.q.out   | 3790 +++---------------
 .../clientpositive/tez/explainuser_3.q.out      |    6 +-
 .../results/clientpositive/union_offcbo.q.out   |  112 +-
 .../results/clientpositive/union_paren.q.out    |  260 ++
 .../test/results/clientpositive/union_ppr.q.out |    4 +-
 .../vector_join_part_col_char.q.out             |    6 +-
 .../clientpositive/vectorized_date_funcs.q.out  |  846 ++--
 .../service/cli/thrift/ThriftHttpServlet.java   |    2 +-
 .../auth/TestLdapAtnProviderWithMiniDS.java     |  349 +-
 .../src/test/resources/ldap/example.com.ldif    |   96 +
 101 files changed, 3964 insertions(+), 6302 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/7d50a151/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
diff --cc common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index ca47e47,5ea9751..1bd3c30
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@@ -3117,26 -3114,11 +3117,27 @@@ public class HiveConf extends Configura
          "Log tracing id that can be used by upstream clients for tracking respective logs. " +
          "Truncated to " + LOG_PREFIX_LENGTH + " characters. Defaults to use auto-generated session id."),
  
 +    HIVE_METASTORE_MM_THREAD_SCAN_INTERVAL("hive.metastore.mm.thread.scan.interval", "900s",
 +        new TimeValidator(TimeUnit.SECONDS),
 +        "MM table housekeeping thread interval in this metastore instance. 0 to disable."),
 +
 +    HIVE_METASTORE_MM_HEARTBEAT_TIMEOUT("hive.metastore.mm.heartbeat.timeout", "1800s",
 +        new TimeValidator(TimeUnit.SECONDS),
 +        "MM write ID times out after this long if a heartbeat is not send. Currently disabled."), // TODO# heartbeating not implemented
 +
 +    HIVE_METASTORE_MM_ABSOLUTE_TIMEOUT("hive.metastore.mm.absolute.timeout", "7d",
 +        new TimeValidator(TimeUnit.SECONDS),
 +        "MM write ID cannot be outstanding for more than this long."),
 +
 +    HIVE_METASTORE_MM_ABORTED_GRACE_PERIOD("hive.metastore.mm.aborted.grace.period", "1d",
 +        new TimeValidator(TimeUnit.SECONDS),
 +        "MM write ID will not be removed up for that long after it has been aborted;\n" +
 +        "this is to work around potential races e.g. with FS visibility, when deleting files."),
  
      HIVE_CONF_RESTRICTED_LIST("hive.conf.restricted.list",
-         "hive.security.authenticator.manager,hive.security.authorization.manager,hive.users.in.admin.role," +
-         "hive.server2.xsrf.filter.enabled",
+         "hive.security.authenticator.manager,hive.security.authorization.manager," +
+         "hive.security.metastore.authorization.manager,hive.security.metastore.authenticator.manager," +
+         "hive.users.in.admin.role,hive.server2.xsrf.filter.enabled,hive.security.authorization.enabled",
          "Comma separated list of configuration options which are immutable at runtime"),
      HIVE_CONF_HIDDEN_LIST("hive.conf.hidden.list",
          METASTOREPWD.varname + "," + HIVE_SERVER2_SSL_KEYSTORE_PASSWORD.varname

http://git-wip-us.apache.org/repos/asf/hive/blob/7d50a151/ql/src/java/org/apache/hadoop/hive/ql/Context.java
----------------------------------------------------------------------


[15/22] hive git commit: HIVE-14917: explainanalyze_2.q fails after HIVE-14861

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/18767237/ql/src/test/results/clientpositive/tez/explainanalyze_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/explainanalyze_2.q.out b/ql/src/test/results/clientpositive/tez/explainanalyze_2.q.out
index 0cbddde..23d151c 100644
--- a/ql/src/test/results/clientpositive/tez/explainanalyze_2.q.out
+++ b/ql/src/test/results/clientpositive/tez/explainanalyze_2.q.out
@@ -1,521 +1,3 @@
-PREHOOK: query: -- SORT_QUERY_RESULTS
-
-CREATE TABLE dest_j1(key STRING, value STRING, val2 STRING) STORED AS TEXTFILE
-PREHOOK: type: CREATETABLE
-PREHOOK: Output: database:default
-PREHOOK: Output: default@dest_j1
-POSTHOOK: query: -- SORT_QUERY_RESULTS
-
-CREATE TABLE dest_j1(key STRING, value STRING, val2 STRING) STORED AS TEXTFILE
-POSTHOOK: type: CREATETABLE
-POSTHOOK: Output: database:default
-POSTHOOK: Output: default@dest_j1
-PREHOOK: query: CREATE TABLE ss(k1 STRING,v1 STRING,k2 STRING,v2 STRING,k3 STRING,v3 STRING) STORED AS TEXTFILE
-PREHOOK: type: CREATETABLE
-PREHOOK: Output: database:default
-PREHOOK: Output: default@ss
-POSTHOOK: query: CREATE TABLE ss(k1 STRING,v1 STRING,k2 STRING,v2 STRING,k3 STRING,v3 STRING) STORED AS TEXTFILE
-POSTHOOK: type: CREATETABLE
-POSTHOOK: Output: database:default
-POSTHOOK: Output: default@ss
-PREHOOK: query: CREATE TABLE sr(k1 STRING,v1 STRING,k2 STRING,v2 STRING,k3 STRING,v3 STRING) STORED AS TEXTFILE
-PREHOOK: type: CREATETABLE
-PREHOOK: Output: database:default
-PREHOOK: Output: default@sr
-POSTHOOK: query: CREATE TABLE sr(k1 STRING,v1 STRING,k2 STRING,v2 STRING,k3 STRING,v3 STRING) STORED AS TEXTFILE
-POSTHOOK: type: CREATETABLE
-POSTHOOK: Output: database:default
-POSTHOOK: Output: default@sr
-PREHOOK: query: CREATE TABLE cs(k1 STRING,v1 STRING,k2 STRING,v2 STRING,k3 STRING,v3 STRING) STORED AS TEXTFILE
-PREHOOK: type: CREATETABLE
-PREHOOK: Output: database:default
-PREHOOK: Output: default@cs
-POSTHOOK: query: CREATE TABLE cs(k1 STRING,v1 STRING,k2 STRING,v2 STRING,k3 STRING,v3 STRING) STORED AS TEXTFILE
-POSTHOOK: type: CREATETABLE
-POSTHOOK: Output: database:default
-POSTHOOK: Output: default@cs
-PREHOOK: query: INSERT OVERWRITE TABLE ss
-SELECT x.key,x.value,y.key,y.value,z.key,z.value
-FROM src1 x 
-JOIN src y ON (x.key = y.key) 
-JOIN srcpart z ON (x.value = z.value and z.ds='2008-04-08' and z.hr=11)
-PREHOOK: type: QUERY
-PREHOOK: Input: default@src
-PREHOOK: Input: default@src1
-PREHOOK: Input: default@srcpart
-PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
-PREHOOK: Output: default@ss
-POSTHOOK: query: INSERT OVERWRITE TABLE ss
-SELECT x.key,x.value,y.key,y.value,z.key,z.value
-FROM src1 x 
-JOIN src y ON (x.key = y.key) 
-JOIN srcpart z ON (x.value = z.value and z.ds='2008-04-08' and z.hr=11)
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@src
-POSTHOOK: Input: default@src1
-POSTHOOK: Input: default@srcpart
-POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
-POSTHOOK: Output: default@ss
-POSTHOOK: Lineage: ss.k1 SIMPLE [(src1)x.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: ss.k2 SIMPLE [(src)y.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: ss.k3 SIMPLE [(srcpart)z.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: ss.v1 SIMPLE [(src1)x.FieldSchema(name:value, type:string, comment:default), ]
-POSTHOOK: Lineage: ss.v2 SIMPLE [(src)y.FieldSchema(name:value, type:string, comment:default), ]
-POSTHOOK: Lineage: ss.v3 SIMPLE [(srcpart)z.FieldSchema(name:value, type:string, comment:default), ]
-PREHOOK: query: INSERT OVERWRITE TABLE sr
-SELECT x.key,x.value,y.key,y.value,z.key,z.value
-FROM src1 x 
-JOIN src y ON (x.key = y.key) 
-JOIN srcpart z ON (x.value = z.value and z.ds='2008-04-08' and z.hr=12)
-PREHOOK: type: QUERY
-PREHOOK: Input: default@src
-PREHOOK: Input: default@src1
-PREHOOK: Input: default@srcpart
-PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
-PREHOOK: Output: default@sr
-POSTHOOK: query: INSERT OVERWRITE TABLE sr
-SELECT x.key,x.value,y.key,y.value,z.key,z.value
-FROM src1 x 
-JOIN src y ON (x.key = y.key) 
-JOIN srcpart z ON (x.value = z.value and z.ds='2008-04-08' and z.hr=12)
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@src
-POSTHOOK: Input: default@src1
-POSTHOOK: Input: default@srcpart
-POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
-POSTHOOK: Output: default@sr
-POSTHOOK: Lineage: sr.k1 SIMPLE [(src1)x.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: sr.k2 SIMPLE [(src)y.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: sr.k3 SIMPLE [(srcpart)z.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: sr.v1 SIMPLE [(src1)x.FieldSchema(name:value, type:string, comment:default), ]
-POSTHOOK: Lineage: sr.v2 SIMPLE [(src)y.FieldSchema(name:value, type:string, comment:default), ]
-POSTHOOK: Lineage: sr.v3 SIMPLE [(srcpart)z.FieldSchema(name:value, type:string, comment:default), ]
-PREHOOK: query: INSERT OVERWRITE TABLE cs
-SELECT x.key,x.value,y.key,y.value,z.key,z.value
-FROM src1 x 
-JOIN src y ON (x.key = y.key) 
-JOIN srcpart z ON (x.value = z.value and z.ds='2008-04-08')
-PREHOOK: type: QUERY
-PREHOOK: Input: default@src
-PREHOOK: Input: default@src1
-PREHOOK: Input: default@srcpart
-PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
-PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
-PREHOOK: Output: default@cs
-POSTHOOK: query: INSERT OVERWRITE TABLE cs
-SELECT x.key,x.value,y.key,y.value,z.key,z.value
-FROM src1 x 
-JOIN src y ON (x.key = y.key) 
-JOIN srcpart z ON (x.value = z.value and z.ds='2008-04-08')
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@src
-POSTHOOK: Input: default@src1
-POSTHOOK: Input: default@srcpart
-POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
-POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
-POSTHOOK: Output: default@cs
-POSTHOOK: Lineage: cs.k1 SIMPLE [(src1)x.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: cs.k2 SIMPLE [(src)y.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: cs.k3 SIMPLE [(srcpart)z.FieldSchema(name:key, type:string, comment:default), ]
-POSTHOOK: Lineage: cs.v1 SIMPLE [(src1)x.FieldSchema(name:value, type:string, comment:default), ]
-POSTHOOK: Lineage: cs.v2 SIMPLE [(src)y.FieldSchema(name:value, type:string, comment:default), ]
-POSTHOOK: Lineage: cs.v3 SIMPLE [(srcpart)z.FieldSchema(name:value, type:string, comment:default), ]
-PREHOOK: query: ANALYZE TABLE ss COMPUTE STATISTICS
-PREHOOK: type: QUERY
-PREHOOK: Input: default@ss
-PREHOOK: Output: default@ss
-POSTHOOK: query: ANALYZE TABLE ss COMPUTE STATISTICS
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@ss
-POSTHOOK: Output: default@ss
-PREHOOK: query: ANALYZE TABLE ss COMPUTE STATISTICS FOR COLUMNS k1,v1,k2,v2,k3,v3
-PREHOOK: type: QUERY
-PREHOOK: Input: default@ss
-#### A masked pattern was here ####
-POSTHOOK: query: ANALYZE TABLE ss COMPUTE STATISTICS FOR COLUMNS k1,v1,k2,v2,k3,v3
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@ss
-#### A masked pattern was here ####
-PREHOOK: query: ANALYZE TABLE sr COMPUTE STATISTICS
-PREHOOK: type: QUERY
-PREHOOK: Input: default@sr
-PREHOOK: Output: default@sr
-POSTHOOK: query: ANALYZE TABLE sr COMPUTE STATISTICS
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@sr
-POSTHOOK: Output: default@sr
-PREHOOK: query: ANALYZE TABLE sr COMPUTE STATISTICS FOR COLUMNS k1,v1,k2,v2,k3,v3
-PREHOOK: type: QUERY
-PREHOOK: Input: default@sr
-#### A masked pattern was here ####
-POSTHOOK: query: ANALYZE TABLE sr COMPUTE STATISTICS FOR COLUMNS k1,v1,k2,v2,k3,v3
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@sr
-#### A masked pattern was here ####
-PREHOOK: query: ANALYZE TABLE cs COMPUTE STATISTICS
-PREHOOK: type: QUERY
-PREHOOK: Input: default@cs
-PREHOOK: Output: default@cs
-POSTHOOK: query: ANALYZE TABLE cs COMPUTE STATISTICS
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@cs
-POSTHOOK: Output: default@cs
-PREHOOK: query: ANALYZE TABLE cs COMPUTE STATISTICS FOR COLUMNS k1,v1,k2,v2,k3,v3
-PREHOOK: type: QUERY
-PREHOOK: Input: default@cs
-#### A masked pattern was here ####
-POSTHOOK: query: ANALYZE TABLE cs COMPUTE STATISTICS FOR COLUMNS k1,v1,k2,v2,k3,v3
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@cs
-#### A masked pattern was here ####
-PREHOOK: query: SELECT x.key, z.value, y.value
-FROM src1 x JOIN src y ON (x.key = y.key) 
-JOIN srcpart z ON (x.value = z.value and z.ds='2008-04-08' and z.hr=11)
-PREHOOK: type: QUERY
-PREHOOK: Input: default@src
-PREHOOK: Input: default@src1
-PREHOOK: Input: default@srcpart
-PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
-#### A masked pattern was here ####
-POSTHOOK: query: SELECT x.key, z.value, y.value
-FROM src1 x JOIN src y ON (x.key = y.key) 
-JOIN srcpart z ON (x.value = z.value and z.ds='2008-04-08' and z.hr=11)
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@src
-POSTHOOK: Input: default@src1
-POSTHOOK: Input: default@srcpart
-POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
-#### A masked pattern was here ####
-PREHOOK: query: explain analyze
-SELECT x.key, z.value, y.value
-FROM src1 x JOIN src y ON (x.key = y.key) 
-JOIN srcpart z ON (x.value = z.value and z.ds='2008-04-08' and z.hr=11)
-PREHOOK: type: QUERY
-POSTHOOK: query: explain analyze
-SELECT x.key, z.value, y.value
-FROM src1 x JOIN src y ON (x.key = y.key) 
-JOIN srcpart z ON (x.value = z.value and z.ds='2008-04-08' and z.hr=11)
-POSTHOOK: type: QUERY
-Plan optimized by CBO.
-
-Vertex dependency in root stage
-Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
-Reducer 3 <- Map 5 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
-
-Stage-0
-  Fetch Operator
-    limit:-1
-    Stage-1
-      Reducer 3
-      File Output Operator [FS_16]
-        Select Operator [SEL_15] (rows=141/85 width=268)
-          Output:["_col0","_col1","_col2"]
-          Merge Join Operator [MERGEJOIN_26] (rows=141/85 width=268)
-            Conds:RS_12._col3=RS_13._col0(Inner),Output:["_col0","_col3","_col6"]
-          <-Map 5 [SIMPLE_EDGE]
-            SHUFFLE [RS_13]
-              PartitionCols:_col0
-              Select Operator [SEL_8] (rows=500/500 width=178)
-                Output:["_col0","_col1"]
-                Filter Operator [FIL_24] (rows=500/500 width=178)
-                  predicate:key is not null
-                  TableScan [TS_6] (rows=500/500 width=178)
-                    default@src,y,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
-          <-Reducer 2 [SIMPLE_EDGE]
-            SHUFFLE [RS_12]
-              PartitionCols:_col3
-              Merge Join Operator [MERGEJOIN_25] (rows=58/41 width=177)
-                Conds:RS_9._col0=RS_10._col1(Inner),Output:["_col0","_col3"]
-              <-Map 1 [SIMPLE_EDGE]
-                SHUFFLE [RS_9]
-                  PartitionCols:_col0
-                  Select Operator [SEL_2] (rows=500/500 width=369)
-                    Output:["_col0"]
-                    Filter Operator [FIL_22] (rows=500/500 width=91)
-                      predicate:value is not null
-                      TableScan [TS_0] (rows=500/500 width=91)
-                        default@srcpart,z,Tbl:COMPLETE,Col:COMPLETE,Output:["value"]
-              <-Map 4 [SIMPLE_EDGE]
-                SHUFFLE [RS_10]
-                  PartitionCols:_col1
-                  Select Operator [SEL_5] (rows=25/25 width=175)
-                    Output:["_col0","_col1"]
-                    Filter Operator [FIL_23] (rows=25/25 width=175)
-                      predicate:(key is not null and value is not null)
-                      TableScan [TS_3] (rows=25/25 width=175)
-                        default@src1,x,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
-
-PREHOOK: query: select 
-ss.k1,sr.k2,cs.k3,count(ss.v1),count(sr.v2),count(cs.v3)
-FROM 
-ss,sr,cs,src d1,src d2,src d3,src1,srcpart
-where
-    ss.k1 = d1.key 
-and sr.k1 = d2.key 
-and cs.k1 = d3.key 
-and ss.k2 = sr.k2
-and ss.k3 = sr.k3
-and ss.v1 = src1.value
-and ss.v2 = srcpart.value
-and sr.v2 = cs.v2
-and sr.v3 = cs.v3
-and ss.v3='ssv3'
-and sr.v1='srv1'
-and src1.key = 'src1key'
-and srcpart.key = 'srcpartkey'
-and d1.value = 'd1value'
-and d2.value in ('2000Q1','2000Q2','2000Q3')
-and d3.value in ('2000Q1','2000Q2','2000Q3')
-group by 
-ss.k1,sr.k2,cs.k3
-order by 
-ss.k1,sr.k2,cs.k3
-limit 100
-PREHOOK: type: QUERY
-PREHOOK: Input: default@cs
-PREHOOK: Input: default@sr
-PREHOOK: Input: default@src
-PREHOOK: Input: default@src1
-PREHOOK: Input: default@srcpart
-PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
-PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
-PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
-PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
-PREHOOK: Input: default@ss
-#### A masked pattern was here ####
-POSTHOOK: query: select 
-ss.k1,sr.k2,cs.k3,count(ss.v1),count(sr.v2),count(cs.v3)
-FROM 
-ss,sr,cs,src d1,src d2,src d3,src1,srcpart
-where
-    ss.k1 = d1.key 
-and sr.k1 = d2.key 
-and cs.k1 = d3.key 
-and ss.k2 = sr.k2
-and ss.k3 = sr.k3
-and ss.v1 = src1.value
-and ss.v2 = srcpart.value
-and sr.v2 = cs.v2
-and sr.v3 = cs.v3
-and ss.v3='ssv3'
-and sr.v1='srv1'
-and src1.key = 'src1key'
-and srcpart.key = 'srcpartkey'
-and d1.value = 'd1value'
-and d2.value in ('2000Q1','2000Q2','2000Q3')
-and d3.value in ('2000Q1','2000Q2','2000Q3')
-group by 
-ss.k1,sr.k2,cs.k3
-order by 
-ss.k1,sr.k2,cs.k3
-limit 100
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@cs
-POSTHOOK: Input: default@sr
-POSTHOOK: Input: default@src
-POSTHOOK: Input: default@src1
-POSTHOOK: Input: default@srcpart
-POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
-POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
-POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
-POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
-POSTHOOK: Input: default@ss
-#### A masked pattern was here ####
-PREHOOK: query: explain analyze
-select 
-ss.k1,sr.k2,cs.k3,count(ss.v1),count(sr.v2),count(cs.v3)
-FROM 
-ss,sr,cs,src d1,src d2,src d3,src1,srcpart
-where
-    ss.k1 = d1.key 
-and sr.k1 = d2.key 
-and cs.k1 = d3.key 
-and ss.k2 = sr.k2
-and ss.k3 = sr.k3
-and ss.v1 = src1.value
-and ss.v2 = srcpart.value
-and sr.v2 = cs.v2
-and sr.v3 = cs.v3
-and ss.v3='ssv3'
-and sr.v1='srv1'
-and src1.key = 'src1key'
-and srcpart.key = 'srcpartkey'
-and d1.value = 'd1value'
-and d2.value in ('2000Q1','2000Q2','2000Q3')
-and d3.value in ('2000Q1','2000Q2','2000Q3')
-group by 
-ss.k1,sr.k2,cs.k3
-order by 
-ss.k1,sr.k2,cs.k3
-limit 100
-PREHOOK: type: QUERY
-POSTHOOK: query: explain analyze
-select 
-ss.k1,sr.k2,cs.k3,count(ss.v1),count(sr.v2),count(cs.v3)
-FROM 
-ss,sr,cs,src d1,src d2,src d3,src1,srcpart
-where
-    ss.k1 = d1.key 
-and sr.k1 = d2.key 
-and cs.k1 = d3.key 
-and ss.k2 = sr.k2
-and ss.k3 = sr.k3
-and ss.v1 = src1.value
-and ss.v2 = srcpart.value
-and sr.v2 = cs.v2
-and sr.v3 = cs.v3
-and ss.v3='ssv3'
-and sr.v1='srv1'
-and src1.key = 'src1key'
-and srcpart.key = 'srcpartkey'
-and d1.value = 'd1value'
-and d2.value in ('2000Q1','2000Q2','2000Q3')
-and d3.value in ('2000Q1','2000Q2','2000Q3')
-group by 
-ss.k1,sr.k2,cs.k3
-order by 
-ss.k1,sr.k2,cs.k3
-limit 100
-POSTHOOK: type: QUERY
-Plan optimized by CBO.
-
-Vertex dependency in root stage
-Reducer 10 <- Reducer 9 (SIMPLE_EDGE)
-Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 11 (SIMPLE_EDGE)
-Reducer 3 <- Map 12 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
-Reducer 4 <- Map 13 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
-Reducer 5 <- Map 14 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-Reducer 6 <- Map 15 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
-Reducer 7 <- Map 16 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
-Reducer 8 <- Map 17 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE)
-Reducer 9 <- Reducer 8 (SIMPLE_EDGE)
-
-Stage-0
-  Fetch Operator
-    limit:100
-    Stage-1
-      Reducer 10
-      File Output Operator [FS_53]
-        Limit [LIM_52] (rows=100/0 width=18)
-          Number of rows:100
-          Select Operator [SEL_51] (rows=550/0 width=18)
-            Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
-          <-Reducer 9 [SIMPLE_EDGE]
-            SHUFFLE [RS_50]
-              Group By Operator [GBY_48] (rows=550/0 width=18)
-                Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["count(VALUE._col0)","count(VALUE._col1)","count(VALUE._col2)"],keys:KEY._col0, KEY._col1, KEY._col2
-              <-Reducer 8 [SIMPLE_EDGE]
-                SHUFFLE [RS_47]
-                  PartitionCols:_col0, _col1, _col2
-                  Group By Operator [GBY_46] (rows=1100/0 width=18)
-                    Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["count(_col1)","count(_col9)","count(_col15)"],keys:_col0, _col8, _col14
-                    Select Operator [SEL_45] (rows=1100/0 width=18)
-                      Output:["_col0","_col8","_col14","_col1","_col9","_col15"]
-                      Merge Join Operator [MERGEJOIN_103] (rows=1100/0 width=18)
-                        Conds:RS_42._col3=RS_43._col1(Inner),Output:["_col0","_col1","_col8","_col9","_col14","_col15"]
-                      <-Map 17 [SIMPLE_EDGE]
-                        SHUFFLE [RS_43]
-                          PartitionCols:_col1
-                          Select Operator [SEL_23] (rows=1000/0 width=18)
-                            Output:["_col1"]
-                            Filter Operator [FIL_96] (rows=1000/0 width=18)
-                              predicate:((key = 'srcpartkey') and value is not null)
-                              TableScan [TS_21] (rows=2000/2000 width=18)
-                                default@srcpart,srcpart,Tbl:COMPLETE,Col:NONE,Output:["key","value"]
-                      <-Reducer 7 [SIMPLE_EDGE]
-                        SHUFFLE [RS_42]
-                          PartitionCols:_col3
-                          Merge Join Operator [MERGEJOIN_102] (rows=129/0 width=625)
-                            Conds:RS_39._col1=RS_40._col1(Inner),Output:["_col0","_col1","_col3","_col8","_col9","_col14","_col15"]
-                          <-Map 16 [SIMPLE_EDGE]
-                            SHUFFLE [RS_40]
-                              PartitionCols:_col1
-                              Select Operator [SEL_20] (rows=2/0 width=180)
-                                Output:["_col1"]
-                                Filter Operator [FIL_95] (rows=2/0 width=175)
-                                  predicate:((key = 'src1key') and value is not null)
-                                  TableScan [TS_18] (rows=25/25 width=175)
-                                    default@src1,src1,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
-                          <-Reducer 6 [SIMPLE_EDGE]
-                            SHUFFLE [RS_39]
-                              PartitionCols:_col1
-                              Merge Join Operator [MERGEJOIN_101] (rows=129/0 width=625)
-                                Conds:RS_36._col12=RS_37._col0(Inner),Output:["_col0","_col1","_col3","_col8","_col9","_col14","_col15"]
-                              <-Map 15 [SIMPLE_EDGE]
-                                SHUFFLE [RS_37]
-                                  PartitionCols:_col0
-                                  Select Operator [SEL_17] (rows=500/0 width=178)
-                                    Output:["_col0"]
-                                    Filter Operator [FIL_94] (rows=500/0 width=178)
-                                      predicate:((value) IN ('2000Q1', '2000Q2', '2000Q3') and key is not null)
-                                      TableScan [TS_15] (rows=500/500 width=178)
-                                        default@src,d3,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
-                              <-Reducer 5 [SIMPLE_EDGE]
-                                SHUFFLE [RS_36]
-                                  PartitionCols:_col12
-                                  Merge Join Operator [MERGEJOIN_100] (rows=53/0 width=712)
-                                    Conds:RS_33._col6=RS_34._col0(Inner),Output:["_col0","_col1","_col3","_col8","_col9","_col12","_col14","_col15"]
-                                  <-Map 14 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_34]
-                                      PartitionCols:_col0
-                                      Select Operator [SEL_14] (rows=500/0 width=178)
-                                        Output:["_col0"]
-                                        Filter Operator [FIL_93] (rows=500/0 width=178)
-                                          predicate:((value) IN ('2000Q1', '2000Q2', '2000Q3') and key is not null)
-                                          TableScan [TS_12] (rows=500/500 width=178)
-                                            default@src,d2,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
-                                  <-Reducer 4 [SIMPLE_EDGE]
-                                    SHUFFLE [RS_33]
-                                      PartitionCols:_col6
-                                      Merge Join Operator [MERGEJOIN_99] (rows=22/0 width=799)
-                                        Conds:RS_30._col0=RS_31._col0(Inner),Output:["_col0","_col1","_col3","_col6","_col8","_col9","_col12","_col14","_col15"]
-                                      <-Map 13 [SIMPLE_EDGE]
-                                        SHUFFLE [RS_31]
-                                          PartitionCols:_col0
-                                          Select Operator [SEL_11] (rows=2/0 width=178)
-                                            Output:["_col0"]
-                                            Filter Operator [FIL_92] (rows=2/0 width=178)
-                                              predicate:((value = 'd1value') and key is not null)
-                                              TableScan [TS_9] (rows=500/500 width=178)
-                                                default@src,d1,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
-                                      <-Reducer 3 [SIMPLE_EDGE]
-                                        SHUFFLE [RS_30]
-                                          PartitionCols:_col0
-                                          Merge Join Operator [MERGEJOIN_98] (rows=22/0 width=799)
-                                            Conds:RS_27._col9, _col11=RS_28._col1, _col3(Inner),Output:["_col0","_col1","_col3","_col6","_col8","_col9","_col12","_col14","_col15"]
-                                          <-Map 12 [SIMPLE_EDGE]
-                                            SHUFFLE [RS_28]
-                                              PartitionCols:_col1, _col3
-                                              Select Operator [SEL_8] (rows=170/170 width=356)
-                                                Output:["_col0","_col1","_col2","_col3"]
-                                                Filter Operator [FIL_91] (rows=170/170 width=356)
-                                                  predicate:(v2 is not null and v3 is not null and k1 is not null)
-                                                  TableScan [TS_6] (rows=170/170 width=356)
-                                                    default@cs,cs,Tbl:COMPLETE,Col:COMPLETE,Output:["k1","v2","k3","v3"]
-                                          <-Reducer 2 [SIMPLE_EDGE]
-                                            SHUFFLE [RS_27]
-                                              PartitionCols:_col9, _col11
-                                              Merge Join Operator [MERGEJOIN_97] (rows=16/0 width=625)
-                                                Conds:RS_24._col2, _col4=RS_25._col2, _col4(Inner),Output:["_col0","_col1","_col3","_col6","_col8","_col9","_col11"]
-                                              <-Map 1 [SIMPLE_EDGE]
-                                                SHUFFLE [RS_24]
-                                                  PartitionCols:_col2, _col4
-                                                  Select Operator [SEL_2] (rows=8/0 width=531)
-                                                    Output:["_col0","_col1","_col2","_col3","_col4"]
-                                                    Filter Operator [FIL_89] (rows=8/0 width=534)
-                                                      predicate:((v3 = 'ssv3') and k2 is not null and k3 is not null and k1 is not null and v1 is not null and v2 is not null)
-                                                      TableScan [TS_0] (rows=85/85 width=534)
-                                                        default@ss,ss,Tbl:COMPLETE,Col:COMPLETE,Output:["k1","v1","k2","v2","k3","v3"]
-                                              <-Map 11 [SIMPLE_EDGE]
-                                                SHUFFLE [RS_25]
-                                                  PartitionCols:_col2, _col4
-                                                  Select Operator [SEL_5] (rows=8/0 width=531)
-                                                    Output:["_col0","_col2","_col3","_col4","_col5"]
-                                                    Filter Operator [FIL_90] (rows=8/0 width=534)
-                                                      predicate:((v1 = 'srv1') and k2 is not null and k3 is not null and v2 is not null and v3 is not null and k1 is not null)
-                                                      TableScan [TS_3] (rows=85/85 width=534)
-                                                        default@sr,sr,Tbl:COMPLETE,Col:COMPLETE,Output:["k1","v1","k2","v2","k3","v3"]
-
 PREHOOK: query: SELECT x.key, z.value, y.value
 FROM src1 x JOIN src y ON (x.key = y.key) 
 JOIN (select * from src1 union select * from src)z ON (x.value = z.value)
@@ -1055,862 +537,45 @@ Stage-0
                                                 TableScan [TS_3] (rows=500/500 width=178)
                                                   Output:["key","value"]
 
-PREHOOK: query: SELECT x.key, z.value, y.value
-FROM src1 x JOIN src y ON (x.key = y.key) 
-JOIN srcpart z ON (x.value = z.value and z.ds='2008-04-08' and z.hr=11)
-PREHOOK: type: QUERY
-PREHOOK: Input: default@src
-PREHOOK: Input: default@src1
-PREHOOK: Input: default@srcpart
-PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+PREHOOK: query: CREATE TABLE srcbucket_mapjoin(key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@srcbucket_mapjoin
+POSTHOOK: query: CREATE TABLE srcbucket_mapjoin(key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@srcbucket_mapjoin
+PREHOOK: query: CREATE TABLE tab_part (key int, value string) PARTITIONED BY(ds STRING) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@tab_part
+POSTHOOK: query: CREATE TABLE tab_part (key int, value string) PARTITIONED BY(ds STRING) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@tab_part
+PREHOOK: query: CREATE TABLE srcbucket_mapjoin_part (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@srcbucket_mapjoin_part
+POSTHOOK: query: CREATE TABLE srcbucket_mapjoin_part (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@srcbucket_mapjoin_part
+PREHOOK: query: load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin partition(ds='2008-04-08')
+PREHOOK: type: LOAD
 #### A masked pattern was here ####
-POSTHOOK: query: SELECT x.key, z.value, y.value
-FROM src1 x JOIN src y ON (x.key = y.key) 
-JOIN srcpart z ON (x.value = z.value and z.ds='2008-04-08' and z.hr=11)
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@src
-POSTHOOK: Input: default@src1
-POSTHOOK: Input: default@srcpart
-POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
+PREHOOK: Output: default@srcbucket_mapjoin
+POSTHOOK: query: load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin partition(ds='2008-04-08')
+POSTHOOK: type: LOAD
 #### A masked pattern was here ####
-PREHOOK: query: explain analyze
-SELECT x.key, z.value, y.value
-FROM src1 x JOIN src y ON (x.key = y.key) 
-JOIN srcpart z ON (x.value = z.value and z.ds='2008-04-08' and z.hr=11)
-PREHOOK: type: QUERY
-POSTHOOK: query: explain analyze
-SELECT x.key, z.value, y.value
-FROM src1 x JOIN src y ON (x.key = y.key) 
-JOIN srcpart z ON (x.value = z.value and z.ds='2008-04-08' and z.hr=11)
-POSTHOOK: type: QUERY
-Plan optimized by CBO.
-
-Vertex dependency in root stage
-Map 1 <- Map 2 (BROADCAST_EDGE), Map 3 (BROADCAST_EDGE)
-
-Stage-0
-  Fetch Operator
-    limit:-1
-    Stage-1
-      Map 1
-      File Output Operator [FS_16]
-        Select Operator [SEL_15] (rows=605/85 width=10)
-          Output:["_col0","_col1","_col2"]
-          Map Join Operator [MAPJOIN_26] (rows=605/85 width=10)
-            Conds:MAPJOIN_25._col3=RS_13._col0(Inner),HybridGraceHashJoin:true,Output:["_col0","_col3","_col6"]
-          <-Map 3 [BROADCAST_EDGE]
-            BROADCAST [RS_13]
-              PartitionCols:_col0
-              Select Operator [SEL_8] (rows=500/500 width=10)
-                Output:["_col0","_col1"]
-                Filter Operator [FIL_24] (rows=500/500 width=10)
-                  predicate:key is not null
-                  TableScan [TS_6] (rows=500/500 width=10)
-                    default@src,y,Tbl:COMPLETE,Col:NONE,Output:["key","value"]
-          <-Map Join Operator [MAPJOIN_25] (rows=550/41 width=10)
-              Conds:SEL_2._col0=RS_10._col1(Inner),HybridGraceHashJoin:true,Output:["_col0","_col3"]
-            <-Map 2 [BROADCAST_EDGE]
-              BROADCAST [RS_10]
-                PartitionCols:_col1
-                Select Operator [SEL_5] (rows=25/25 width=7)
-                  Output:["_col0","_col1"]
-                  Filter Operator [FIL_23] (rows=25/25 width=7)
-                    predicate:(key is not null and value is not null)
-                    TableScan [TS_3] (rows=25/25 width=7)
-                      default@src1,x,Tbl:COMPLETE,Col:NONE,Output:["key","value"]
-            <-Select Operator [SEL_2] (rows=500/500 width=10)
-                Output:["_col0"]
-                Filter Operator [FIL_22] (rows=500/500 width=10)
-                  predicate:value is not null
-                  TableScan [TS_0] (rows=500/500 width=10)
-                    default@srcpart,z,Tbl:COMPLETE,Col:NONE,Output:["value"]
-
-PREHOOK: query: select 
-ss.k1,sr.k2,cs.k3,count(ss.v1),count(sr.v2),count(cs.v3)
-FROM 
-ss,sr,cs,src d1,src d2,src d3,src1,srcpart
-where
-    ss.k1 = d1.key 
-and sr.k1 = d2.key 
-and cs.k1 = d3.key 
-and ss.k2 = sr.k2
-and ss.k3 = sr.k3
-and ss.v1 = src1.value
-and ss.v2 = srcpart.value
-and sr.v2 = cs.v2
-and sr.v3 = cs.v3
-and ss.v3='ssv3'
-and sr.v1='srv1'
-and src1.key = 'src1key'
-and srcpart.key = 'srcpartkey'
-and d1.value = 'd1value'
-and d2.value in ('2000Q1','2000Q2','2000Q3')
-and d3.value in ('2000Q1','2000Q2','2000Q3')
-group by 
-ss.k1,sr.k2,cs.k3
-order by 
-ss.k1,sr.k2,cs.k3
-limit 100
-PREHOOK: type: QUERY
-PREHOOK: Input: default@cs
-PREHOOK: Input: default@sr
-PREHOOK: Input: default@src
-PREHOOK: Input: default@src1
-PREHOOK: Input: default@srcpart
-PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
-PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
-PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
-PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
-PREHOOK: Input: default@ss
+POSTHOOK: Output: default@srcbucket_mapjoin
+POSTHOOK: Output: default@srcbucket_mapjoin@ds=2008-04-08
+PREHOOK: query: load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin partition(ds='2008-04-08')
+PREHOOK: type: LOAD
 #### A masked pattern was here ####
-POSTHOOK: query: select 
-ss.k1,sr.k2,cs.k3,count(ss.v1),count(sr.v2),count(cs.v3)
-FROM 
-ss,sr,cs,src d1,src d2,src d3,src1,srcpart
-where
-    ss.k1 = d1.key 
-and sr.k1 = d2.key 
-and cs.k1 = d3.key 
-and ss.k2 = sr.k2
-and ss.k3 = sr.k3
-and ss.v1 = src1.value
-and ss.v2 = srcpart.value
-and sr.v2 = cs.v2
-and sr.v3 = cs.v3
-and ss.v3='ssv3'
-and sr.v1='srv1'
-and src1.key = 'src1key'
-and srcpart.key = 'srcpartkey'
-and d1.value = 'd1value'
-and d2.value in ('2000Q1','2000Q2','2000Q3')
-and d3.value in ('2000Q1','2000Q2','2000Q3')
-group by 
-ss.k1,sr.k2,cs.k3
-order by 
-ss.k1,sr.k2,cs.k3
-limit 100
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@cs
-POSTHOOK: Input: default@sr
-POSTHOOK: Input: default@src
-POSTHOOK: Input: default@src1
-POSTHOOK: Input: default@srcpart
-POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
-POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
-POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11
-POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12
-POSTHOOK: Input: default@ss
-#### A masked pattern was here ####
-PREHOOK: query: explain analyze
-select 
-ss.k1,sr.k2,cs.k3,count(ss.v1),count(sr.v2),count(cs.v3)
-FROM 
-ss,sr,cs,src d1,src d2,src d3,src1,srcpart
-where
-    ss.k1 = d1.key 
-and sr.k1 = d2.key 
-and cs.k1 = d3.key 
-and ss.k2 = sr.k2
-and ss.k3 = sr.k3
-and ss.v1 = src1.value
-and ss.v2 = srcpart.value
-and sr.v2 = cs.v2
-and sr.v3 = cs.v3
-and ss.v3='ssv3'
-and sr.v1='srv1'
-and src1.key = 'src1key'
-and srcpart.key = 'srcpartkey'
-and d1.value = 'd1value'
-and d2.value in ('2000Q1','2000Q2','2000Q3')
-and d3.value in ('2000Q1','2000Q2','2000Q3')
-group by 
-ss.k1,sr.k2,cs.k3
-order by 
-ss.k1,sr.k2,cs.k3
-limit 100
-PREHOOK: type: QUERY
-POSTHOOK: query: explain analyze
-select 
-ss.k1,sr.k2,cs.k3,count(ss.v1),count(sr.v2),count(cs.v3)
-FROM 
-ss,sr,cs,src d1,src d2,src d3,src1,srcpart
-where
-    ss.k1 = d1.key 
-and sr.k1 = d2.key 
-and cs.k1 = d3.key 
-and ss.k2 = sr.k2
-and ss.k3 = sr.k3
-and ss.v1 = src1.value
-and ss.v2 = srcpart.value
-and sr.v2 = cs.v2
-and sr.v3 = cs.v3
-and ss.v3='ssv3'
-and sr.v1='srv1'
-and src1.key = 'src1key'
-and srcpart.key = 'srcpartkey'
-and d1.value = 'd1value'
-and d2.value in ('2000Q1','2000Q2','2000Q3')
-and d3.value in ('2000Q1','2000Q2','2000Q3')
-group by 
-ss.k1,sr.k2,cs.k3
-order by 
-ss.k1,sr.k2,cs.k3
-limit 100
-POSTHOOK: type: QUERY
-Plan optimized by CBO.
-
-Vertex dependency in root stage
-Map 10 <- Map 9 (BROADCAST_EDGE)
-Map 2 <- Map 1 (BROADCAST_EDGE)
-Map 3 <- Map 10 (BROADCAST_EDGE), Map 2 (BROADCAST_EDGE), Map 6 (BROADCAST_EDGE), Map 7 (BROADCAST_EDGE), Map 8 (BROADCAST_EDGE)
-Reducer 4 <- Map 3 (SIMPLE_EDGE)
-Reducer 5 <- Reducer 4 (SIMPLE_EDGE)
-
-Stage-0
-  Fetch Operator
-    limit:100
-    Stage-1
-      Reducer 5
-      File Output Operator [FS_55]
-        Limit [LIM_54] (rows=100/0 width=10)
-          Number of rows:100
-          Select Operator [SEL_53] (rows=805/0 width=10)
-            Output:["_col0","_col1","_col2","_col3","_col4","_col5"]
-          <-Reducer 4 [SIMPLE_EDGE]
-            SHUFFLE [RS_52]
-              Group By Operator [GBY_50] (rows=805/0 width=10)
-                Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["count(VALUE._col0)","count(VALUE._col1)","count(VALUE._col2)"],keys:KEY._col0, KEY._col1, KEY._col2
-              <-Map 3 [SIMPLE_EDGE]
-                SHUFFLE [RS_49]
-                  PartitionCols:_col0, _col1, _col2
-                  Group By Operator [GBY_48] (rows=1610/0 width=10)
-                    Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["count(_col13)","count(_col21)","count(_col3)"],keys:_col12, _col20, _col2
-                    Select Operator [SEL_47] (rows=1610/0 width=10)
-                      Output:["_col12","_col20","_col2","_col13","_col21","_col3"]
-                      Map Join Operator [MAPJOIN_97] (rows=1610/0 width=10)
-                        Conds:RS_44._col1, _col3=SEL_40._col15, _col17(Inner),HybridGraceHashJoin:true,Output:["_col2","_col3","_col12","_col13","_col20","_col21"]
-                      <-Map 2 [BROADCAST_EDGE]
-                        BROADCAST [RS_44]
-                          PartitionCols:_col1, _col3
-                          Map Join Operator [MAPJOIN_91] (rows=275/0 width=10)
-                            Conds:RS_41._col0=SEL_5._col0(Inner),HybridGraceHashJoin:true,Output:["_col1","_col2","_col3"]
-                          <-Map 1 [BROADCAST_EDGE]
-                            BROADCAST [RS_41]
-                              PartitionCols:_col0
-                              Select Operator [SEL_2] (rows=170/170 width=34)
-                                Output:["_col0","_col1","_col2","_col3"]
-                                Filter Operator [FIL_83] (rows=170/170 width=34)
-                                  predicate:(v2 is not null and v3 is not null and k1 is not null)
-                                  TableScan [TS_0] (rows=170/170 width=34)
-                                    default@cs,cs,Tbl:COMPLETE,Col:NONE,Output:["k1","v2","k3","v3"]
-                          <-Select Operator [SEL_5] (rows=250/0 width=10)
-                              Output:["_col0"]
-                              Filter Operator [FIL_84] (rows=250/0 width=10)
-                                predicate:((value) IN ('2000Q1', '2000Q2', '2000Q3') and key is not null)
-                                TableScan [TS_3] (rows=500/500 width=10)
-                                  default@src,d3,Tbl:COMPLETE,Col:NONE,Output:["key","value"]
-                      <-Select Operator [SEL_40] (rows=1464/0 width=10)
-                          Output:["_col14","_col15","_col17","_col6","_col7"]
-                          Map Join Operator [MAPJOIN_96] (rows=1464/0 width=10)
-                            Conds:MAPJOIN_94._col6, _col4=RS_38._col4, _col2(Inner),HybridGraceHashJoin:true,Output:["_col2","_col3","_col14","_col15","_col17"]
-                          <-Map 10 [BROADCAST_EDGE]
-                            BROADCAST [RS_38]
-                              PartitionCols:_col4, _col2
-                              Map Join Operator [MAPJOIN_95] (rows=275/0 width=10)
-                                Conds:RS_24._col0=SEL_23._col0(Inner),HybridGraceHashJoin:true,Output:["_col2","_col3","_col4","_col5"]
-                              <-Map 9 [BROADCAST_EDGE]
-                                BROADCAST [RS_24]
-                                  PartitionCols:_col0
-                                  Select Operator [SEL_20] (rows=42/0 width=34)
-                                    Output:["_col0","_col2","_col3","_col4","_col5"]
-                                    Filter Operator [FIL_89] (rows=42/0 width=34)
-                                      predicate:((v1 = 'srv1') and k2 is not null and k3 is not null and v2 is not null and v3 is not null and k1 is not null)
-                                      TableScan [TS_18] (rows=85/85 width=34)
-                                        default@sr,sr,Tbl:COMPLETE,Col:NONE,Output:["k1","v1","k2","v2","k3","v3"]
-                              <-Select Operator [SEL_23] (rows=250/0 width=10)
-                                  Output:["_col0"]
-                                  Filter Operator [FIL_90] (rows=250/0 width=10)
-                                    predicate:((value) IN ('2000Q1', '2000Q2', '2000Q3') and key is not null)
-                                    TableScan [TS_21] (rows=500/500 width=10)
-                                      default@src,d2,Tbl:COMPLETE,Col:NONE,Output:["key","value"]
-                          <-Map Join Operator [MAPJOIN_94] (rows=1331/0 width=10)
-                              Conds:MAPJOIN_93._col3=RS_35._col1(Inner),HybridGraceHashJoin:true,Output:["_col2","_col3","_col4","_col6"]
-                            <-Map 8 [BROADCAST_EDGE]
-                              BROADCAST [RS_35]
-                                PartitionCols:_col1
-                                Select Operator [SEL_17] (rows=12/0 width=7)
-                                  Output:["_col1"]
-                                  Filter Operator [FIL_88] (rows=12/0 width=7)
-                                    predicate:((key = 'src1key') and value is not null)
-                                    TableScan [TS_15] (rows=25/25 width=7)
-                                      default@src1,src1,Tbl:COMPLETE,Col:NONE,Output:["key","value"]
-                            <-Map Join Operator [MAPJOIN_93] (rows=1210/0 width=10)
-                                Conds:MAPJOIN_92._col2=RS_32._col0(Inner),HybridGraceHashJoin:true,Output:["_col2","_col3","_col4","_col6"]
-                              <-Map 7 [BROADCAST_EDGE]
-                                BROADCAST [RS_32]
-                                  PartitionCols:_col0
-                                  Select Operator [SEL_14] (rows=250/0 width=10)
-                                    Output:["_col0"]
-                                    Filter Operator [FIL_87] (rows=250/0 width=10)
-                                      predicate:((value = 'd1value') and key is not null)
-                                      TableScan [TS_12] (rows=500/500 width=10)
-                                        default@src,d1,Tbl:COMPLETE,Col:NONE,Output:["key","value"]
-                              <-Map Join Operator [MAPJOIN_92] (rows=1100/0 width=10)
-                                  Conds:SEL_8._col1=RS_29._col3(Inner),HybridGraceHashJoin:true,Output:["_col2","_col3","_col4","_col6"]
-                                <-Map 6 [BROADCAST_EDGE]
-                                  BROADCAST [RS_29]
-                                    PartitionCols:_col3
-                                    Select Operator [SEL_11] (rows=42/0 width=34)
-                                      Output:["_col0","_col1","_col2","_col3","_col4"]
-                                      Filter Operator [FIL_86] (rows=42/0 width=34)
-                                        predicate:((v3 = 'ssv3') and k2 is not null and k3 is not null and k1 is not null and v1 is not null and v2 is not null)
-                                        TableScan [TS_9] (rows=85/85 width=34)
-                                          default@ss,ss,Tbl:COMPLETE,Col:NONE,Output:["k1","v1","k2","v2","k3","v3"]
-                                <-Select Operator [SEL_8] (rows=1000/0 width=10)
-                                    Output:["_col1"]
-                                    Filter Operator [FIL_85] (rows=1000/0 width=10)
-                                      predicate:((key = 'srcpartkey') and value is not null)
-                                      TableScan [TS_6] (rows=2000/2000 width=10)
-                                        default@srcpart,srcpart,Tbl:COMPLETE,Col:NONE,Output:["key","value"]
-
-PREHOOK: query: SELECT x.key, z.value, y.value
-FROM src1 x JOIN src y ON (x.key = y.key) 
-JOIN (select * from src1 union select * from src)z ON (x.value = z.value)
-union
-SELECT x.key, z.value, y.value
-FROM src1 x JOIN src y ON (x.key = y.key) 
-JOIN (select * from src1 union select * from src)z ON (x.value = z.value)
-PREHOOK: type: QUERY
-PREHOOK: Input: default@src
-PREHOOK: Input: default@src1
-#### A masked pattern was here ####
-POSTHOOK: query: SELECT x.key, z.value, y.value
-FROM src1 x JOIN src y ON (x.key = y.key) 
-JOIN (select * from src1 union select * from src)z ON (x.value = z.value)
-union
-SELECT x.key, z.value, y.value
-FROM src1 x JOIN src y ON (x.key = y.key) 
-JOIN (select * from src1 union select * from src)z ON (x.value = z.value)
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@src
-POSTHOOK: Input: default@src1
-#### A masked pattern was here ####
-PREHOOK: query: explain analyze
-SELECT x.key, z.value, y.value
-FROM src1 x JOIN src y ON (x.key = y.key) 
-JOIN (select * from src1 union select * from src)z ON (x.value = z.value)
-union
-SELECT x.key, z.value, y.value
-FROM src1 x JOIN src y ON (x.key = y.key) 
-JOIN (select * from src1 union select * from src)z ON (x.value = z.value)
-PREHOOK: type: QUERY
-POSTHOOK: query: explain analyze
-SELECT x.key, z.value, y.value
-FROM src1 x JOIN src y ON (x.key = y.key) 
-JOIN (select * from src1 union select * from src)z ON (x.value = z.value)
-union
-SELECT x.key, z.value, y.value
-FROM src1 x JOIN src y ON (x.key = y.key) 
-JOIN (select * from src1 union select * from src)z ON (x.value = z.value)
-POSTHOOK: type: QUERY
-Plan optimized by CBO.
-
-Vertex dependency in root stage
-Map 1 <- Union 2 (CONTAINS)
-Map 12 <- Union 10 (CONTAINS)
-Map 6 <- Union 2 (CONTAINS)
-Map 9 <- Union 10 (CONTAINS)
-Reducer 11 <- Map 13 (BROADCAST_EDGE), Map 14 (BROADCAST_EDGE), Union 10 (SIMPLE_EDGE), Union 4 (CONTAINS)
-Reducer 3 <- Map 7 (BROADCAST_EDGE), Map 8 (BROADCAST_EDGE), Union 2 (SIMPLE_EDGE), Union 4 (CONTAINS)
-Reducer 5 <- Union 4 (SIMPLE_EDGE)
-
-Stage-0
-  Fetch Operator
-    limit:-1
-    Stage-1
-      Reducer 5
-      File Output Operator [FS_59]
-        Group By Operator [GBY_57] (rows=550/15 width=10)
-          Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-        <-Union 4 [SIMPLE_EDGE]
-          <-Reducer 11 [CONTAINS]
-            Reduce Output Operator [RS_56]
-              PartitionCols:_col0, _col1
-              Group By Operator [GBY_55] (rows=1100/30 width=10)
-                Output:["_col0","_col1"],keys:_col0, _col1
-                Select Operator [SEL_51] (rows=550/61 width=10)
-                  Output:["_col0","_col1"]
-                  Map Join Operator [MAPJOIN_85] (rows=550/61 width=10)
-                    Conds:MAPJOIN_84._col2=RS_49._col0(Inner),HybridGraceHashJoin:true,Output:["_col1","_col2"]
-                  <-Map 14 [BROADCAST_EDGE]
-                    BROADCAST [RS_49]
-                      PartitionCols:_col0
-                      Select Operator [SEL_44] (rows=500/500 width=10)
-                        Output:["_col0"]
-                        Filter Operator [FIL_81] (rows=500/500 width=10)
-                          predicate:key is not null
-                          TableScan [TS_42] (rows=500/500 width=10)
-                            default@src,y,Tbl:COMPLETE,Col:NONE,Output:["key"]
-                  <-Map Join Operator [MAPJOIN_84] (rows=288/52 width=10)
-                      Conds:SEL_38._col1=RS_46._col1(Inner),HybridGraceHashJoin:true,Output:["_col1","_col2"]
-                    <-Map 13 [BROADCAST_EDGE]
-                      BROADCAST [RS_46]
-                        PartitionCols:_col1
-                        Select Operator [SEL_41] (rows=25/25 width=7)
-                          Output:["_col0","_col1"]
-                          Filter Operator [FIL_80] (rows=25/25 width=7)
-                            predicate:(key is not null and value is not null)
-                            TableScan [TS_39] (rows=25/25 width=7)
-                              default@src1,x,Tbl:COMPLETE,Col:NONE,Output:["key","value"]
-                    <-Select Operator [SEL_38] (rows=262/319 width=10)
-                        Output:["_col1"]
-                        Group By Operator [GBY_37] (rows=262/319 width=10)
-                          Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-                        <-Union 10 [SIMPLE_EDGE]
-                          <-Map 12 [CONTAINS]
-                            Reduce Output Operator [RS_36]
-                              PartitionCols:_col0, _col1
-                              Group By Operator [GBY_35] (rows=525/331 width=10)
-                                Output:["_col0","_col1"],keys:_col1, _col0
-                                Select Operator [SEL_31] (rows=500/500 width=10)
-                                  Output:["_col0","_col1"]
-                                  Filter Operator [FIL_79] (rows=500/500 width=10)
-                                    predicate:value is not null
-                                    TableScan [TS_29] (rows=500/500 width=10)
-                                      Output:["key","value"]
-                          <-Map 9 [CONTAINS]
-                            Reduce Output Operator [RS_36]
-                              PartitionCols:_col0, _col1
-                              Group By Operator [GBY_35] (rows=525/331 width=10)
-                                Output:["_col0","_col1"],keys:_col1, _col0
-                                Select Operator [SEL_28] (rows=25/25 width=7)
-                                  Output:["_col0","_col1"]
-                                  Filter Operator [FIL_78] (rows=25/25 width=7)
-                                    predicate:value is not null
-                                    TableScan [TS_26] (rows=25/25 width=7)
-                                      Output:["key","value"]
-          <-Reducer 3 [CONTAINS]
-            Reduce Output Operator [RS_56]
-              PartitionCols:_col0, _col1
-              Group By Operator [GBY_55] (rows=1100/30 width=10)
-                Output:["_col0","_col1"],keys:_col0, _col1
-                Select Operator [SEL_25] (rows=550/61 width=10)
-                  Output:["_col0","_col1"]
-                  Map Join Operator [MAPJOIN_83] (rows=550/61 width=10)
-                    Conds:MAPJOIN_82._col2=RS_23._col0(Inner),HybridGraceHashJoin:true,Output:["_col1","_col2"]
-                  <-Map 8 [BROADCAST_EDGE]
-                    BROADCAST [RS_23]
-                      PartitionCols:_col0
-                      Select Operator [SEL_18] (rows=500/500 width=10)
-                        Output:["_col0"]
-                        Filter Operator [FIL_77] (rows=500/500 width=10)
-                          predicate:key is not null
-                          TableScan [TS_16] (rows=500/500 width=10)
-                            default@src,y,Tbl:COMPLETE,Col:NONE,Output:["key"]
-                  <-Map Join Operator [MAPJOIN_82] (rows=288/52 width=10)
-                      Conds:SEL_12._col1=RS_20._col1(Inner),HybridGraceHashJoin:true,Output:["_col1","_col2"]
-                    <-Map 7 [BROADCAST_EDGE]
-                      BROADCAST [RS_20]
-                        PartitionCols:_col1
-                        Select Operator [SEL_15] (rows=25/25 width=7)
-                          Output:["_col0","_col1"]
-                          Filter Operator [FIL_76] (rows=25/25 width=7)
-                            predicate:(key is not null and value is not null)
-                            TableScan [TS_13] (rows=25/25 width=7)
-                              default@src1,x,Tbl:COMPLETE,Col:NONE,Output:["key","value"]
-                    <-Select Operator [SEL_12] (rows=262/319 width=10)
-                        Output:["_col1"]
-                        Group By Operator [GBY_11] (rows=262/319 width=10)
-                          Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-                        <-Union 2 [SIMPLE_EDGE]
-                          <-Map 1 [CONTAINS]
-                            Reduce Output Operator [RS_10]
-                              PartitionCols:_col0, _col1
-                              Group By Operator [GBY_9] (rows=525/331 width=10)
-                                Output:["_col0","_col1"],keys:_col1, _col0
-                                Select Operator [SEL_2] (rows=25/25 width=7)
-                                  Output:["_col0","_col1"]
-                                  Filter Operator [FIL_74] (rows=25/25 width=7)
-                                    predicate:value is not null
-                                    TableScan [TS_0] (rows=25/25 width=7)
-                                      Output:["key","value"]
-                          <-Map 6 [CONTAINS]
-                            Reduce Output Operator [RS_10]
-                              PartitionCols:_col0, _col1
-                              Group By Operator [GBY_9] (rows=525/331 width=10)
-                                Output:["_col0","_col1"],keys:_col1, _col0
-                                Select Operator [SEL_5] (rows=500/500 width=10)
-                                  Output:["_col0","_col1"]
-                                  Filter Operator [FIL_75] (rows=500/500 width=10)
-                                    predicate:value is not null
-                                    TableScan [TS_3] (rows=500/500 width=10)
-                                      Output:["key","value"]
-
-PREHOOK: query: SELECT x.key, y.value
-FROM src1 x JOIN src y ON (x.key = y.key) 
-JOIN (select * from src1 union select * from src)z ON (x.value = z.value)
-union
-SELECT x.key, y.value
-FROM src1 x JOIN src y ON (x.key = y.key) 
-JOIN (select key, value from src1 union select key, value from src union select key, value from src)z ON (x.value = z.value)
-union
-SELECT x.key, y.value
-FROM src1 x JOIN src y ON (x.key = y.key) 
-JOIN (select key, value from src1 union select key, value from src union select key, value from src union select key, value from src)z ON (x.value = z.value)
-PREHOOK: type: QUERY
-PREHOOK: Input: default@src
-PREHOOK: Input: default@src1
-#### A masked pattern was here ####
-POSTHOOK: query: SELECT x.key, y.value
-FROM src1 x JOIN src y ON (x.key = y.key) 
-JOIN (select * from src1 union select * from src)z ON (x.value = z.value)
-union
-SELECT x.key, y.value
-FROM src1 x JOIN src y ON (x.key = y.key) 
-JOIN (select key, value from src1 union select key, value from src union select key, value from src)z ON (x.value = z.value)
-union
-SELECT x.key, y.value
-FROM src1 x JOIN src y ON (x.key = y.key) 
-JOIN (select key, value from src1 union select key, value from src union select key, value from src union select key, value from src)z ON (x.value = z.value)
-POSTHOOK: type: QUERY
-POSTHOOK: Input: default@src
-POSTHOOK: Input: default@src1
-#### A masked pattern was here ####
-PREHOOK: query: explain analyze
-SELECT x.key, y.value
-FROM src1 x JOIN src y ON (x.key = y.key) 
-JOIN (select * from src1 union select * from src)z ON (x.value = z.value)
-union
-SELECT x.key, y.value
-FROM src1 x JOIN src y ON (x.key = y.key) 
-JOIN (select key, value from src1 union select key, value from src union select key, value from src)z ON (x.value = z.value)
-union
-SELECT x.key, y.value
-FROM src1 x JOIN src y ON (x.key = y.key) 
-JOIN (select key, value from src1 union select key, value from src union select key, value from src union select key, value from src)z ON (x.value = z.value)
-PREHOOK: type: QUERY
-POSTHOOK: query: explain analyze
-SELECT x.key, y.value
-FROM src1 x JOIN src y ON (x.key = y.key) 
-JOIN (select * from src1 union select * from src)z ON (x.value = z.value)
-union
-SELECT x.key, y.value
-FROM src1 x JOIN src y ON (x.key = y.key) 
-JOIN (select key, value from src1 union select key, value from src union select key, value from src)z ON (x.value = z.value)
-union
-SELECT x.key, y.value
-FROM src1 x JOIN src y ON (x.key = y.key) 
-JOIN (select key, value from src1 union select key, value from src union select key, value from src union select key, value from src)z ON (x.value = z.value)
-POSTHOOK: type: QUERY
-Plan optimized by CBO.
-
-Vertex dependency in root stage
-Map 1 <- Union 2 (CONTAINS)
-Map 11 <- Union 12 (CONTAINS)
-Map 16 <- Union 12 (CONTAINS)
-Map 17 <- Union 14 (CONTAINS)
-Map 20 <- Union 21 (CONTAINS)
-Map 27 <- Union 21 (CONTAINS)
-Map 28 <- Union 23 (CONTAINS)
-Map 29 <- Union 25 (CONTAINS)
-Map 8 <- Union 2 (CONTAINS)
-Reducer 13 <- Union 12 (SIMPLE_EDGE), Union 14 (CONTAINS)
-Reducer 15 <- Map 18 (BROADCAST_EDGE), Map 19 (BROADCAST_EDGE), Union 14 (SIMPLE_EDGE), Union 4 (CONTAINS)
-Reducer 22 <- Union 21 (SIMPLE_EDGE), Union 23 (CONTAINS)
-Reducer 24 <- Union 23 (SIMPLE_EDGE), Union 25 (CONTAINS)
-Reducer 26 <- Map 30 (BROADCAST_EDGE), Map 31 (BROADCAST_EDGE), Union 25 (SIMPLE_EDGE), Union 6 (CONTAINS)
-Reducer 3 <- Map 10 (BROADCAST_EDGE), Map 9 (BROADCAST_EDGE), Union 2 (SIMPLE_EDGE), Union 4 (CONTAINS)
-Reducer 5 <- Union 4 (SIMPLE_EDGE), Union 6 (CONTAINS)
-Reducer 7 <- Union 6 (SIMPLE_EDGE)
-
-Stage-0
-  Fetch Operator
-    limit:-1
-    Stage-1
-      Reducer 7
-      File Output Operator [FS_122]
-        Group By Operator [GBY_120] (rows=550/15 width=10)
-          Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-        <-Union 6 [SIMPLE_EDGE]
-          <-Reducer 26 [CONTAINS]
-            Reduce Output Operator [RS_119]
-              PartitionCols:_col0, _col1
-              Group By Operator [GBY_118] (rows=1100/30 width=10)
-                Output:["_col0","_col1"],keys:_col0, _col1
-                Select Operator [SEL_114] (rows=550/61 width=10)
-                  Output:["_col0","_col1"]
-                  Map Join Operator [MAPJOIN_170] (rows=550/61 width=10)
-                    Conds:MAPJOIN_169._col2=RS_112._col0(Inner),HybridGraceHashJoin:true,Output:["_col2","_col5"]
-                  <-Map 31 [BROADCAST_EDGE]
-                    BROADCAST [RS_112]
-                      PartitionCols:_col0
-                      Select Operator [SEL_107] (rows=500/500 width=10)
-                        Output:["_col0","_col1"]
-                        Filter Operator [FIL_164] (rows=500/500 width=10)
-                          predicate:key is not null
-                          TableScan [TS_105] (rows=500/500 width=10)
-                            default@src,y,Tbl:COMPLETE,Col:NONE,Output:["key","value"]
-                  <-Map Join Operator [MAPJOIN_169] (rows=484/52 width=10)
-                      Conds:SEL_101._col1=RS_109._col1(Inner),HybridGraceHashJoin:true,Output:["_col2"]
-                    <-Map 30 [BROADCAST_EDGE]
-                      BROADCAST [RS_109]
-                        PartitionCols:_col1
-                        Select Operator [SEL_104] (rows=25/25 width=7)
-                          Output:["_col0","_col1"]
-                          Filter Operator [FIL_163] (rows=25/25 width=7)
-                            predicate:(key is not null and value is not null)
-                            TableScan [TS_102] (rows=25/25 width=7)
-                              default@src1,x,Tbl:COMPLETE,Col:NONE,Output:["key","value"]
-                    <-Select Operator [SEL_101] (rows=440/319 width=10)
-                        Output:["_col1"]
-                        Group By Operator [GBY_100] (rows=440/319 width=10)
-                          Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-                        <-Union 25 [SIMPLE_EDGE]
-                          <-Map 29 [CONTAINS]
-                            Reduce Output Operator [RS_99]
-                              PartitionCols:_col0, _col1
-                              Group By Operator [GBY_98] (rows=881/628 width=10)
-                                Output:["_col0","_col1"],keys:_col1, _col0
-                                Select Operator [SEL_94] (rows=500/500 width=10)
-                                  Output:["_col0","_col1"]
-                                  Filter Operator [FIL_162] (rows=500/500 width=10)
-                                    predicate:value is not null
-                                    TableScan [TS_92] (rows=500/500 width=10)
-                                      Output:["key","value"]
-                          <-Reducer 24 [CONTAINS]
-                            Reduce Output Operator [RS_99]
-                              PartitionCols:_col0, _col1
-                              Group By Operator [GBY_98] (rows=881/628 width=10)
-                                Output:["_col0","_col1"],keys:_col1, _col0
-                                Select Operator [SEL_91] (rows=381/319 width=10)
-                                  Output:["_col0","_col1"]
-                                  Group By Operator [GBY_90] (rows=381/319 width=10)
-                                    Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-                                  <-Union 23 [SIMPLE_EDGE]
-                                    <-Map 28 [CONTAINS]
-                                      Reduce Output Operator [RS_89]
-                                        PartitionCols:_col0, _col1
-                                        Group By Operator [GBY_88] (rows=762/628 width=10)
-                                          Output:["_col0","_col1"],keys:_col1, _col0
-                                          Select Operator [SEL_84] (rows=500/500 width=10)
-                                            Output:["_col0","_col1"]
-                                            Filter Operator [FIL_161] (rows=500/500 width=10)
-                                              predicate:value is not null
-                                              TableScan [TS_82] (rows=500/500 width=10)
-                                                Output:["key","value"]
-                                    <-Reducer 22 [CONTAINS]
-                                      Reduce Output Operator [RS_89]
-                                        PartitionCols:_col0, _col1
-                                        Group By Operator [GBY_88] (rows=762/628 width=10)
-                                          Output:["_col0","_col1"],keys:_col1, _col0
-                                          Select Operator [SEL_81] (rows=262/319 width=10)
-                                            Output:["_col0","_col1"]
-                                            Group By Operator [GBY_80] (rows=262/319 width=10)
-                                              Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-                                            <-Union 21 [SIMPLE_EDGE]
-                                              <-Map 20 [CONTAINS]
-                                                Reduce Output Operator [RS_79]
-                                                  PartitionCols:_col0, _col1
-                                                  Group By Operator [GBY_78] (rows=525/331 width=10)
-                                                    Output:["_col0","_col1"],keys:_col1, _col0
-                                                    Select Operator [SEL_71] (rows=25/25 width=7)
-                                                      Output:["_col0","_col1"]
-                                                      Filter Operator [FIL_159] (rows=25/25 width=7)
-                                                        predicate:value is not null
-                                                        TableScan [TS_69] (rows=25/25 width=7)
-                                                          Output:["key","value"]
-                                              <-Map 27 [CONTAINS]
-                                                Reduce Output Operator [RS_79]
-                                                  PartitionCols:_col0, _col1
-                                                  Group By Operator [GBY_78] (rows=525/331 width=10)
-                                                    Output:["_col0","_col1"],keys:_col1, _col0
-                                                    Select Operator [SEL_74] (rows=500/500 width=10)
-                                                      Output:["_col0","_col1"]
-                                                      Filter Operator [FIL_160] (rows=500/500 width=10)
-                                                        predicate:value is not null
-                                                        TableScan [TS_72] (rows=500/500 width=10)
-                                                          Output:["key","value"]
-          <-Reducer 5 [CONTAINS]
-            Reduce Output Operator [RS_119]
-              PartitionCols:_col0, _col1
-              Group By Operator [GBY_118] (rows=1100/30 width=10)
-                Output:["_col0","_col1"],keys:_col0, _col1
-                Group By Operator [GBY_67] (rows=550/15 width=10)
-                  Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-                <-Union 4 [SIMPLE_EDGE]
-                  <-Reducer 15 [CONTAINS]
-                    Reduce Output Operator [RS_66]
-                      PartitionCols:_col0, _col1
-                      Group By Operator [GBY_65] (rows=1100/30 width=10)
-                        Output:["_col0","_col1"],keys:_col0, _col1
-                        Select Operator [SEL_61] (rows=550/61 width=10)
-                          Output:["_col0","_col1"]
-                          Map Join Operator [MAPJOIN_168] (rows=550/61 width=10)
-                            Conds:MAPJOIN_167._col2=RS_59._col0(Inner),HybridGraceHashJoin:true,Output:["_col2","_col5"]
-                          <-Map 19 [BROADCAST_EDGE]
-                            BROADCAST [RS_59]
-                              PartitionCols:_col0
-                              Select Operator [SEL_54] (rows=500/500 width=10)
-                                Output:["_col0","_col1"]
-                                Filter Operator [FIL_158] (rows=500/500 width=10)
-                                  predicate:key is not null
-                                  TableScan [TS_52] (rows=500/500 width=10)
-                                    default@src,y,Tbl:COMPLETE,Col:NONE,Output:["key","value"]
-                          <-Map Join Operator [MAPJOIN_167] (rows=419/52 width=10)
-                              Conds:SEL_48._col1=RS_56._col1(Inner),HybridGraceHashJoin:true,Output:["_col2"]
-                            <-Map 18 [BROADCAST_EDGE]
-                              BROADCAST [RS_56]
-                                PartitionCols:_col1
-                                Select Operator [SEL_51] (rows=25/25 width=7)
-                                  Output:["_col0","_col1"]
-                                  Filter Operator [FIL_157] (rows=25/25 width=7)
-                                    predicate:(key is not null and value is not null)
-                                    TableScan [TS_49] (rows=25/25 width=7)
-                                      default@src1,x,Tbl:COMPLETE,Col:NONE,Output:["key","value"]
-                            <-Select Operator [SEL_48] (rows=381/319 width=10)
-                                Output:["_col1"]
-                                Group By Operator [GBY_47] (rows=381/319 width=10)
-                                  Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-                                <-Union 14 [SIMPLE_EDGE]
-                                  <-Map 17 [CONTAINS]
-                                    Reduce Output Operator [RS_46]
-                                      PartitionCols:_col0, _col1
-                                      Group By Operator [GBY_45] (rows=762/628 width=10)
-                                        Output:["_col0","_col1"],keys:_col1, _col0
-                                        Select Operator [SEL_41] (rows=500/500 width=10)
-                                          Output:["_col0","_col1"]
-                                          Filter Operator [FIL_156] (rows=500/500 width=10)
-                                            predicate:value is not null
-                                            TableScan [TS_39] (rows=500/500 width=10)
-                                              Output:["key","value"]
-                                  <-Reducer 13 [CONTAINS]
-                                    Reduce Output Operator [RS_46]
-                                      PartitionCols:_col0, _col1
-                                      Group By Operator [GBY_45] (rows=762/628 width=10)
-                                        Output:["_col0","_col1"],keys:_col1, _col0
-                                        Select Operator [SEL_38] (rows=262/319 width=10)
-                                          Output:["_col0","_col1"]
-                                          Group By Operator [GBY_37] (rows=262/319 width=10)
-                                            Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-                                          <-Union 12 [SIMPLE_EDGE]
-                                            <-Map 11 [CONTAINS]
-                                              Reduce Output Operator [RS_36]
-                                                PartitionCols:_col0, _col1
-                                                Group By Operator [GBY_35] (rows=525/331 width=10)
-                                                  Output:["_col0","_col1"],keys:_col1, _col0
-                                                  Select Operator [SEL_28] (rows=25/25 width=7)
-                                                    Output:["_col0","_col1"]
-                                                    Filter Operator [FIL_154] (rows=25/25 width=7)
-                                                      predicate:value is not null
-                                                      TableScan [TS_26] (rows=25/25 width=7)
-                                                        Output:["key","value"]
-                                            <-Map 16 [CONTAINS]
-                                              Reduce Output Operator [RS_36]
-                                                PartitionCols:_col0, _col1
-                                                Group By Operator [GBY_35] (rows=525/331 width=10)
-                                                  Output:["_col0","_col1"],keys:_col1, _col0
-                                                  Select Operator [SEL_31] (rows=500/500 width=10)
-                                                    Output:["_col0","_col1"]
-                                                    Filter Operator [FIL_155] (rows=500/500 width=10)
-                                                      predicate:value is not null
-                                                      TableScan [TS_29] (rows=500/500 width=10)
-                                                        Output:["key","value"]
-                  <-Reducer 3 [CONTAINS]
-                    Reduce Output Operator [RS_66]
-                      PartitionCols:_col0, _col1
-                      Group By Operator [GBY_65] (rows=1100/30 width=10)
-                        Output:["_col0","_col1"],keys:_col0, _col1
-                        Select Operator [SEL_25] (rows=550/61 width=10)
-                          Output:["_col0","_col1"]
-                          Map Join Operator [MAPJOIN_166] (rows=550/61 width=10)
-                            Conds:MAPJOIN_165._col2=RS_23._col0(Inner),HybridGraceHashJoin:true,Output:["_col2","_col5"]
-                          <-Map 10 [BROADCAST_EDGE]
-                            BROADCAST [RS_23]
-                              PartitionCols:_col0
-                              Select Operator [SEL_18] (rows=500/500 width=10)
-                                Output:["_col0","_col1"]
-                                Filter Operator [FIL_153] (rows=500/500 width=10)
-                                  predicate:key is not null
-                                  TableScan [TS_16] (rows=500/500 width=10)
-                                    default@src,y,Tbl:COMPLETE,Col:NONE,Output:["key","value"]
-                          <-Map Join Operator [MAPJOIN_165] (rows=288/52 width=10)
-                              Conds:SEL_12._col1=RS_20._col1(Inner),HybridGraceHashJoin:true,Output:["_col2"]
-                            <-Map 9 [BROADCAST_EDGE]
-                              BROADCAST [RS_20]
-                                PartitionCols:_col1
-                                Select Operator [SEL_15] (rows=25/25 width=7)
-                                  Output:["_col0","_col1"]
-                                  Filter Operator [FIL_152] (rows=25/25 width=7)
-                                    predicate:(key is not null and value is not null)
-                                    TableScan [TS_13] (rows=25/25 width=7)
-                                      default@src1,x,Tbl:COMPLETE,Col:NONE,Output:["key","value"]
-                            <-Select Operator [SEL_12] (rows=262/319 width=10)
-                                Output:["_col1"]
-                                Group By Operator [GBY_11] (rows=262/319 width=10)
-                                  Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
-                                <-Union 2 [SIMPLE_EDGE]
-                                  <-Map 1 [CONTAINS]
-                                    Reduce Output Operator [RS_10]
-                                      PartitionCols:_col0, _col1
-                                      Group By Operator [GBY_9] (rows=525/331 width=10)
-                                        Output:["_col0","_col1"],keys:_col1, _col0
-                                        Select Operator [SEL_2] (rows=25/25 width=7)
-                                          Output:["_col0","_col1"]
-                                          Filter Operator [FIL_150] (rows=25/25 width=7)
-                                            predicate:value is not null
-                                            TableScan [TS_0] (rows=25/25 width=7)
-                                              Output:["key","value"]
-                                  <-Map 8 [CONTAINS]
-                                    Reduce Output Operator [RS_10]
-                                      PartitionCols:_col0, _col1
-                                      Group By Operator [GBY_9] (rows=525/331 width=10)
-                                        Output:["_col0","_col1"],keys:_col1, _col0
-                                        Select Operator [SEL_5] (rows=500/500 width=10)
-                                          Output:["_col0","_col1"]
-                                          Filter Operator [FIL_151] (rows=500/500 width=10)
-                                            predicate:value is not null
-                                            TableScan [TS_3] (rows=500/500 width=10)
-                                              Output:["key","value"]
-
-PREHOOK: query: CREATE TABLE srcbucket_mapjoin(key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE
-PREHOOK: type: CREATETABLE
-PREHOOK: Output: database:default
-PREHOOK: Output: default@srcbucket_mapjoin
-POSTHOOK: query: CREATE TABLE srcbucket_mapjoin(key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE
-POSTHOOK: type: CREATETABLE
-POSTHOOK: Output: database:default
-POSTHOOK: Output: default@srcbucket_mapjoin
-PREHOOK: query: CREATE TABLE tab_part (key int, value string) PARTITIONED BY(ds STRING) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE
-PREHOOK: type: CREATETABLE
-PREHOOK: Output: database:default
-PREHOOK: Output: default@tab_part
-POSTHOOK: query: CREATE TABLE tab_part (key int, value string) PARTITIONED BY(ds STRING) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE
-POSTHOOK: type: CREATETABLE
-POSTHOOK: Output: database:default
-POSTHOOK: Output: default@tab_part
-PREHOOK: query: CREATE TABLE srcbucket_mapjoin_part (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE
-PREHOOK: type: CREATETABLE
-PREHOOK: Output: database:default
-PREHOOK: Output: default@srcbucket_mapjoin_part
-POSTHOOK: query: CREATE TABLE srcbucket_mapjoin_part (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE
-POSTHOOK: type: CREATETABLE
-POSTHOOK: Output: database:default
-POSTHOOK: Output: default@srcbucket_mapjoin_part
-PREHOOK: query: load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin partition(ds='2008-04-08')
-PREHOOK: type: LOAD
-#### A masked pattern was here ####
-PREHOOK: Output: default@srcbucket_mapjoin
-POSTHOOK: query: load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin partition(ds='2008-04-08')
-POSTHOOK: type: LOAD
-#### A masked pattern was here ####
-POSTHOOK: Output: default@srcbucket_mapjoin
-POSTHOOK: Output: default@srcbucket_mapjoin@ds=2008-04-08
-PREHOOK: query: load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin partition(ds='2008-04-08')
-PREHOOK: type: LOAD
-#### A masked pattern was here ####
-PREHOOK: Output: default@srcbucket_mapjoin@ds=2008-04-08
-POSTHOOK: query: load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin partition(ds='2008-04-08')
-POSTHOOK: type: LOAD
+PREHOOK: Output: default@srcbucket_mapjoin@ds=2008-04-08
+POSTHOOK: query: load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin partition(ds='2008-04-08')
+POSTHOOK: type: LOAD
 #### A masked pattern was here ####
 POSTHOOK: Output: default@srcbucket_mapjoin@ds=2008-04-08
 PREHOOK: query: load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08')
@@ -2028,19 +693,19 @@ Stage-0
     Stage-1
       Map 1
       File Output Operator [FS_10]
-        Merge Join Operator [MERGEJOIN_15] (rows=266/480 width=10)
+        Merge Join Operator [MERGEJOIN_15] (rows=266/480 width=18)
           Conds:SEL_2._col0=SEL_5._col0(Inner),Output:["_col0","_col1"]
-        <-Select Operator [SEL_5] (rows=242/242 width=10)
+        <-Select Operator [SEL_5] (rows=242/242 width=18)
             Output:["_col0"]
-            Filter Operator [FIL_14] (rows=242/242 width=10)
+            Filter Operator [FIL_14] (rows=242/242 width=18)
               predicate:key is not null
-              TableScan [TS_3] (rows=242/242 width=10)
+              TableScan [TS_3] (rows=242/242 width=18)
                 default@tab,s3,Tbl:COMPLETE,Col:NONE,Output:["key"]
-        <-Select Operator [SEL_2] (rows=242/242 width=10)
+        <-Select Operator [SEL_2] (rows=242/242 width=18)
             Output:["_col0","_col1"]
-            Filter Operator [FIL_13] (rows=242/242 width=10)
+            Filter Operator [FIL_13] (rows=242/242 width=18)
               predicate:key is not null
-              TableScan [TS_0] (rows=242/242 width=10)
+              TableScan [TS_0] (rows=242/242 width=18)
                 default@tab,s1,Tbl:COMPLETE,Col:NONE,Output:["key","value"]
 
 PREHOOK: query: select s1.key as key, s1.value as value from tab s1 join tab s3 on s1.key=s3.key join tab s2 on s1.value=s2.value
@@ -2070,33 +735,33 @@ Stage-0
     Stage-1
       Reducer 2
       File Output Operator [FS_16]
-        Merge Join Operator [MERGEJOIN_27] (rows=292/1166 width=10)
+        Merge Join Operator [MERGEJOIN_27] (rows=292/1166 width=18)
           Conds:RS_12._col1=RS_13._col1(Inner),Output:["_col0","_col1"]
         <-Map 1 [SIMPLE_EDGE]
           SHUFFLE [RS_12]
             PartitionCols:_col1
-            Merge Join Operator [MERGEJOIN_25] (rows=266/480 width=10)
+            Merge Join Operator [MERGEJOIN_25] (rows=266/480 width=18)
               Conds:SEL_2._col0=SEL_5._col0(Inner),Output:["_col0","_col1"]
-            <-Select Operator [SEL_5] (rows=242/242 width=10)
+            <-Select Operator [SEL_5] (rows=242/242 width=18)
                 Output:["_col0"]
-                Filter Operator [FIL_23] (rows=242/242 width=10)
+                Filter Operator [FIL_23] (rows=242/242 width=18)
                   predicate:key is not null
-                  TableScan [TS_3] (rows=242/242 width=10)
+                  TableScan [TS_3] (rows=242/242 width=18)
                     default@tab,s3,Tbl:COMPLETE,Col:NONE,Output:["key"]
-            <-Select Operator [SEL_2] (rows=242/242 width=10)
+            <-Select Operator [SEL_2] (rows=242/242 width=18)
                 Output:["_col0","_col1"]
-                Filter Operator [FIL_22] (rows=242/242 width=10)
+                Filter Operator [FIL_22] (rows=242/242 width=18)
                   predicate:(key is not null and value is not null)
-                  TableScan [TS_0] (rows=242/242 width=10)
+                  TableScan [TS_0] (rows=242/242 width=18)
                     default@tab,s1,Tbl:COMPLETE,Col:NONE,Output:["key","value"]
         <-Map 4 [SIMPLE_EDGE]
           SHUFFLE [RS_13]
             PartitionCols:_col1
-            Select Operator [SEL_8] (rows=242/242 width=10)
+            Select Operator [SEL_8] (rows=242/242 width=18)
               Output:["_col1"]
-              Filter Operator [FIL_24] (rows=242/242 width=10)
+              Filter Operator [FIL_24] (rows=242/242 width=18)
                 predicate:value is not null
-                TableScan [TS_6] (rows=242/242 width=10)
+                TableScan [TS_6] (rows=242/242 width=18)
                   default@tab,s2,Tbl:COMPLETE,Col:NONE,Output:["value"]
 
 PREHOOK: query: select s1.key as key, s1.value as value from tab s1 join tab2 s3 on s1.key=s3.key
@@ -2127,19 +792,19 @@ Stage-0
     Stage-1
       Map 1
       File Output Operator [FS_10]
-        Merge Join Operator [MERGEJOIN_15] (rows=266/480 width=10)
+        Merge Join Operator [MERGEJOIN_15] (rows=266/480 width=18)
           Conds:SEL_2._col0=SEL_5._col0(Inner),Output:["_col0","_col1"]
-        <-Select Operator [SEL_5] (rows=242/242 width=10)
+        <-Select Operator [SEL_5] (rows=242/242 width=18)
             Output:["_col0"]
-            Filter Operator [FIL_14] (rows=242/242 width=10)
+            Filter Operator [FIL_14] (rows=242/242 width=18)
               predicate:key is not null
-              TableScan [TS_3] (rows=242/242 width=10)
+              TableScan [TS_3] (rows=242/242 width=18)
                 default@tab2,s3,Tbl:COMPLETE,Col:NONE,Output:["key"]
-        <-Select Operator [SEL_2] (rows=242/242 width=10)
+        <-Select Operator [SEL_2] (rows=242/242 width=18)
             Output:["_col0","_col1"]
-            Filter Operator [FIL_13] (rows=242/242 width=10)
+            Filter Operator [FIL_13] (rows=242/242 width=18)
               predicate:key is not null
-              TableScan [TS_0] (rows=242/242 width=10)
+              TableScan [TS_0] (rows=242/242 width=18)
                 default@tab,s1,Tbl:COMPLETE,Col:NONE,Output:["key","value"]
 
 PREHOOK: query: select s1.key as key, s1.value as value from tab s1 join tab2 s3 on s1.key=s3.key join tab2 s2 on s1.value=s2.value
@@ -2173,33 +838,33 @@ Stage-0
     Stage-1
       Reducer 2
       File Output Operator [FS_16]
-        Merge Join Operator [MERGEJOIN_27] (rows=292/1166 width=10)
+        Merge Join Operator [MERGEJOIN_27] (rows=292/1166 width=18)
           Conds:RS_12._col1=RS_13._col1(Inner),Output:["_col0","_col1"]
         <-Map 1 [SIMPLE_EDGE]
           SHUFFLE [RS_12]
             PartitionCols:_col1
-            Merge Join Operator [MERGEJOIN_25] (rows=266/480 width=10)
+            Merge Join Operator [MERGEJOIN_25] (rows=266/480 width=18)
               Conds:SEL_2._col0=SEL_5._col0(Inner),Output:["_col0","_col1"]
-            <-Select Operator [SEL_5] (rows=242/242 width=10)
+            <-Select Operator [SEL_5] (rows=242/242 width=18)
                 Output:["_col0"]
-                Filter Operator [FIL_23] (rows=242/242 width=10)
+                Filter Operator [FIL_23] (rows=242/242 width=18)
                   predicate:key is not null
-                  TableScan [TS_3] (rows=242/242 width=10)
+                  TableScan [TS_3] (rows=242/242 width=18)
                     default@tab2,s3,Tbl:COMPLETE,Col:NONE,Output:["key"]
-            <-Select Operator [SEL_2] (rows=242/242 width=10)
+            <-Select Operator [SEL_2] (rows=242/242 width=18)
                 Output:["_col0","_col1"]
-                Filter Operator [FIL_22] (rows=242/242 width=10)
+                Filter Operator [FIL_22] (rows=242/242 width=18)
                   predicate:(key is not null and value is not null)
-                  TableScan [TS_0] (rows=242/242 width=10)
+                  TableScan [TS_0] (rows=242/242 width=18)
                     default@tab,s1,Tbl:COMPLETE,Col:NONE,Output:["key","value"]
         <-Map 4 [SIMPLE_EDGE]
           SHUFFLE [RS_13]
             PartitionCols:_col1
-            Select Operator [SEL_8] (rows=242/242 width=10)
+            Select Operator [SEL_8] (rows=242/242 width=18)
               Output:["_col1"]
-              Filter Operator [FIL_24] (rows=242/242 width=10)
+              Filter Operator [FIL_24] (rows=242/242 width=18)
                 predicate:value is not null
-                TableScan [TS_6] (rows=242/242 width=10)
+                TableScan [TS_6] (rows=242/242 width=18)
                   default@tab2,s2,Tbl:COMPLETE,Col:NONE,Output:["value"]
 
 PREHOOK: query: select count(*) from (select s1.key as key, s1.value as value from tab s1 join tab s3 on s1.key=s3.key
@@ -2254,43 +919,43 @@ Stage-0
           SHUFFLE [RS_23]
             Group By Operator [GBY_22] (rows=1/1 width=8)
               Output:["_col0"],aggregations:["count()"]
-              Merge Join Operator [MERGEJOIN_39] (rows=558/1646 width=10)
+              Merge Join Operator [MERGEJOIN_39] (rows=558/1646 width=18)
                 Conds:Union 2._col0=RS_19._col0(Inner)
               <-Map 7 [SIMPLE_EDGE]
                 SHUFFLE [RS_19]
                   PartitionCols:_col0
-                  Select Operator [SEL_17] (rows=500/500 width=10)
+                  Select Operator [SEL_17] (rows=500/500 width=18)
                     Output:["_col0"]
-                    Filter Operator [FIL_36] (rows=500/500 width=10)
+                    Filter Operator [FIL_36] (rows=500/500 width=18)
                       predicate:key is not null
-                      TableScan [TS_15] (rows=500/500 width=10)
+                      TableScan [TS_15] (rows=500/500 width=18)
                         default@tab_part,b,Tbl:COMPLETE,Col:NONE,Output:["key"]
               <-Union 2 [SIMPLE_EDGE]
                 <-Map 1 [CONTAINS]
                   Reduce Output Operator [RS_18]
                     PartitionCols:_col0
-                    Merge Join Operator [MERGEJOIN_37] (rows=266/480 width=10)
+                    Merge Join Operator [MERGEJOIN_37] (rows=266/480 width=18)
                       Conds:SEL_2._col0=SEL_5._col0(Inner),Output:["_col0"]
-                    <-Select Operator [SEL_5] (rows=242/242 width=10)
+                    <-Select Operator [SEL_5] (rows=242/242 width=18)
                         Output:["_col0"]
-                        Filter Operator [FIL_34] (rows=242/242 width=10)
+                        Filter Operator [FIL_34] (rows=242/242 width=18)
                           predicate:key is not null
-                          TableScan [TS_3] (rows=242/242 width=10)
+                          TableScan [TS_3] (rows=242/242 width=18)
                             default@tab,s3,Tbl:COMPLETE,Col:NONE,Output:["key"]
-                    <-Select Operator [SEL_2] (rows=242/242 width=10)
+                    <-Select Operator [SEL_2] (rows=242/242 width=18)
                         Output:["_col0"]
-                        Filter Operator [FIL_33] (rows=242/242 width=10)
+                        Filter Operator [FIL_33] (rows=242/242 width=18)
                           predicate:key is not null
-                          TableScan [TS_0] (rows=242/242 width=10)
+                          TableScan [TS_0] (rows=242/242 width=18)
                             Output:["key"]
                 <-Map 6 [CONTAINS]
                   Reduce Output Operator [RS_18]
                     PartitionCols:_col0
-                    Select Operator [SEL_12] (rows=242/242 width=10)
+                    Select Operator [SEL_12] (rows=242/242 width=18)
                       Output:["_col0"]
-                      Filter Operator [FIL_35] (rows=242/242 width=10)
+                      Filter Operator [FIL_35] (rows=242/242 width=18)
                         predicate:key is not null
-                        TableScan [TS_10] (rows=242/242 width=10)
+                        TableScan [TS_10] (rows=242/242 width=18)
                           Output:["key"]
 
 PREHOOK: query: select count(*) from (select s1.key as key, s1.value as value from tab s1 join tab s3 on s1.key=s3.key join tab s2 on s1.value=s2.value
@@ -2345,934 +1010,59 @@ Stage-0
           SHUFFLE [RS_29]
             Group By Operator [GBY_28] (rows=1/1 width=8)
               Output:["_col0"],aggregations:["count()"]
-              Merge Join Operator [MERGEJOIN_51] (rows=587/3768 width=10)
+              Merge Join Operator [MERGEJOIN_51] (rows=587/3768 width=18)
                 Conds:Union 3._col0=RS_25._col0(Inner)
               <-Map 9 [SIMPLE_EDGE]
                 SHUFFLE [RS_25]
                   PartitionCols:_col0
-                  Select Operator [SEL_23] (rows=500/500 width=10)
+                  Select Operator [SEL_23] (rows=500/500 width=18)
                     Output:["_col0"]
-                    Filter Operator [FIL_47] (rows=500/500 width=10)
+                    Filter Operator [FIL_47] (rows=500/500 width=18)
                       predicate:key is not null
-                      TableScan [TS_21] (rows=500/500 width=10)
+                      TableScan [TS_21] (rows=500/500 width

<TRUNCATED>