You are viewing a plain text version of this content. The canonical link for it is here.
Posted to gitbox@hive.apache.org by GitBox <gi...@apache.org> on 2020/09/21 00:30:35 UTC

[GitHub] [hive] jcamachor commented on a change in pull request #1472: HIVE-24009 Support partition pruning, vectorization and other physical transformations for EXECUTE statement

jcamachor commented on a change in pull request #1472:
URL: https://github.com/apache/hive/pull/1472#discussion_r491751273



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/parse/ExecuteStatementAnalyzer.java
##########
@@ -253,14 +191,17 @@ public void analyzeInternal(ASTNode root) throws SemanticException {
     String queryName = getQueryName(root);
     if (ss.getPreparePlans().containsKey(queryName)) {
       // retrieve cached plan from session state
-      BaseSemanticAnalyzer cachedPlan = ss.getPreparePlans().get(queryName);
+      SemanticAnalyzer cachedPlan = ss.getPreparePlans().get(queryName);
 
       // make copy of the plan
-      createTaskCopy(cachedPlan);
+      //createTaskCopy(cachedPlan);

Review comment:
       Can remove line commented out.

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/parse/PrepareStatementAnalyzer.java
##########
@@ -54,6 +58,21 @@ private void savePlan(String queryName) throws SemanticException{
     ss.getPreparePlans().put(queryName, this);
   }
 
+  private <T> T makeCopy(final Object task, Class<T> objClass) {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();

Review comment:
       Can we leave a comment on this method to understand what it is trying to do?

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/exec/TableScanOperator.java
##########
@@ -63,19 +63,19 @@
 
   private VectorizationContext taskVectorizationContext;
 
-  protected transient JobConf jc;
-  private transient boolean inputFileChanged = false;
+  protected JobConf jc;

Review comment:
       Do we need to keep all these fields for the plan cache in the operator, table, etc.? I am wondering about the implications of keeping them when the operator plan is serialized (i.e., whether that could have an performance impact). @t3rmin4t0r , @rbalamohan , could you comment on this?

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/parse/ExecuteStatementAnalyzer.java
##########
@@ -286,6 +227,24 @@ public void analyzeInternal(ASTNode root) throws SemanticException {
       this.acidFileSinks.addAll(cachedPlan.getAcidFileSinks());
       this.initCtx(cachedPlan.getCtx());
       this.ctx.setCboInfo(cachedPlan.getCboInfo());
+      this.setLoadFileWork(cachedPlan.getLoadFileWork());
+      this.setLoadTableWork(cachedPlan.getLoadTableWork());
+
+      this.setQB(cachedPlan.getQB());
+
+      ParseContext pctxt = this.getParseContext();
+      // partition pruner
+      Transform ppr = new PartitionPruner();
+      ppr.transform(pctxt);
+
+      //pctxt.setQueryProperties(this.queryProperties);
+      if (!ctx.getExplainLogical()) {
+        TaskCompiler compiler = TaskCompilerFactory.getCompiler(conf, pctxt);
+        compiler.init(queryState, console, db);
+        compiler.compile(pctxt, rootTasks, inputs, outputs);
+        fetchTask = pctxt.getFetchTask();
+        //fetchTask = makeCopy(cachedPlan.getFetchTask(), cachedPlan.getFetchTask().getClass());

Review comment:
       This comment too.

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/parse/ExecuteStatementAnalyzer.java
##########
@@ -286,6 +227,24 @@ public void analyzeInternal(ASTNode root) throws SemanticException {
       this.acidFileSinks.addAll(cachedPlan.getAcidFileSinks());
       this.initCtx(cachedPlan.getCtx());
       this.ctx.setCboInfo(cachedPlan.getCboInfo());
+      this.setLoadFileWork(cachedPlan.getLoadFileWork());
+      this.setLoadTableWork(cachedPlan.getLoadTableWork());
+
+      this.setQB(cachedPlan.getQB());
+
+      ParseContext pctxt = this.getParseContext();
+      // partition pruner
+      Transform ppr = new PartitionPruner();
+      ppr.transform(pctxt);
+
+      //pctxt.setQueryProperties(this.queryProperties);

Review comment:
       Same, can be removed?

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
##########
@@ -387,6 +387,12 @@
   protected volatile boolean disableJoinMerge = false;
   protected final boolean defaultJoinMerge;
 
+  /*
+   * This is used by prepare/execute statement
+   * Prepare/Execute requires operators to be copied and cached
+   */
+  protected Map<String, TableScanOperator> topOpsCopy = null;

Review comment:
       Why do you need to keep a copy instead of using the original operators? Could you leave a comment on that?

##########
File path: ql/src/test/results/clientpositive/llap/constprog_dpp.q.out
##########
@@ -84,12 +84,13 @@ Stage-0
                         Select Operator [SEL_40] (rows=1 width=4)
                           Output:["_col0"]
                           TableScan [TS_24] (rows=1 width=4)
-                            Output:["id"]
+                            default@tb2,tb2,Tbl:COMPLETE,Col:NONE,Output:["id"]

Review comment:
       I guess this was actually unrelated to current patch? Probably due to some data structure not being serialized when different union branches are copied?

##########
File path: ql/src/test/results/clientpositive/llap/constprog_dpp.q.out
##########
@@ -84,12 +84,13 @@ Stage-0
                         Select Operator [SEL_40] (rows=1 width=4)
                           Output:["_col0"]
                           TableScan [TS_24] (rows=1 width=4)
-                            Output:["id"]
+                            default@tb2,tb2,Tbl:COMPLETE,Col:NONE,Output:["id"]
                   <-Map 6 [CONTAINS] vectorized, llap
                     Reduce Output Operator [RS_45]
                       Limit [LIM_44] (rows=1 width=2)
                         Number of rows:1
                         Select Operator [SEL_43] (rows=1 width=0)
                           Output:["_col0"]
                           TableScan [TS_29] (rows=1 width=0)
+                            default@tb2,tb2,Tbl:PARTIAL,Col:COMPLETE

Review comment:
       This seems different than the stats above. Do you know if it is expected? May be worth checking in a follow-up.

##########
File path: ql/src/test/results/clientpositive/llap/prepare_plan.q.out
##########
@@ -170,10 +170,10 @@ STAGE PLANS:
                           sort order: 
                           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                           value expressions: _col0 (type: bigint)
-            Execution mode: llap
+            Execution mode: vectorized, llap

Review comment:
       Nice!




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org