You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by pr...@apache.org on 2016/10/05 00:47:16 UTC

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

Repository: hive
Updated Branches:
  refs/heads/master efe9c84e7 -> 481d7cd5c


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/master
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