You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by jc...@apache.org on 2018/08/09 21:36:31 UTC

hive git commit: HIVE-20347: hive.optimize.sort.dynamic.partition should work with partitioned CTAS and MV (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Repository: hive
Updated Branches:
  refs/heads/master 873d31f33 -> 689423981


HIVE-20347: hive.optimize.sort.dynamic.partition should work with partitioned CTAS and MV (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/68942398
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/68942398
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/68942398

Branch: refs/heads/master
Commit: 689423981e6ec13ec35d6b5c0a988d1059f5425c
Parents: 873d31f
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Thu Aug 9 14:36:23 2018 -0700
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Thu Aug 9 14:36:23 2018 -0700

----------------------------------------------------------------------
 .../test/resources/testconfiguration.properties |   1 +
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |  26 ++---
 .../materialized_view_partitioned_3.q           |  13 +++
 .../llap/materialized_view_partitioned_3.q.out  | 108 +++++++++++++++++++
 4 files changed, 133 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/68942398/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------
diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index 64003d0..27d2974 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -567,6 +567,7 @@ minillaplocal.query.files=\
   materialized_view_drop.q,\
   materialized_view_partitioned.q,\
   materialized_view_partitioned_2.q,\
+  materialized_view_partitioned_3.q,\
   materialized_view_rebuild.q,\
   materialized_view_rewrite_empty.q,\
   materialized_view_rewrite_1.q,\

http://git-wip-us.apache.org/repos/asf/hive/blob/68942398/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
index 4638805..a63aabe 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
@@ -6847,12 +6847,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       sortOrders = getSortOrders(dest, qb, dest_tab, input);
       if (!enforceBucketing) {
         throw new SemanticException(ErrorMsg.TBL_SORTED_NOT_BUCKETED.getErrorCodedMsg(dest_tab.getCompleteName()));
-      } else {
-        if (!enforceBucketing) {
-          partnCols = sortCols;
-        }
       }
-      enforceBucketing = true;
     }
 
     if (enforceBucketing) {
@@ -7185,7 +7180,6 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     boolean isLocal = false;
     SortBucketRSCtx rsCtx = new SortBucketRSCtx();
     DynamicPartitionCtx dpCtx = null;
-    Table partitionedCTASOrMVTable = null; // destination partitioned CTAS or MV table if any
     LoadTableDesc ltd = null;
     ListBucketingCtx lbCtx = null;
     Map<String, String> partSpec = null;
@@ -7578,6 +7572,13 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
             new SelectDesc(columnExprs, colNames), new RowSchema(rowResolver
                 .getColumnInfos()), input), rowResolver);
         input.setColumnExprMap(colExprMap);
+
+        try {
+          destinationTable = tblDesc != null ? tblDesc.toTable(conf) : viewDesc.toTable(conf);
+        } catch (HiveException e) {
+          throw new SemanticException(e);
+        }
+
         // If this is a partitioned CTAS or MV statement, we are going to create a LoadTableDesc
         // object. Although the table does not exist in metastore, we will swamp the CreateTableTask
         // and MoveTask resulting from this LoadTable so in this specific case, first we create
@@ -7608,15 +7609,10 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
           ltd.setInsertOverwrite(false);
           ltd.setLoadFileType(LoadFileType.KEEP_EXISTING);
         }
