You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ha...@apache.org on 2020/07/07 19:48:52 UTC
[hive] branch master updated: HIVE-23665 Rewrite last_value to
first_value to enable streaming results (Ramesh Kumar via Jesus Camacho
Rodriguez)
This is an automated email from the ASF dual-hosted git repository.
hashutosh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git
The following commit(s) were added to refs/heads/master by this push:
new 5c1a3d6 HIVE-23665 Rewrite last_value to first_value to enable streaming results (Ramesh Kumar via Jesus Camacho Rodriguez)
5c1a3d6 is described below
commit 5c1a3d6dfe64d0daf2ea47047cfcb9dae62e2fa5
Author: Ramesh Thangarajan <ra...@cloudera.com>
AuthorDate: Wed Jun 24 10:13:31 2020 -0700
HIVE-23665 Rewrite last_value to first_value to enable streaming results (Ramesh Kumar via Jesus Camacho Rodriguez)
Signed-off-by: Ashutosh Chauhan <ha...@apache.org>
---
.../rules/HiveWindowingLastValueRewrite.java | 119 +++++++++++
.../hadoop/hive/ql/parse/CalcitePlanner.java | 4 +-
.../vector_ptf_last_value_streaming.q | 9 +
.../llap/vector_ptf_last_value_streaming.q.out | 192 +++++++++++++++++
.../llap/vector_ptf_part_simple.q.out | 238 ++++++++++++++-------
5 files changed, 478 insertions(+), 84 deletions(-)
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveWindowingLastValueRewrite.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveWindowingLastValueRewrite.java
new file mode 100644
index 0000000..5845553
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveWindowingLastValueRewrite.java
@@ -0,0 +1,119 @@
+/*
+ * 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.optimizer.calcite.rules;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexFieldCollation;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexOver;
+import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.rex.RexWindow;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.commons.collections.CollectionUtils;
+import org.apache.hadoop.hive.ql.optimizer.calcite.translator.SqlFunctionConverter;
+
+/**
+ * Rule to rewrite a window function containing a last value clause.
+ */
+public class HiveWindowingLastValueRewrite extends RelOptRule {
+
+ public static final HiveWindowingLastValueRewrite INSTANCE = new HiveWindowingLastValueRewrite();
+
+ private static final String FIRST_VALUE_FUNC = "first_value";
+ private static final String LAST_VALUE_FUNC = "last_value";
+
+
+ private HiveWindowingLastValueRewrite() {
+ super(operand(Project.class, any()));
+ }
+
+ @Override
+ public void onMatch(RelOptRuleCall call) {
+ Project project = call.rel(0);
+
+ List<RexNode> newExprs = new ArrayList<>();
+ LastValueRewriteRexShuttle lastValueRewrite = new LastValueRewriteRexShuttle(
+ project.getCluster().getRexBuilder());
+ boolean modified = false;
+ for (RexNode expr : project.getChildExps()) {
+ RexNode newExpr = lastValueRewrite.apply(expr);
+ newExprs.add(newExpr);
+ modified |= (newExpr != expr);
+ }
+ if (modified) {
+ RelNode newProject = project.copy(
+ project.getTraitSet(), project.getInput(), newExprs, project.getRowType());
+ call.transformTo(newProject);
+ }
+ }
+
+ private static class LastValueRewriteRexShuttle extends RexShuttle {
+
+ private final RexBuilder rexBuilder;
+
+ private LastValueRewriteRexShuttle(RexBuilder rexBuilder) {
+ this.rexBuilder = rexBuilder;
+ }
+
+ public RexNode visitOver(RexOver over) {
+ if (over.op.getName().equals(LAST_VALUE_FUNC) && over.getWindow().getLowerBound().isUnbounded()
+ && over.getWindow().getUpperBound().isUnbounded()) {
+ ImmutableList<RexFieldCollation> orderKeys = over.getWindow().orderKeys;
+ if (CollectionUtils.isEmpty(orderKeys)) {
+ return over;
+ }
+ ImmutableList.Builder<RexFieldCollation> newOrderKeys = ImmutableList.builder();
+ for (RexFieldCollation orderKey : orderKeys) {
+ Set<SqlKind> flags = new HashSet<>();
+ if (orderKey.getDirection() == RelFieldCollation.Direction.ASCENDING) {
+ flags.add(SqlKind.DESCENDING);
+ }
+ if (orderKey.right.contains(SqlKind.NULLS_FIRST)) {
+ flags.add(SqlKind.NULLS_LAST);
+ } else {
+ flags.add(SqlKind.NULLS_FIRST);
+ }
+ newOrderKeys.add(new RexFieldCollation(orderKey.left, flags));
+ }
+ SqlAggFunction s = (SqlAggFunction) over.op;
+ SqlFunctionConverter.CalciteUDAF newSqlAggFunction = new SqlFunctionConverter.CalciteUDAF(
+ over.isDistinct(), FIRST_VALUE_FUNC, s.getReturnTypeInference(), s.getOperandTypeInference(),
+ s.getOperandTypeChecker());
+ List<RexNode> clonedOperands = visitList(over.operands, new boolean[] {false});
+ RexWindow window = visitWindow(over.getWindow());
+ return rexBuilder.makeOver(over.type, newSqlAggFunction, clonedOperands,
+ window.partitionKeys, newOrderKeys.build(),
+ window.getLowerBound(), window.getUpperBound(),
+ window.isRows(), true, false, over.isDistinct(), over.ignoreNulls());
+ }
+ return over;
+ }
+ }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
index 2b89eb9..4222a9b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
@@ -252,6 +252,7 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveSubQueryRemoveRule;
import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveUnionMergeRule;
import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveUnionPullUpConstantsRule;
import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveWindowingFixRule;
+import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveWindowingLastValueRewrite;
import org.apache.hadoop.hive.ql.optimizer.calcite.rules.jdbc.JDBCAbstractSplitFilterRule;
import org.apache.hadoop.hive.ql.optimizer.calcite.rules.jdbc.JDBCAggregationPushDownRule;
import org.apache.hadoop.hive.ql.optimizer.calcite.rules.jdbc.JDBCExpandExpressionsRule;
@@ -2437,8 +2438,9 @@ public class CalcitePlanner extends SemanticAnalyzer {
if (profilesCBO.contains(ExtendedCBOProfile.WINDOWING_POSTPROCESSING)) {
generatePartialProgram(program, false, HepMatchOrder.DEPTH_FIRST,
HiveWindowingFixRule.INSTANCE);
+ generatePartialProgram(program, false, HepMatchOrder.DEPTH_FIRST,
+ HiveWindowingLastValueRewrite.INSTANCE);
}
-
// 6. Sort predicates in filter expressions
if (conf.getBoolVar(HiveConf.ConfVars.HIVE_OPTIMIZE_SORT_PREDS_WITH_STATS)) {
generatePartialProgram(program, false, HepMatchOrder.DEPTH_FIRST,
diff --git a/ql/src/test/queries/clientpositive/vector_ptf_last_value_streaming.q b/ql/src/test/queries/clientpositive/vector_ptf_last_value_streaming.q
new file mode 100644
index 0000000..04d565b
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/vector_ptf_last_value_streaming.q
@@ -0,0 +1,9 @@
+set hive.query.results.cache.enabled=false;
+create temporary table test2(id STRING,name STRING,event_dt date) stored as orc;
+
+insert into test2 values ('100','A','2019-08-15'), ('100','A','2019-10-12');
+
+explain vectorization detail SELECT name, event_dt, last_value(event_dt) over (PARTITION BY name ORDER BY event_dt desc ROWS BETWEEN unbounded preceding and unbounded following) last_event_dt FROM test2;
+SELECT name, event_dt, last_value(event_dt) over (PARTITION BY name ORDER BY event_dt desc ROWS BETWEEN unbounded preceding and unbounded following) last_event_dt FROM test2;
+SELECT name, event_dt, last_value(event_dt) over (PARTITION BY name ORDER BY event_dt desc ROWS BETWEEN unbounded preceding and current row) last_event_dt FROM test2;
+SELECT name, event_dt, first_value(event_dt) over (PARTITION BY name ORDER BY event_dt asc ROWS BETWEEN unbounded preceding and current row) last_event_dt FROM test2;
diff --git a/ql/src/test/results/clientpositive/llap/vector_ptf_last_value_streaming.q.out b/ql/src/test/results/clientpositive/llap/vector_ptf_last_value_streaming.q.out
new file mode 100644
index 0000000..6ca64d9
--- /dev/null
+++ b/ql/src/test/results/clientpositive/llap/vector_ptf_last_value_streaming.q.out
@@ -0,0 +1,192 @@
+PREHOOK: query: create temporary table test2(id STRING,name STRING,event_dt date) stored as orc
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@test2
+POSTHOOK: query: create temporary table test2(id STRING,name STRING,event_dt date) stored as orc
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@test2
+PREHOOK: query: insert into test2 values ('100','A','2019-08-15'), ('100','A','2019-10-12')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@test2
+POSTHOOK: query: insert into test2 values ('100','A','2019-08-15'), ('100','A','2019-10-12')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@test2
+POSTHOOK: Lineage: test2.event_dt SCRIPT []
+POSTHOOK: Lineage: test2.id SCRIPT []
+POSTHOOK: Lineage: test2.name SCRIPT []
+PREHOOK: query: explain vectorization detail SELECT name, event_dt, last_value(event_dt) over (PARTITION BY name ORDER BY event_dt desc ROWS BETWEEN unbounded preceding and unbounded following) last_event_dt FROM test2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test2
+#### A masked pattern was here ####
+POSTHOOK: query: explain vectorization detail SELECT name, event_dt, last_value(event_dt) over (PARTITION BY name ORDER BY event_dt desc ROWS BETWEEN unbounded preceding and unbounded following) last_event_dt FROM test2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@test2
+#### A masked pattern was here ####
+PLAN VECTORIZATION:
+ enabled: true
+ enabledConditionsMet: [hive.vectorized.execution.enabled IS true]
+
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+
+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: test2
+ Statistics: Num rows: 2 Data size: 480 Basic stats: COMPLETE Column stats: NONE
+ TableScan Vectorization:
+ native: true
+ vectorizationSchemaColumns: [0:id:string, 1:name:string, 2:event_dt:date, 3:ROW__ID:struct<writeid:bigint,bucketid:int,rowid:bigint>]
+ Reduce Output Operator
+ key expressions: name (type: string), event_dt (type: date)
+ null sort order: az
+ sort order: ++
+ Map-reduce partition columns: name (type: string)
+ Reduce Sink Vectorization:
+ className: VectorReduceSinkObjectHashOperator
+ keyColumns: 1:string, 2:date
+ native: true
+ nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
+ partitionColumns: 1:string
+ Statistics: Num rows: 2 Data size: 480 Basic stats: COMPLETE Column stats: NONE
+ Execution mode: vectorized, llap
+ LLAP IO: all inputs
+ Map Vectorization:
+ enabled: true
+ enabledConditionsMet: hive.vectorized.use.vectorized.input.format IS true
+ inputFormatFeatureSupport: [DECIMAL_64]
+ featureSupportInUse: [DECIMAL_64]
+ inputFileFormats: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+ allNative: true
+ usesVectorUDFAdaptor: false
+ vectorized: true
+ rowBatchContext:
+ dataColumnCount: 3
+ includeColumns: [1, 2]
+ dataColumns: id:string, name:string, event_dt:date
+ partitionColumnCount: 0
+ scratchColumnTypeNames: []
+ Reducer 2
+ Execution mode: vectorized, llap
+ Reduce Vectorization:
+ enabled: true
+ enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true
+ reduceColumnNullOrder: az
+ reduceColumnSortOrder: ++
+ allNative: false
+ usesVectorUDFAdaptor: false
+ vectorized: true
+ rowBatchContext:
+ dataColumnCount: 2
+ dataColumns: KEY.reducesinkkey0:string, KEY.reducesinkkey1:date
+ partitionColumnCount: 0
+ scratchColumnTypeNames: [bigint]
+ Reduce Operator Tree:
+ Select Operator
+ expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: date)
+ outputColumnNames: _col1, _col2
+ Select Vectorization:
+ className: VectorSelectOperator
+ native: true
+ projectedOutputColumnNums: [0, 1]
+ Statistics: Num rows: 2 Data size: 480 Basic stats: COMPLETE Column stats: NONE
+ PTF Operator
+ Function definitions:
+ Input definition
+ input alias: ptf_0
+ output shape: _col1: string, _col2: date
+ type: WINDOWING
+ Windowing table definition
+ input alias: ptf_1
+ name: windowingtablefunction
+ order by: _col2 ASC NULLS LAST
+ partition by: _col1
+ raw input shape:
+ window functions:
+ window function definition
+ alias: first_value_window_0
+ arguments: _col2
+ name: first_value
+ window function: GenericUDAFFirstValueEvaluator
+ window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX)
+ PTF Vectorization:
+ className: VectorPTFOperator
+ evaluatorClasses: [VectorPTFEvaluatorLongFirstValue]
+ functionInputExpressions: [col 1:date]
+ functionNames: [first_value]
+ keyInputColumns: [0, 1]
+ native: true
+ nonKeyInputColumns: []
+ orderExpressions: [col 1:date]
+ outputColumns: [2, 0, 1]
+ outputTypes: [date, string, date]
+ partitionExpressions: [col 0:string]
+ streamingColumns: [2]
+ Statistics: Num rows: 2 Data size: 480 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col1 (type: string), _col2 (type: date), first_value_window_0 (type: date)
+ outputColumnNames: _col0, _col1, _col2
+ Select Vectorization:
+ className: VectorSelectOperator
+ native: true
+ projectedOutputColumnNums: [0, 1, 2]
+ Statistics: Num rows: 2 Data size: 480 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ File Sink Vectorization:
+ className: VectorFileSinkOperator
+ native: false
+ Statistics: Num rows: 2 Data size: 480 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: SELECT name, event_dt, last_value(event_dt) over (PARTITION BY name ORDER BY event_dt desc ROWS BETWEEN unbounded preceding and unbounded following) last_event_dt FROM test2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test2
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT name, event_dt, last_value(event_dt) over (PARTITION BY name ORDER BY event_dt desc ROWS BETWEEN unbounded preceding and unbounded following) last_event_dt FROM test2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@test2
+#### A masked pattern was here ####
+A 2019-08-15 2019-08-15
+A 2019-10-12 2019-08-15
+PREHOOK: query: SELECT name, event_dt, last_value(event_dt) over (PARTITION BY name ORDER BY event_dt desc ROWS BETWEEN unbounded preceding and current row) last_event_dt FROM test2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test2
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT name, event_dt, last_value(event_dt) over (PARTITION BY name ORDER BY event_dt desc ROWS BETWEEN unbounded preceding and current row) last_event_dt FROM test2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@test2
+#### A masked pattern was here ####
+A 2019-10-12 2019-10-12
+A 2019-08-15 2019-08-15
+PREHOOK: query: SELECT name, event_dt, first_value(event_dt) over (PARTITION BY name ORDER BY event_dt asc ROWS BETWEEN unbounded preceding and current row) last_event_dt FROM test2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test2
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT name, event_dt, first_value(event_dt) over (PARTITION BY name ORDER BY event_dt asc ROWS BETWEEN unbounded preceding and current row) last_event_dt FROM test2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@test2
+#### A masked pattern was here ####
+A 2019-08-15 2019-08-15
+A 2019-10-12 2019-08-15
diff --git a/ql/src/test/results/clientpositive/llap/vector_ptf_part_simple.q.out b/ql/src/test/results/clientpositive/llap/vector_ptf_part_simple.q.out
index 51c6288..13c5655 100644
--- a/ql/src/test/results/clientpositive/llap/vector_ptf_part_simple.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_ptf_part_simple.q.out
@@ -130,6 +130,7 @@ STAGE PLANS:
#### A masked pattern was here ####
Edges:
Reducer 2 <- Map 1 (SIMPLE_EDGE)
+ Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
#### A masked pattern was here ####
Vertices:
Map 1
@@ -177,14 +178,14 @@ STAGE PLANS:
enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true
reduceColumnNullOrder: a
reduceColumnSortOrder: +
- allNative: false
+ allNative: true
usesVectorUDFAdaptor: false
vectorized: true
rowBatchContext:
dataColumnCount: 3
dataColumns: KEY.reducesinkkey0:string, VALUE._col0:string, VALUE._col1:double
partitionColumnCount: 0
- scratchColumnTypeNames: [bigint, bigint, bigint, double, double, bigint, bigint]
+ scratchColumnTypeNames: [bigint, bigint, bigint, double, bigint, bigint]
Reduce Operator Tree:
Select Operator
expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string), VALUE._col1 (type: double)
@@ -234,12 +235,6 @@ STAGE PLANS:
window function: GenericUDAFFirstValueEvaluator
window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX)
window function definition
- alias: last_value_window_4
- arguments: _col2
- name: last_value
- window function: GenericUDAFLastValueEvaluator
- window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX)
- window function definition
alias: count_window_5
arguments: _col2
name: count
@@ -253,31 +248,108 @@ STAGE PLANS:
isStar: true
PTF Vectorization:
className: VectorPTFOperator
- evaluatorClasses: [VectorPTFEvaluatorRowNumber, VectorPTFEvaluatorRank, VectorPTFEvaluatorDenseRank, VectorPTFEvaluatorDoubleFirstValue, VectorPTFEvaluatorDoubleLastValue, VectorPTFEvaluatorCount, VectorPTFEvaluatorCountStar]
- functionInputExpressions: [null, col 0:string, col 0:string, col 2:double, col 2:double, col 2:double, null]
- functionNames: [row_number, rank, dense_rank, first_value, last_value, count, count]
+ evaluatorClasses: [VectorPTFEvaluatorRowNumber, VectorPTFEvaluatorRank, VectorPTFEvaluatorDenseRank, VectorPTFEvaluatorDoubleFirstValue, VectorPTFEvaluatorCount, VectorPTFEvaluatorCountStar]
+ functionInputExpressions: [null, col 0:string, col 0:string, col 2:double, col 2:double, null]
+ functionNames: [row_number, rank, dense_rank, first_value, count, count]
keyInputColumns: [0]
native: true
nonKeyInputColumns: [1, 2]
orderExpressions: [col 0:string]
- outputColumns: [3, 4, 5, 6, 7, 8, 9, 0, 1, 2]
- outputTypes: [int, int, int, double, double, bigint, bigint, string, string, double]
+ outputColumns: [3, 4, 5, 6, 7, 8, 0, 1, 2]
+ outputTypes: [int, int, int, double, bigint, bigint, string, string, double]
streamingColumns: [3, 4, 5, 6]
Statistics: Num rows: 40 Data size: 19816 Basic stats: COMPLETE Column stats: COMPLETE
Select Operator
- expressions: _col0 (type: string), _col1 (type: string), _col2 (type: double), row_number_window_0 (type: int), rank_window_1 (type: int), dense_rank_window_2 (type: int), first_value_window_3 (type: double), last_value_window_4 (type: double), count_window_5 (type: bigint), count_window_6 (type: bigint)
+ expressions: row_number_window_0 (type: int), rank_window_1 (type: int), dense_rank_window_2 (type: int), first_value_window_3 (type: double), count_window_5 (type: bigint), count_window_6 (type: bigint), _col0 (type: string), _col1 (type: string), _col2 (type: double)
+ outputColumnNames: row_number_window_0, rank_window_1, dense_rank_window_2, first_value_window_3, count_window_5, count_window_6, _col0, _col1, _col2
+ Select Vectorization:
+ className: VectorSelectOperator
+ native: true
+ projectedOutputColumnNums: [3, 4, 5, 6, 7, 8, 0, 1, 2]
+ Statistics: Num rows: 40 Data size: 19816 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ null sort order: z
+ sort order: -
+ Map-reduce partition columns: _col0 (type: string)
+ Reduce Sink Vectorization:
+ className: VectorReduceSinkStringOperator
+ keyColumns: 0:string
+ native: true
+ nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
+ valueColumns: 3:int, 4:int, 5:int, 6:double, 7:bigint, 8:bigint, 1:string, 2:double
+ Statistics: Num rows: 40 Data size: 19816 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: row_number_window_0 (type: int), rank_window_1 (type: int), dense_rank_window_2 (type: int), first_value_window_3 (type: double), count_window_5 (type: bigint), count_window_6 (type: bigint), _col1 (type: string), _col2 (type: double)
+ Reducer 3
+ Execution mode: vectorized, llap
+ Reduce Vectorization:
+ enabled: true
+ enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true
+ reduceColumnNullOrder: z
+ reduceColumnSortOrder: -
+ allNative: false
+ usesVectorUDFAdaptor: false
+ vectorized: true
+ rowBatchContext:
+ dataColumnCount: 9
+ dataColumns: KEY.reducesinkkey0:string, VALUE._col0:int, VALUE._col1:int, VALUE._col2:int, VALUE._col3:double, VALUE._col4:bigint, VALUE._col5:bigint, VALUE._col6:string, VALUE._col7:double
+ partitionColumnCount: 0
+ scratchColumnTypeNames: [double]
+ Reduce Operator Tree:
+ Select Operator
+ expressions: VALUE._col0 (type: int), VALUE._col1 (type: int), VALUE._col2 (type: int), VALUE._col3 (type: double), VALUE._col4 (type: bigint), VALUE._col5 (type: bigint), KEY.reducesinkkey0 (type: string), VALUE._col6 (type: string), VALUE._col7 (type: double)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
+ Select Vectorization:
+ className: VectorSelectOperator
+ native: true
+ projectedOutputColumnNums: [1, 2, 3, 4, 5, 6, 0, 7, 8]
+ Statistics: Num rows: 40 Data size: 21224 Basic stats: COMPLETE Column stats: COMPLETE
+ PTF Operator
+ Function definitions:
+ Input definition
+ input alias: ptf_0
+ output shape: _col0: int, _col1: int, _col2: int, _col3: double, _col4: bigint, _col5: bigint, _col6: string, _col7: string, _col8: double
+ type: WINDOWING
+ Windowing table definition
+ input alias: ptf_1
+ name: windowingtablefunction
+ order by: _col6 DESC NULLS LAST
+ partition by: _col6
+ raw input shape:
+ window functions:
+ window function definition
+ alias: first_value_window_4
+ arguments: _col8
+ name: first_value
+ window function: GenericUDAFFirstValueEvaluator
+ window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX)
+ PTF Vectorization:
+ className: VectorPTFOperator
+ evaluatorClasses: [VectorPTFEvaluatorDoubleFirstValue]
+ functionInputExpressions: [col 8:double]
+ functionNames: [first_value]
+ keyInputColumns: [0]
+ native: true
+ nonKeyInputColumns: [1, 2, 3, 4, 5, 6, 7, 8]
+ orderExpressions: [col 0:string]
+ outputColumns: [9, 1, 2, 3, 4, 5, 6, 0, 7, 8]
+ outputTypes: [double, int, int, int, double, bigint, bigint, string, string, double]
+ streamingColumns: [9]
+ Statistics: Num rows: 40 Data size: 21224 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: _col6 (type: string), _col7 (type: string), _col8 (type: double), _col0 (type: int), _col1 (type: int), _col2 (type: int), _col3 (type: double), first_value_window_4 (type: double), _col4 (type: bigint), _col5 (type: bigint)
outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
Select Vectorization:
className: VectorSelectOperator
native: true
- projectedOutputColumnNums: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9]
- Statistics: Num rows: 40 Data size: 10824 Basic stats: COMPLETE Column stats: COMPLETE
+ projectedOutputColumnNums: [0, 7, 8, 1, 2, 3, 4, 9, 5, 6]
+ Statistics: Num rows: 40 Data size: 10760 Basic stats: COMPLETE Column stats: COMPLETE
File Output Operator
compressed: false
File Sink Vectorization:
className: VectorFileSinkOperator
native: false
- Statistics: Num rows: 40 Data size: 10824 Basic stats: COMPLETE Column stats: COMPLETE
+ Statistics: Num rows: 40 Data size: 10760 Basic stats: COMPLETE Column stats: COMPLETE
table:
input format: org.apache.hadoop.mapred.SequenceFileInputFormat
output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -314,46 +386,46 @@ POSTHOOK: type: QUERY
POSTHOOK: Input: default@vector_ptf_part_simple_orc
#### A masked pattern was here ####
p_mfgr p_name p_retailprice rn r dr fv lv c cs
-Manufacturer#2 almond aquamarine rose maroon antique 900.66 1 1 1 900.66 2031.98 8 8
-Manufacturer#2 almond aquamarine rose maroon antique 1698.66 2 1 1 900.66 2031.98 8 8
-Manufacturer#2 almond antique violet turquoise frosted 1800.7 3 1 1 900.66 2031.98 8 8
-Manufacturer#2 almond antique violet chocolate turquoise 1690.68 4 1 1 900.66 2031.98 8 8
-Manufacturer#2 almond antique violet turquoise frosted 1800.7 5 1 1 900.66 2031.98 8 8
-Manufacturer#2 almond antique violet turquoise frosted 1800.7 6 1 1 900.66 2031.98 8 8
-Manufacturer#2 almond aquamarine sandy cyan gainsboro 1000.6 7 1 1 900.66 2031.98 8 8
-Manufacturer#2 almond aquamarine midnight light salmon 2031.98 8 1 1 900.66 2031.98 8 8
Manufacturer#3 almond antique forest lavender goldenrod 1190.27 1 1 1 1190.27 1190.27 7 8
-Manufacturer#3 almond antique chartreuse khaki white 99.68 2 1 1 1190.27 1190.27 7 8
-Manufacturer#3 almond antique forest lavender goldenrod NULL 3 1 1 1190.27 1190.27 7 8
-Manufacturer#3 almond antique metallic orange dim 55.39 4 1 1 1190.27 1190.27 7 8
-Manufacturer#3 almond antique misty red olive 1922.98 5 1 1 1190.27 1190.27 7 8
-Manufacturer#3 almond antique forest lavender goldenrod 590.27 6 1 1 1190.27 1190.27 7 8
-Manufacturer#3 almond antique olive coral navajo 1337.29 7 1 1 1190.27 1190.27 7 8
Manufacturer#3 almond antique forest lavender goldenrod 1190.27 8 1 1 1190.27 1190.27 7 8
-Manufacturer#4 almond antique gainsboro frosted violet NULL 1 1 1 NULL 1290.35 4 6
-Manufacturer#4 almond aquamarine floral ivory bisque NULL 2 1 1 NULL 1290.35 4 6
-Manufacturer#4 almond antique violet mint lemon 1375.42 3 1 1 NULL 1290.35 4 6
-Manufacturer#4 almond aquamarine yellow dodger mint 1844.92 4 1 1 NULL 1290.35 4 6
-Manufacturer#4 almond aquamarine floral ivory bisque 1206.26 5 1 1 NULL 1290.35 4 6
-Manufacturer#4 almond azure aquamarine papaya violet 1290.35 6 1 1 NULL 1290.35 4 6
+Manufacturer#3 almond antique olive coral navajo 1337.29 7 1 1 1190.27 1190.27 7 8
+Manufacturer#3 almond antique forest lavender goldenrod 590.27 6 1 1 1190.27 1190.27 7 8
+Manufacturer#3 almond antique misty red olive 1922.98 5 1 1 1190.27 1190.27 7 8
+Manufacturer#3 almond antique metallic orange dim 55.39 4 1 1 1190.27 1190.27 7 8
+Manufacturer#3 almond antique forest lavender goldenrod NULL 3 1 1 1190.27 1190.27 7 8
+Manufacturer#3 almond antique chartreuse khaki white 99.68 2 1 1 1190.27 1190.27 7 8
+Manufacturer#1 almond aquamarine pink moccasin thistle 1632.66 1 1 1 1632.66 1632.66 11 12
+Manufacturer#1 almond antique chartreuse lavender yellow 1753.76 2 1 1 1632.66 1632.66 11 12
+Manufacturer#1 almond aquamarine pink moccasin thistle 1632.66 3 1 1 1632.66 1632.66 11 12
+Manufacturer#1 almond antique chartreuse lavender yellow 1753.76 4 1 1 1632.66 1632.66 11 12
+Manufacturer#1 almond aquamarine pink moccasin thistle 1632.66 5 1 1 1632.66 1632.66 11 12
+Manufacturer#1 almond antique chartreuse lavender yellow 1753.76 6 1 1 1632.66 1632.66 11 12
+Manufacturer#1 almond antique salmon chartreuse burlywood 1602.59 7 1 1 1632.66 1632.66 11 12
+Manufacturer#1 almond antique burnished rose metallic 1173.15 8 1 1 1632.66 1632.66 11 12
+Manufacturer#1 almond aquamarine burnished black steel 1414.42 9 1 1 1632.66 1632.66 11 12
+Manufacturer#1 almond aquamarine pink moccasin thistle NULL 10 1 1 1632.66 1632.66 11 12
+Manufacturer#1 almond antique chartreuse lavender yellow 1753.76 11 1 1 1632.66 1632.66 11 12
+Manufacturer#1 almond antique burnished rose metallic 1173.15 12 1 1 1632.66 1632.66 11 12
+Manufacturer#5 almond antique blue firebrick mint 1789.69 6 1 1 1611.66 1789.69 6 6
Manufacturer#5 almond antique medium spring khaki 1611.66 1 1 1 1611.66 1789.69 6 6
Manufacturer#5 almond aquamarine dodger light gainsboro 1018.1 2 1 1 1611.66 1789.69 6 6
Manufacturer#5 almond azure blanched chiffon midnight 1464.48 3 1 1 1611.66 1789.69 6 6
Manufacturer#5 almond antique medium spring khaki 1611.66 4 1 1 1611.66 1789.69 6 6
Manufacturer#5 almond antique sky peru orange 1788.73 5 1 1 1611.66 1789.69 6 6
-Manufacturer#5 almond antique blue firebrick mint 1789.69 6 1 1 1611.66 1789.69 6 6
-Manufacturer#1 almond aquamarine pink moccasin thistle 1632.66 1 1 1 1632.66 1173.15 11 12
-Manufacturer#1 almond antique chartreuse lavender yellow 1753.76 2 1 1 1632.66 1173.15 11 12
-Manufacturer#1 almond aquamarine pink moccasin thistle 1632.66 3 1 1 1632.66 1173.15 11 12
-Manufacturer#1 almond antique chartreuse lavender yellow 1753.76 4 1 1 1632.66 1173.15 11 12
-Manufacturer#1 almond aquamarine pink moccasin thistle 1632.66 5 1 1 1632.66 1173.15 11 12
-Manufacturer#1 almond antique chartreuse lavender yellow 1753.76 6 1 1 1632.66 1173.15 11 12
-Manufacturer#1 almond antique salmon chartreuse burlywood 1602.59 7 1 1 1632.66 1173.15 11 12
-Manufacturer#1 almond antique burnished rose metallic 1173.15 8 1 1 1632.66 1173.15 11 12
-Manufacturer#1 almond aquamarine burnished black steel 1414.42 9 1 1 1632.66 1173.15 11 12
-Manufacturer#1 almond aquamarine pink moccasin thistle NULL 10 1 1 1632.66 1173.15 11 12
-Manufacturer#1 almond antique chartreuse lavender yellow 1753.76 11 1 1 1632.66 1173.15 11 12
-Manufacturer#1 almond antique burnished rose metallic 1173.15 12 1 1 1632.66 1173.15 11 12
+Manufacturer#4 almond aquamarine floral ivory bisque 1206.26 5 1 1 NULL 1206.26 4 6
+Manufacturer#4 almond azure aquamarine papaya violet 1290.35 6 1 1 NULL 1206.26 4 6
+Manufacturer#4 almond antique gainsboro frosted violet NULL 1 1 1 NULL 1206.26 4 6
+Manufacturer#4 almond aquamarine floral ivory bisque NULL 2 1 1 NULL 1206.26 4 6
+Manufacturer#4 almond antique violet mint lemon 1375.42 3 1 1 NULL 1206.26 4 6
+Manufacturer#4 almond aquamarine yellow dodger mint 1844.92 4 1 1 NULL 1206.26 4 6
+Manufacturer#2 almond antique violet turquoise frosted 1800.7 3 1 1 900.66 1800.7 8 8
+Manufacturer#2 almond aquamarine rose maroon antique 1698.66 2 1 1 900.66 1800.7 8 8
+Manufacturer#2 almond aquamarine rose maroon antique 900.66 1 1 1 900.66 1800.7 8 8
+Manufacturer#2 almond aquamarine midnight light salmon 2031.98 8 1 1 900.66 1800.7 8 8
+Manufacturer#2 almond aquamarine sandy cyan gainsboro 1000.6 7 1 1 900.66 1800.7 8 8
+Manufacturer#2 almond antique violet turquoise frosted 1800.7 6 1 1 900.66 1800.7 8 8
+Manufacturer#2 almond antique violet turquoise frosted 1800.7 5 1 1 900.66 1800.7 8 8
+Manufacturer#2 almond antique violet chocolate turquoise 1690.68 4 1 1 900.66 1800.7 8 8
PREHOOK: query: explain vectorization detail
select p_mfgr,p_name, p_retailprice,
row_number() over(partition by p_mfgr range between unbounded preceding and unbounded following) as rn,
@@ -6256,46 +6328,46 @@ POSTHOOK: type: QUERY
POSTHOOK: Input: default@vector_ptf_part_simple_orc
#### A masked pattern was here ####
p_mfgr p_name p_retailprice rn r dr fv lv c cs
-Manufacturer#2 almond aquamarine rose maroon antique 900.66 1 1 1 900.66 2031.98 8 8
-Manufacturer#2 almond aquamarine rose maroon antique 1698.66 2 1 1 900.66 2031.98 8 8
-Manufacturer#2 almond antique violet turquoise frosted 1800.7 3 1 1 900.66 2031.98 8 8
-Manufacturer#2 almond antique violet chocolate turquoise 1690.68 4 1 1 900.66 2031.98 8 8
-Manufacturer#2 almond antique violet turquoise frosted 1800.7 5 1 1 900.66 2031.98 8 8
-Manufacturer#2 almond antique violet turquoise frosted 1800.7 6 1 1 900.66 2031.98 8 8
-Manufacturer#2 almond aquamarine sandy cyan gainsboro 1000.6 7 1 1 900.66 2031.98 8 8
-Manufacturer#2 almond aquamarine midnight light salmon 2031.98 8 1 1 900.66 2031.98 8 8
Manufacturer#3 almond antique forest lavender goldenrod 1190.27 1 1 1 1190.27 1190.27 7 8
-Manufacturer#3 almond antique chartreuse khaki white 99.68 2 1 1 1190.27 1190.27 7 8
-Manufacturer#3 almond antique forest lavender goldenrod NULL 3 1 1 1190.27 1190.27 7 8
-Manufacturer#3 almond antique metallic orange dim 55.39 4 1 1 1190.27 1190.27 7 8
-Manufacturer#3 almond antique misty red olive 1922.98 5 1 1 1190.27 1190.27 7 8
-Manufacturer#3 almond antique forest lavender goldenrod 590.27 6 1 1 1190.27 1190.27 7 8
-Manufacturer#3 almond antique olive coral navajo 1337.29 7 1 1 1190.27 1190.27 7 8
Manufacturer#3 almond antique forest lavender goldenrod 1190.27 8 1 1 1190.27 1190.27 7 8
-Manufacturer#4 almond antique gainsboro frosted violet NULL 1 1 1 NULL 1290.35 4 6
-Manufacturer#4 almond aquamarine floral ivory bisque NULL 2 1 1 NULL 1290.35 4 6
-Manufacturer#4 almond antique violet mint lemon 1375.42 3 1 1 NULL 1290.35 4 6
-Manufacturer#4 almond aquamarine yellow dodger mint 1844.92 4 1 1 NULL 1290.35 4 6
-Manufacturer#4 almond aquamarine floral ivory bisque 1206.26 5 1 1 NULL 1290.35 4 6
-Manufacturer#4 almond azure aquamarine papaya violet 1290.35 6 1 1 NULL 1290.35 4 6
+Manufacturer#3 almond antique olive coral navajo 1337.29 7 1 1 1190.27 1190.27 7 8
+Manufacturer#3 almond antique forest lavender goldenrod 590.27 6 1 1 1190.27 1190.27 7 8
+Manufacturer#3 almond antique misty red olive 1922.98 5 1 1 1190.27 1190.27 7 8
+Manufacturer#3 almond antique metallic orange dim 55.39 4 1 1 1190.27 1190.27 7 8
+Manufacturer#3 almond antique forest lavender goldenrod NULL 3 1 1 1190.27 1190.27 7 8
+Manufacturer#3 almond antique chartreuse khaki white 99.68 2 1 1 1190.27 1190.27 7 8
+Manufacturer#1 almond aquamarine pink moccasin thistle 1632.66 1 1 1 1632.66 1632.66 11 12
+Manufacturer#1 almond antique chartreuse lavender yellow 1753.76 2 1 1 1632.66 1632.66 11 12
+Manufacturer#1 almond aquamarine pink moccasin thistle 1632.66 3 1 1 1632.66 1632.66 11 12
+Manufacturer#1 almond antique chartreuse lavender yellow 1753.76 4 1 1 1632.66 1632.66 11 12
+Manufacturer#1 almond aquamarine pink moccasin thistle 1632.66 5 1 1 1632.66 1632.66 11 12
+Manufacturer#1 almond antique chartreuse lavender yellow 1753.76 6 1 1 1632.66 1632.66 11 12
+Manufacturer#1 almond antique salmon chartreuse burlywood 1602.59 7 1 1 1632.66 1632.66 11 12
+Manufacturer#1 almond antique burnished rose metallic 1173.15 8 1 1 1632.66 1632.66 11 12
+Manufacturer#1 almond aquamarine burnished black steel 1414.42 9 1 1 1632.66 1632.66 11 12
+Manufacturer#1 almond aquamarine pink moccasin thistle NULL 10 1 1 1632.66 1632.66 11 12
+Manufacturer#1 almond antique chartreuse lavender yellow 1753.76 11 1 1 1632.66 1632.66 11 12
+Manufacturer#1 almond antique burnished rose metallic 1173.15 12 1 1 1632.66 1632.66 11 12
+Manufacturer#5 almond antique blue firebrick mint 1789.69 6 1 1 1611.66 1789.69 6 6
Manufacturer#5 almond antique medium spring khaki 1611.66 1 1 1 1611.66 1789.69 6 6
Manufacturer#5 almond aquamarine dodger light gainsboro 1018.1 2 1 1 1611.66 1789.69 6 6
Manufacturer#5 almond azure blanched chiffon midnight 1464.48 3 1 1 1611.66 1789.69 6 6
Manufacturer#5 almond antique medium spring khaki 1611.66 4 1 1 1611.66 1789.69 6 6
Manufacturer#5 almond antique sky peru orange 1788.73 5 1 1 1611.66 1789.69 6 6
-Manufacturer#5 almond antique blue firebrick mint 1789.69 6 1 1 1611.66 1789.69 6 6
-Manufacturer#1 almond aquamarine pink moccasin thistle 1632.66 1 1 1 1632.66 1173.15 11 12
-Manufacturer#1 almond antique chartreuse lavender yellow 1753.76 2 1 1 1632.66 1173.15 11 12
-Manufacturer#1 almond aquamarine pink moccasin thistle 1632.66 3 1 1 1632.66 1173.15 11 12
-Manufacturer#1 almond antique chartreuse lavender yellow 1753.76 4 1 1 1632.66 1173.15 11 12
-Manufacturer#1 almond aquamarine pink moccasin thistle 1632.66 5 1 1 1632.66 1173.15 11 12
-Manufacturer#1 almond antique chartreuse lavender yellow 1753.76 6 1 1 1632.66 1173.15 11 12
-Manufacturer#1 almond antique salmon chartreuse burlywood 1602.59 7 1 1 1632.66 1173.15 11 12
-Manufacturer#1 almond antique burnished rose metallic 1173.15 8 1 1 1632.66 1173.15 11 12
-Manufacturer#1 almond aquamarine burnished black steel 1414.42 9 1 1 1632.66 1173.15 11 12
-Manufacturer#1 almond aquamarine pink moccasin thistle NULL 10 1 1 1632.66 1173.15 11 12
-Manufacturer#1 almond antique chartreuse lavender yellow 1753.76 11 1 1 1632.66 1173.15 11 12
-Manufacturer#1 almond antique burnished rose metallic 1173.15 12 1 1 1632.66 1173.15 11 12
+Manufacturer#4 almond aquamarine floral ivory bisque 1206.26 5 1 1 NULL 1206.26 4 6
+Manufacturer#4 almond azure aquamarine papaya violet 1290.35 6 1 1 NULL 1206.26 4 6
+Manufacturer#4 almond antique gainsboro frosted violet NULL 1 1 1 NULL 1206.26 4 6
+Manufacturer#4 almond aquamarine floral ivory bisque NULL 2 1 1 NULL 1206.26 4 6
+Manufacturer#4 almond antique violet mint lemon 1375.42 3 1 1 NULL 1206.26 4 6
+Manufacturer#4 almond aquamarine yellow dodger mint 1844.92 4 1 1 NULL 1206.26 4 6
+Manufacturer#2 almond antique violet turquoise frosted 1800.7 3 1 1 900.66 1800.7 8 8
+Manufacturer#2 almond aquamarine rose maroon antique 1698.66 2 1 1 900.66 1800.7 8 8
+Manufacturer#2 almond aquamarine rose maroon antique 900.66 1 1 1 900.66 1800.7 8 8
+Manufacturer#2 almond aquamarine midnight light salmon 2031.98 8 1 1 900.66 1800.7 8 8
+Manufacturer#2 almond aquamarine sandy cyan gainsboro 1000.6 7 1 1 900.66 1800.7 8 8
+Manufacturer#2 almond antique violet turquoise frosted 1800.7 6 1 1 900.66 1800.7 8 8
+Manufacturer#2 almond antique violet turquoise frosted 1800.7 5 1 1 900.66 1800.7 8 8
+Manufacturer#2 almond antique violet chocolate turquoise 1690.68 4 1 1 900.66 1800.7 8 8
PREHOOK: query: select p_mfgr,p_name, p_retailprice,
row_number() over(partition by p_mfgr order by p_name) as rn,
rank() over(partition by p_mfgr order by p_name) as r,