-        try {
-          partitionedCTASOrMVTable = tblDesc != null ? tblDesc.toTable(conf) : viewDesc.toTable(conf);
-          ltd.setMdTable(partitionedCTASOrMVTable);
-          WriteEntity output = generateTableWriteEntity(
-              dest, partitionedCTASOrMVTable, dpCtx.getPartSpec(), ltd, dpCtx, isNonNativeTable);
-          ctx.getLoadTableOutputMap().put(ltd, output);
-        } catch (HiveException e) {
-          throw new SemanticException(e);
-        }
+        ltd.setMdTable(destinationTable);
+        WriteEntity output = generateTableWriteEntity(
+            dest, destinationTable, dpCtx.getPartSpec(), ltd, dpCtx, isNonNativeTable);
+        ctx.getLoadTableOutputMap().put(ltd, output);
       } else {
         // Create LFD even for MM CTAS - it's a no-op move, but it still seems to be used for stats.
         loadFileWork.add(new LoadFileDesc(tblDesc, viewDesc, queryTmpdir, destinationPath, isDfsDir, cols,

http://git-wip-us.apache.org/repos/asf/hive/blob/68942398/ql/src/test/queries/clientpositive/materialized_view_partitioned_3.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/materialized_view_partitioned_3.q b/ql/src/test/queries/clientpositive/materialized_view_partitioned_3.q
new file mode 100644
index 0000000..472563f
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/materialized_view_partitioned_3.q
@@ -0,0 +1,13 @@
+--! qt:dataset:src
+
+set hive.support.concurrency=true;
+set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+set hive.optimize.sort.dynamic.partition=true;
+set hive.exec.dynamic.partition.mode=nonstrict;
+
+CREATE TABLE src_txn stored as orc TBLPROPERTIES ('transactional' = 'true')
+AS SELECT * FROM src;
+
+EXPLAIN
+CREATE MATERIALIZED VIEW partition_mv_sdp PARTITIONED ON (key) AS
+SELECT value, key FROM src_txn where key > 200 and key < 250;

http://git-wip-us.apache.org/repos/asf/hive/blob/68942398/ql/src/test/results/clientpositive/llap/materialized_view_partitioned_3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/materialized_view_partitioned_3.q.out b/ql/src/test/results/clientpositive/llap/materialized_view_partitioned_3.q.out
new file mode 100644
index 0000000..726c660
--- /dev/null
+++ b/ql/src/test/results/clientpositive/llap/materialized_view_partitioned_3.q.out
@@ -0,0 +1,108 @@
+PREHOOK: query: CREATE TABLE src_txn stored as orc TBLPROPERTIES ('transactional' = 'true')
+AS SELECT * FROM src
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@src
+PREHOOK: Output: database:default
+PREHOOK: Output: default@src_txn
+POSTHOOK: query: CREATE TABLE src_txn stored as orc TBLPROPERTIES ('transactional' = 'true')
+AS SELECT * FROM src
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@src
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@src_txn
+POSTHOOK: Lineage: src_txn.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: src_txn.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+PREHOOK: query: EXPLAIN
+CREATE MATERIALIZED VIEW partition_mv_sdp PARTITIONED ON (key) AS
+SELECT value, key FROM src_txn where key > 200 and key < 250
+PREHOOK: type: CREATE_MATERIALIZED_VIEW
+POSTHOOK: query: EXPLAIN
+CREATE MATERIALIZED VIEW partition_mv_sdp PARTITIONED ON (key) AS
+SELECT value, key FROM src_txn where key > 200 and key < 250
+POSTHOOK: type: CREATE_MATERIALIZED_VIEW
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1
+  Stage-4 depends on stages: Stage-2
+  Stage-0 depends on stages: Stage-4
+  Stage-3 depends on stages: Stage-0, Stage-4
+  Stage-5 depends on stages: Stage-3
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Reducer 2 <- Map 1 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: src_txn
+                  filterExpr: ((UDFToDouble(key) > 200.0D) and (UDFToDouble(key) < 250.0D)) (type: boolean)
+                  Statistics: Num rows: 500 Data size: 175168 Basic stats: COMPLETE Column stats: NONE
+                  Filter Operator
+                    predicate: ((UDFToDouble(key) < 250.0D) and (UDFToDouble(key) > 200.0D)) (type: boolean)
+                    Statistics: Num rows: 55 Data size: 19268 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: value (type: string), key (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 55 Data size: 19268 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: 55 Data size: 19268 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col0 (type: string)
+            Execution mode: vectorized, llap
+            LLAP IO: may be used (ACID table)
+        Reducer 2 
+            Execution mode: vectorized, llap
+            Reduce Operator Tree:
+              Select Operator
+                expressions: VALUE._col0 (type: string), KEY._col1 (type: string)
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 55 Data size: 19268 Basic stats: COMPLETE Column stats: NONE
+                File Output Operator
+                  compressed: false
+                  Dp Sort State: PARTITION_SORTED
+                  Statistics: Num rows: 55 Data size: 19268 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+                      serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+                      name: default.partition_mv_sdp
+
+  Stage: Stage-2
+    Dependency Collection
+
+  Stage: Stage-4
+      Create View Operator:
+        Create View
+          partition columns: key string
+          columns: value string
+          expanded text: SELECT `src_txn`.`value`, `src_txn`.`key` FROM `default`.`src_txn` where `src_txn`.`key` > 200 and `src_txn`.`key` < 250
+          name: default.partition_mv_sdp
+          original text: SELECT value, key FROM src_txn where key > 200 and key < 250
+          rewrite enabled: true
+
+  Stage: Stage-0
+    Move Operator
+      tables:
+          partition:
+            key 
+          replace: false
+          table:
+              input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+              output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat
+              serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
+              name: default.partition_mv_sdp
+
+  Stage: Stage-3
+    Stats Work
+      Basic Stats Work:
+
+  Stage: Stage-5
+    Materialized View Work
+