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 2020/08/08 01:22:54 UTC
[hive] branch master updated: HIVE-23716: Support Anti Join in Hive
(Mahesh Kumar Behera, reviewed by Vineet Garg, Ramesh Kumar Thangarajan,
Panagiotis Garefalakis, Jesus Camacho Rodriguez)
This is an automated email from the ASF dual-hosted git repository.
jcamacho 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 14c653a HIVE-23716: Support Anti Join in Hive (Mahesh Kumar Behera, reviewed by Vineet Garg, Ramesh Kumar Thangarajan, Panagiotis Garefalakis, Jesus Camacho Rodriguez)
14c653a is described below
commit 14c653a5fed76649b128945f266c350a9db324af
Author: mahesh kumar behera <ma...@apache.org>
AuthorDate: Sat Aug 8 06:52:37 2020 +0530
HIVE-23716: Support Anti Join in Hive (Mahesh Kumar Behera, reviewed by Vineet Garg, Ramesh Kumar Thangarajan, Panagiotis Garefalakis, Jesus Camacho Rodriguez)
Closes apache/hive#1147
---
.../java/org/apache/hadoop/hive/conf/HiveConf.java | 3 +-
.../mapjoin/MapJoinMultiKeyBench.java | 2 +-
.../test/resources/testconfiguration.properties | 1 +
.../apache/hadoop/hive/ql/parse/FromClauseParser.g | 1 +
.../org/apache/hadoop/hive/ql/parse/HintParser.g | 1 +
.../org/apache/hadoop/hive/ql/parse/HiveParser.g | 1 +
.../hadoop/hive/ql/exec/CommonJoinOperator.java | 61 +-
.../apache/hadoop/hive/ql/exec/JoinOperator.java | 2 +-
...ectorMapJoinAntiJoinGenerateResultOperator.java | 227 +++
.../mapjoin/VectorMapJoinAntiJoinLongOperator.java | 307 ++++
.../VectorMapJoinAntiJoinMultiKeyOperator.java | 392 +++++
.../VectorMapJoinAntiJoinStringOperator.java | 363 +++++
.../ql/optimizer/ConstantPropagateProcFactory.java | 2 +-
.../hive/ql/optimizer/ConvertJoinMapJoin.java | 1 +
.../hadoop/hive/ql/optimizer/MapJoinProcessor.java | 3 +-
.../hive/ql/optimizer/calcite/HiveCalciteUtil.java | 23 +
.../ql/optimizer/calcite/HiveRelFactories.java | 5 +
.../HiveRelOptMaterializationValidator.java | 10 +
.../hive/ql/optimizer/calcite/HiveRelOptUtil.java | 7 +
.../calcite/HiveSubQRemoveRelBuilder.java | 11 +-
.../calcite/reloperators/HiveAntiJoin.java | 97 ++
.../calcite/rules/HiveAntiSemiJoinRule.java | 160 ++
.../calcite/rules/HiveJoinAddNotNullRule.java | 16 +-
.../calcite/rules/HiveJoinConstraintsRule.java | 9 +-
.../rules/HiveJoinProjectTransposeRule.java | 8 +
.../HiveJoinPushTransitivePredicatesRule.java | 4 +
.../rules/HiveProjectJoinTransposeRule.java | 3 +-
.../calcite/rules/HiveRelDecorrelator.java | 58 +-
.../calcite/rules/HiveRemoveGBYSemiJoinRule.java | 26 +-
.../calcite/rules/HiveSubQueryRemoveRule.java | 30 +-
.../calcite/stats/EstimateUniqueKeys.java | 9 +
.../calcite/stats/HiveRelMdDistinctRowCount.java | 8 +
.../calcite/stats/HiveRelMdPredicates.java | 5 +-
.../optimizer/calcite/stats/HiveRelMdRowCount.java | 33 +-
.../calcite/stats/HiveRelMdSelectivity.java | 10 +-
.../ql/optimizer/calcite/stats/HiveRelMdSize.java | 16 +-
.../optimizer/calcite/translator/ASTBuilder.java | 3 +
.../optimizer/calcite/translator/ASTConverter.java | 2 +-
.../calcite/translator/PlanModifierForASTConv.java | 8 +-
.../translator/opconventer/HiveOpConverter.java | 5 +-
.../translator/opconventer/JoinVisitor.java | 28 +-
.../correlation/CorrelationOptimizer.java | 3 +-
.../hive/ql/optimizer/physical/Vectorizer.java | 16 +
.../stats/annotation/StatsRulesProcFactory.java | 12 +
.../hadoop/hive/ql/parse/CalcitePlanner.java | 39 +-
.../org/apache/hadoop/hive/ql/parse/JoinType.java | 2 +-
.../hadoop/hive/ql/parse/SemanticAnalyzer.java | 12 +-
.../apache/hadoop/hive/ql/plan/JoinCondDesc.java | 9 +
.../org/apache/hadoop/hive/ql/plan/JoinDesc.java | 1 +
.../hadoop/hive/ql/plan/VectorMapJoinDesc.java | 3 +-
.../hive/ql/ppd/PredicateTransitivePropagate.java | 1 +
.../hadoop/hive/ql/ppd/SyntheticJoinPredicate.java | 6 +
.../ql/exec/vector/mapjoin/MapJoinTestConfig.java | 6 +-
.../ql/exec/vector/mapjoin/MapJoinTestData.java | 1 +
.../exec/vector/mapjoin/TestMapJoinOperator.java | 6 +-
ql/src/test/queries/clientpositive/antijoin.q | 67 +
ql/src/test/queries/clientpositive/explainuser_1.q | 43 +
.../clientpositive/external_jdbc_table_perf.q | 193 +++
ql/src/test/queries/clientpositive/lineage3.q | 7 +
.../queries/clientpositive/subquery_in_having.q | 16 +
.../test/queries/clientpositive/subquery_multi.q | 42 +
.../test/queries/clientpositive/subquery_scalar.q | 6 +-
.../results/clientpositive/llap/antijoin.q.out | 1701 ++++++++++++++++++++
.../clientpositive/llap/explainuser_1.q.out | 418 ++++-
.../llap/external_jdbc_table_perf.q.out | 1116 ++++++++++++-
.../results/clientpositive/llap/lineage3.q.out | 15 +-
.../clientpositive/llap/subquery_in_having.q.out | 368 +++++
.../clientpositive/llap/subquery_multi.q.out | 1079 +++++++++++++
.../clientpositive/llap/subquery_notexists.q.out | 333 ++--
.../llap/subquery_notexists_having.q.out | 147 +-
.../clientpositive/llap/subquery_scalar.q.out | 144 ++
.../clientpositive/perf/tez/cbo_query16.q.out | 50 +-
.../clientpositive/perf/tez/cbo_query69.q.out | 83 +-
.../clientpositive/perf/tez/cbo_query78.q.out | 45 +-
.../clientpositive/perf/tez/cbo_query94.q.out | 50 +-
.../perf/tez/constraints/cbo_query16.q.out | 48 +-
.../perf/tez/constraints/cbo_query69.q.out | 85 +-
.../perf/tez/constraints/cbo_query78.q.out | 39 +-
.../perf/tez/constraints/cbo_query94.q.out | 48 +-
.../perf/tez/constraints/query16.q.out | 218 ++-
.../perf/tez/constraints/query69.q.out | 346 ++--
.../perf/tez/constraints/query78.q.out | 303 ++--
.../perf/tez/constraints/query94.q.out | 218 ++-
.../results/clientpositive/perf/tez/query16.q.out | 222 ++-
.../results/clientpositive/perf/tez/query69.q.out | 354 ++--
.../results/clientpositive/perf/tez/query78.q.out | 327 ++--
.../results/clientpositive/perf/tez/query94.q.out | 222 ++-
87 files changed, 8530 insertions(+), 1901 deletions(-)
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 58615e1..742dddf 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -2249,7 +2249,8 @@ public class HiveConf extends Configuration {
"Whether Hive enables the optimization about converting common join into mapjoin based on the input file size. \n" +
"If this parameter is on, and the sum of size for n-1 of the tables/partitions for a n-way join is smaller than the\n" +
"specified size, the join is directly converted to a mapjoin (there is no conditional task)."),
-
+ HIVE_CONVERT_ANTI_JOIN("hive.auto.convert.anti.join", true,
+ "Whether Hive enables the optimization about converting join with null filter to anti join"),
HIVECONVERTJOINNOCONDITIONALTASKTHRESHOLD("hive.auto.convert.join.noconditionaltask.size",
10000000L,
"If hive.auto.convert.join.noconditionaltask is off, this parameter does not take affect. \n" +
diff --git a/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/mapjoin/MapJoinMultiKeyBench.java b/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/mapjoin/MapJoinMultiKeyBench.java
index ca76e6c..d54002a 100644
--- a/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/mapjoin/MapJoinMultiKeyBench.java
+++ b/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/mapjoin/MapJoinMultiKeyBench.java
@@ -36,7 +36,7 @@ import org.openjdk.jmh.runner.options.OptionsBuilder;
* From itests/hive-jmh directory, run:
* java -jar target/benchmarks.jar org.apache.hive.benchmark.vectorization.mapjoin.MapJoinMultiKeyBench
*
- * {INNER, INNER_BIG_ONLY, LEFT_SEMI, OUTER}
+ * {INNER, INNER_BIG_ONLY, LEFT_SEMI, OUTER, ANTI}
* X
* {ROW_MODE_HASH_MAP, ROW_MODE_OPTIMIZED, VECTOR_PASS_THROUGH, NATIVE_VECTOR_OPTIMIZED, NATIVE_VECTOR_FAST}
*
diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index e2cee9c..5d69d29 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -342,6 +342,7 @@ spark.query.files=\
alter_merge_orc.q,\
alter_merge_stats_orc.q,\
annotate_stats_join.q,\
+ antijoin.q,\
auto_join0.q,\
auto_join1.q,\
auto_join10.q,\
diff --git a/parser/src/java/org/apache/hadoop/hive/ql/parse/FromClauseParser.g b/parser/src/java/org/apache/hadoop/hive/ql/parse/FromClauseParser.g
index e6b6fd6..b1a0b62 100644
--- a/parser/src/java/org/apache/hadoop/hive/ql/parse/FromClauseParser.g
+++ b/parser/src/java/org/apache/hadoop/hive/ql/parse/FromClauseParser.g
@@ -145,6 +145,7 @@ joinToken
| KW_RIGHT (KW_OUTER)? KW_JOIN -> TOK_RIGHTOUTERJOIN
| KW_FULL (KW_OUTER)? KW_JOIN -> TOK_FULLOUTERJOIN
| KW_LEFT KW_SEMI KW_JOIN -> TOK_LEFTSEMIJOIN
+ | KW_LEFT KW_ANTI KW_JOIN -> TOK_LEFTANTISEMIJOIN
;
lateralView
diff --git a/parser/src/java/org/apache/hadoop/hive/ql/parse/HintParser.g b/parser/src/java/org/apache/hadoop/hive/ql/parse/HintParser.g
index e66b28d..f9607fc 100644
--- a/parser/src/java/org/apache/hadoop/hive/ql/parse/HintParser.g
+++ b/parser/src/java/org/apache/hadoop/hive/ql/parse/HintParser.g
@@ -33,6 +33,7 @@ tokens {
TOK_HINTARGLIST;
TOK_LEFTSEMIJOIN;
TOK_PKFK_JOIN;
+ TOK_LEFTANTISEMIJOIN;
}
@header {
diff --git a/parser/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g b/parser/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
index 7a47e2d..4805323 100644
--- a/parser/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
+++ b/parser/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
@@ -304,6 +304,7 @@ TOK_USERSCRIPTCOLSCHEMA;
TOK_RECORDREADER;
TOK_RECORDWRITER;
TOK_LEFTSEMIJOIN;
+TOK_LEFTANTISEMIJOIN;
TOK_LATERAL_VIEW;
TOK_LATERAL_VIEW_OUTER;
TOK_TABALIAS;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/CommonJoinOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/CommonJoinOperator.java
index 7a7c8a5..888aa85 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/CommonJoinOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/CommonJoinOperator.java
@@ -153,6 +153,8 @@ public abstract class CommonJoinOperator<T extends JoinDesc> extends
transient boolean hasLeftSemiJoin = false;
+ transient boolean hasLeftAntiSemiJoin = false;
+
protected transient int countAfterReport;
protected transient int heartbeatInterval;
protected static final int NOTSKIPBIGTABLE = -1;
@@ -365,6 +367,8 @@ public abstract class CommonJoinOperator<T extends JoinDesc> extends
for( int i = 0; i < condn.length; i++ ) {
if(condn[i].getType() == JoinDesc.LEFT_SEMI_JOIN) {
hasLeftSemiJoin = true;
+ } else if(condn[i].getType() == JoinDesc.ANTI_JOIN) {
+ hasLeftAntiSemiJoin = true;
}
}
@@ -509,11 +513,17 @@ public abstract class CommonJoinOperator<T extends JoinDesc> extends
}
}
+ private void createForwardJoinObjectForAntiJoin(boolean[] skip) throws HiveException {
+ boolean forward = fillForwardCache(skip);
+ if (forward) {
+ internalForward(forwardCache, outputObjInspector);
+ countAfterReport = 0;
+ }
+ }
+
// fill forwardCache with skipvector
- // returns whether a record was forwarded
- private boolean createForwardJoinObject(boolean[] skip) throws HiveException {
+ private boolean fillForwardCache(boolean[] skip) {
Arrays.fill(forwardCache, null);
-
boolean forward = false;
for (int i = 0; i < numAliases; i++) {
if (!skip[i]) {
@@ -523,11 +533,19 @@ public abstract class CommonJoinOperator<T extends JoinDesc> extends
forward = true;
}
}
+ return forward;
+ }
+
+ // returns whether a record was forwarded
+ private boolean createForwardJoinObject(boolean[] skip, boolean antiJoin) throws HiveException {
+ boolean forward = fillForwardCache(skip);
if (forward) {
if (needsPostEvaluation) {
forward = !JoinUtil.isFiltered(forwardCache, residualJoinFilters, residualJoinFiltersOIs);
}
- if (forward) {
+
+ // For anti join, check all right side and if nothing is matched then only forward.
+ if (forward && !antiJoin) {
// If it is not an outer join, or the post-condition filters
// are empty or the row passed them
internalForward(forwardCache, outputObjInspector);
@@ -620,10 +638,11 @@ public abstract class CommonJoinOperator<T extends JoinDesc> extends
boolean rightFirst = true;
AbstractRowContainer.RowIterator<List<Object>> iter = aliasRes.rowIter();
int pos = 0;
+
for (List<Object> rightObj = iter.first(); !done && rightObj != null;
rightObj = loopAgain ? rightObj : iter.next(), rightFirst = loopAgain = false, pos++) {
+ // Keep a copy of the skip vector and update the bit for current alias only in the loop.
System.arraycopy(prevSkip, 0, skip, 0, prevSkip.length);
-
boolean rightNull = rightObj == dummyObj[aliasNum];
if (hasFilter(order[aliasNum])) {
filterTags[aliasNum] = getFilterTag(rightObj);
@@ -638,6 +657,12 @@ public abstract class CommonJoinOperator<T extends JoinDesc> extends
// skipping the rest of the rows in the rhs table of the semijoin
done = !needsPostEvaluation;
}
+ } else if (type == JoinDesc.ANTI_JOIN) {
+ if (innerJoin(skip, left, right)) {
+ // if inner join found a match then the condition is not matched for anti join, so we can skip rest of the
+ // record. But if there is some post evaluation we have to handle that.
+ done = !needsPostEvaluation;
+ }
} else if (type == JoinDesc.LEFT_OUTER_JOIN ||
(type == JoinDesc.FULL_OUTER_JOIN && rightNull)) {
int result = leftOuterJoin(skip, left, right);
@@ -668,7 +693,8 @@ public abstract class CommonJoinOperator<T extends JoinDesc> extends
needToProduceLeftRow = true;
if (needsPostEvaluation) {
// This is only executed for outer joins with residual filters
- boolean forward = createForwardJoinObject(skipVectors[numAliases - 1]);
+ boolean forward = createForwardJoinObject(skipVectors[numAliases - 1],
+ type == JoinDesc.ANTI_JOIN);
producedRow |= forward;
done = (type == JoinDesc.LEFT_SEMI_JOIN) && forward;
if (!rightNull &&
@@ -687,7 +713,7 @@ public abstract class CommonJoinOperator<T extends JoinDesc> extends
}
}
} else {
- createForwardJoinObject(skipVectors[numAliases - 1]);
+ createForwardJoinObject(skipVectors[numAliases - 1], type == JoinDesc.ANTI_JOIN);
}
}
} else {
@@ -696,6 +722,17 @@ public abstract class CommonJoinOperator<T extends JoinDesc> extends
}
}
+ // For anti join, we should proceed to emit records if the right side is empty or not matching.
+ if (type == JoinDesc.ANTI_JOIN && !producedRow) {
+ System.arraycopy(prevSkip, 0, skip, 0, prevSkip.length);
+ skip[right] = true;
+ if (aliasNum == numAliases - 1) {
+ createForwardJoinObjectForAntiJoin(skipVectors[numAliases - 1]);
+ } else {
+ genObject(aliasNum + 1, allLeftFirst, allLeftNull);
+ }
+ }
+
// Consolidation for outer joins
if (needsPostEvaluation && aliasNum == numAliases - 1 &&
needToProduceLeftRow && !producedRow && !allLeftNull) {
@@ -935,9 +972,17 @@ public abstract class CommonJoinOperator<T extends JoinDesc> extends
for (int i = 0; i < numAliases; i++) {
Byte alias = order[i];
AbstractRowContainer<List<Object>> alw = storage[alias];
+ boolean isRightOfAntiJoin = (i != 0 && condn[i-1].getType() == JoinDesc.ANTI_JOIN);
if (noOuterJoin) {
if (!alw.hasRows()) {
+ if (!isRightOfAntiJoin) {
+ // For anti join the right side can be empty.
+ return;
+ }
+ } else if (isRightOfAntiJoin && !needsPostEvaluation) {
+ // For anti join the right side should be empty. For needsPostEvaluation case we will
+ // wait till evaluation is done. For other cases we can directly return from here.
return;
} else if (!alw.isSingleRow()) {
mayHasMoreThanOne = true;
@@ -968,7 +1013,7 @@ public abstract class CommonJoinOperator<T extends JoinDesc> extends
if (!needsPostEvaluation && !hasEmpty && !mayHasMoreThanOne) {
genAllOneUniqueJoinObject();
- } else if (!needsPostEvaluation && !hasEmpty && !hasLeftSemiJoin) {
+ } else if (!needsPostEvaluation && !hasEmpty && !hasLeftSemiJoin && !hasLeftAntiSemiJoin) {
genUniqueJoinObject(0, 0);
} else {
genJoinObject();
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/JoinOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/JoinOperator.java
index 47ae047..306d6dd 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/JoinOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/JoinOperator.java
@@ -97,7 +97,7 @@ public class JoinOperator extends CommonJoinOperator<JoinDesc> implements Serial
List keyObject = (List) soi.getStructFieldData(row, sf);
// Are we consuming too much memory
if (alias == numAliases - 1 && !(handleSkewJoin && skewJoinKeyContext.currBigKeyTag >= 0) &&
- !hasLeftSemiJoin) {
+ !hasLeftSemiJoin && !hasLeftAntiSemiJoin) {
if (sz == joinEmitInterval && !hasFilter(condn[alias-1].getLeft()) &&
!hasFilter(condn[alias-1].getRight())) {
// The input is sorted by alias, so if we are already in the last join
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinAntiJoinGenerateResultOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinAntiJoinGenerateResultOperator.java
new file mode 100644
index 0000000..962f920
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinAntiJoinGenerateResultOperator.java
@@ -0,0 +1,227 @@
+/*
+ * 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.mapjoin;
+
+import org.apache.hadoop.hive.ql.CompilationOpContext;
+import org.apache.hadoop.hive.ql.exec.JoinUtil;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizationContext;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.mapjoin.hashtable.VectorMapJoinHashSet;
+import org.apache.hadoop.hive.ql.exec.vector.mapjoin.hashtable.VectorMapJoinHashSetResult;
+import org.apache.hadoop.hive.ql.exec.vector.mapjoin.hashtable.VectorMapJoinHashTableResult;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.plan.OperatorDesc;
+import org.apache.hadoop.hive.ql.plan.VectorDesc;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+// TODO : This class is duplicate of semi join. Need to do a refactoring to merge it with semi join.
+/**
+ * This class has methods for generating vectorized join results for Anti joins.
+ * The big difference between inner joins and anti joins is existence testing.
+ * Inner joins use a hash map to lookup the 1 or more small table values.
+ * Anti joins are a specialized join for outputting big table rows whose key does not
+ * exists in the small table.
+ *
+ * No small table values are needed for anti since they would be empty. So,
+ * we use a hash set as the hash table. Hash sets just report whether a key exists. This
+ * is a big performance optimization.
+ */
+public abstract class VectorMapJoinAntiJoinGenerateResultOperator
+ extends VectorMapJoinGenerateResultOperator {
+
+ private static final long serialVersionUID = 1L;
+ private static final Logger LOG = LoggerFactory.getLogger(VectorMapJoinAntiJoinGenerateResultOperator.class.getName());
+
+ // Anti join specific members.
+
+ // An array of hash set results so we can do lookups on the whole batch before output result
+ // generation.
+ protected transient VectorMapJoinHashSetResult hashSetResults[];
+
+ // Pre-allocated member for storing the (physical) batch index of matching row (single- or
+ // multi-small-table-valued) indexes during a process call.
+ protected transient int[] allMatchs;
+
+ // Pre-allocated member for storing the (physical) batch index of rows that need to be spilled.
+ protected transient int[] spills;
+
+ // Pre-allocated member for storing index into the hashSetResults for each spilled row.
+ protected transient int[] spillHashMapResultIndices;
+
+ /** Kryo ctor. */
+ protected VectorMapJoinAntiJoinGenerateResultOperator() {
+ super();
+ }
+
+ public VectorMapJoinAntiJoinGenerateResultOperator(CompilationOpContext ctx) {
+ super(ctx);
+ }
+
+ public VectorMapJoinAntiJoinGenerateResultOperator(CompilationOpContext ctx, OperatorDesc conf,
+ VectorizationContext vContext, VectorDesc vectorDesc) throws HiveException {
+ super(ctx, conf, vContext, vectorDesc);
+ }
+
+ /*
+ * Setup our anti join specific members.
+ */
+ protected void commonSetup() throws HiveException {
+ super.commonSetup();
+
+ // Anti join specific.
+ VectorMapJoinHashSet baseHashSet = (VectorMapJoinHashSet) vectorMapJoinHashTable;
+
+ hashSetResults = new VectorMapJoinHashSetResult[VectorizedRowBatch.DEFAULT_SIZE];
+ for (int i = 0; i < hashSetResults.length; i++) {
+ hashSetResults[i] = baseHashSet.createHashSetResult();
+ }
+
+ allMatchs = new int[VectorizedRowBatch.DEFAULT_SIZE];
+
+ spills = new int[VectorizedRowBatch.DEFAULT_SIZE];
+ spillHashMapResultIndices = new int[VectorizedRowBatch.DEFAULT_SIZE];
+ }
+
+ //-----------------------------------------------------------------------------------------------
+
+ /*
+ * Anti join (hash set).
+ */
+
+ /**
+ * Generate the anti join output results for one vectorized row batch. The result is modified during hash
+ * table match to reverse the result for anti join. So here matching means, the row can be emitted as the row
+ * is actually not matching.
+ *
+ * @param batch
+ * The big table batch with any matching and any non matching rows both as
+ * selected in use.
+ * @param allMatchCount
+ * Number of matches in allMatches.
+ * @param spillCount
+ * Number of spills in spills.
+ * @param hashTableResults
+ * The array of all hash table results for the batch. We need the
+ * VectorMapJoinHashTableResult for the spill information.
+ */
+ protected void finishAnti(VectorizedRowBatch batch,
+ int allMatchCount, int spillCount,
+ VectorMapJoinHashTableResult[] hashTableResults) throws HiveException, IOException {
+
+ // Get rid of spills before we start modifying the batch.
+ if (spillCount > 0) {
+ spillHashMapBatch(batch, hashTableResults,
+ spills, spillHashMapResultIndices, spillCount);
+ }
+
+ /*
+ * Optimize by running value expressions only over the matched rows.
+ */
+ if (allMatchCount > 0 && bigTableValueExpressions != null) {
+ performValueExpressions(batch, allMatchs, allMatchCount);
+ }
+
+ batch.size = generateHashSetResults(batch, allMatchs, allMatchCount);
+ batch.selectedInUse = true;
+ }
+
+ /**
+ * Generate the matching anti join output results of a vectorized row batch.
+ *
+ * @param batch
+ * The big table batch.
+ * @param allMatches
+ * A subset of the rows of the batch that are matches.
+ * @param allMatchCount
+ * Number of matches in allMatches.
+ */
+ private int generateHashSetResults(VectorizedRowBatch batch,
+ int[] allMatches, int allMatchCount) {
+ int numSel = 0;
+ // Generate result within big table batch itself.
+ for (int i = 0; i < allMatchCount; i++) {
+ int batchIndex = allMatchs[i];
+ // Use the big table row as output.
+ batch.selected[numSel++] = batchIndex;
+ }
+ return numSel;
+ }
+
+ protected JoinUtil.JoinResult inverseResultForAntiJoin(JoinUtil.JoinResult joinResult) {
+ if (joinResult == JoinUtil.JoinResult.NOMATCH) {
+ return JoinUtil.JoinResult.MATCH;
+ } else if (joinResult == JoinUtil.JoinResult.MATCH) {
+ return JoinUtil.JoinResult.NOMATCH;
+ }
+ return joinResult;
+ }
+
+ /**
+ * Generate the anti join output results for one vectorized row batch with a repeated key.
+ *
+ * @param batch
+ * The big table batch whose repeated key matches.
+ */
+ protected int generateHashSetResultRepeatedAll(VectorizedRowBatch batch) {
+ if (batch.selectedInUse) {
+ // The selected array is already filled in as we want it.
+ } else {
+ int[] selected = batch.selected;
+ for (int i = 0; i < batch.size; i++) {
+ selected[i] = i;
+ }
+ batch.selectedInUse = true;
+ }
+ return batch.size;
+ }
+
+ protected void finishAntiRepeated(VectorizedRowBatch batch, JoinUtil.JoinResult joinResult,
+ VectorMapJoinHashTableResult hashSetResult) throws HiveException, IOException {
+ switch (joinResult) {
+ case MATCH:
+
+ if (bigTableValueExpressions != null) {
+ // Run our value expressions over whole batch.
+ for(VectorExpression ve: bigTableValueExpressions) {
+ ve.evaluate(batch);
+ }
+ }
+
+ // Generate special repeated case.
+ batch.size = generateHashSetResultRepeatedAll(batch);
+ batch.selectedInUse = true;
+ break;
+
+ case SPILL:
+ // Whole batch is spilled.
+ spillBatchRepeated(batch, hashSetResult);
+ batch.size = 0;
+ break;
+
+ case NOMATCH:
+ // No match for entire batch.
+ batch.size = 0;
+ break;
+ }
+ }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinAntiJoinLongOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinAntiJoinLongOperator.java
new file mode 100644
index 0000000..ea7d52d
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinAntiJoinLongOperator.java
@@ -0,0 +1,307 @@
+/*
+ * 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.mapjoin;
+
+import org.apache.hadoop.hive.ql.CompilationOpContext;
+import org.apache.hadoop.hive.ql.exec.JoinUtil;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizationContext;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.mapjoin.hashtable.VectorMapJoinLongHashSet;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.plan.OperatorDesc;
+import org.apache.hadoop.hive.ql.plan.VectorDesc;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Arrays;
+
+// TODO : Duplicate codes need to merge with semi join.
+// Single-Column Long hash table import.
+// Single-Column Long specific imports.
+
+/*
+ * Specialized class for doing a vectorized map join that is an anti join on a Single-Column Long
+ * using a hash set.
+ */
+public class VectorMapJoinAntiJoinLongOperator extends VectorMapJoinAntiJoinGenerateResultOperator {
+
+ private static final long serialVersionUID = 1L;
+ private static final String CLASS_NAME = VectorMapJoinAntiJoinLongOperator.class.getName();
+ private static final Logger LOG = LoggerFactory.getLogger(CLASS_NAME);
+ protected String getLoggingPrefix() {
+ return super.getLoggingPrefix(CLASS_NAME);
+ }
+
+ // The above members are initialized by the constructor and must not be
+ // transient.
+
+ // The hash map for this specialized class.
+ private transient VectorMapJoinLongHashSet hashSet;
+
+ // Single-Column Long specific members.
+ // For integers, we have optional min/max filtering.
+ private transient boolean useMinMax;
+ private transient long min;
+ private transient long max;
+
+ // The column number for this one column join specialization.
+ private transient int singleJoinColumn;
+
+ // Pass-thru constructors.
+ /** Kryo ctor. */
+ protected VectorMapJoinAntiJoinLongOperator() {
+ super();
+ }
+
+ public VectorMapJoinAntiJoinLongOperator(CompilationOpContext ctx) {
+ super(ctx);
+ }
+
+ public VectorMapJoinAntiJoinLongOperator(CompilationOpContext ctx, OperatorDesc conf,
+ VectorizationContext vContext, VectorDesc vectorDesc) throws HiveException {
+ super(ctx, conf, vContext, vectorDesc);
+ }
+
+ // Process Single-Column Long Anti Join on a vectorized row batch.
+ @Override
+ protected void commonSetup() throws HiveException {
+ super.commonSetup();
+
+ // Initialize Single-Column Long members for this specialized class.
+ singleJoinColumn = bigTableKeyColumnMap[0];
+ }
+
+ @Override
+ public void hashTableSetup() throws HiveException {
+ super.hashTableSetup();
+
+ // Get our Single-Column Long hash set information for this specialized class.
+ hashSet = (VectorMapJoinLongHashSet) vectorMapJoinHashTable;
+ useMinMax = hashSet.useMinMax();
+ if (useMinMax) {
+ min = hashSet.min();
+ max = hashSet.max();
+ }
+ }
+
+ @Override
+ public void processBatch(VectorizedRowBatch batch) throws HiveException {
+
+ try {
+ // (Currently none)
+ // antiPerBatchSetup(batch);
+
+ // For anti joins, we may apply the filter(s) now.
+ for(VectorExpression ve : bigTableFilterExpressions) {
+ ve.evaluate(batch);
+ }
+
+ final int inputLogicalSize = batch.size;
+ if (inputLogicalSize == 0) {
+ return;
+ }
+
+ // Perform any key expressions. Results will go into scratch columns.
+ if (bigTableKeyExpressions != null) {
+ for (VectorExpression ve : bigTableKeyExpressions) {
+ ve.evaluate(batch);
+ }
+ }
+
+ // The one join column for this specialized class.
+ LongColumnVector joinColVector = (LongColumnVector) batch.cols[singleJoinColumn];
+ long[] vector = joinColVector.vector;
+
+ // Check single column for repeating.
+ boolean allKeyInputColumnsRepeating = joinColVector.isRepeating;
+
+ if (allKeyInputColumnsRepeating) {
+ // All key input columns are repeating. Generate key once. Lookup once.
+ // Since the key is repeated, we must use entry 0 regardless of selectedInUse.
+ JoinUtil.JoinResult joinResult;
+ if (!joinColVector.noNulls && joinColVector.isNull[0]) {
+ // For anti join, if the right side is null then its a match.
+ joinResult = JoinUtil.JoinResult.MATCH;
+ } else {
+ long key = vector[0];
+ if (useMinMax && (key < min || key > max)) {
+ // Out of range for whole batch. Its a match for anti join. We can emit the row.
+ joinResult = JoinUtil.JoinResult.MATCH;
+ } else {
+ joinResult = hashSet.contains(key, hashSetResults[0]);
+ // reverse the join result for anti join.
+ joinResult = inverseResultForAntiJoin(joinResult);
+ }
+ }
+
+ // Common repeated join result processing.
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(CLASS_NAME + " batch #" + batchCounter + " repeated joinResult " + joinResult.name());
+ }
+ finishAntiRepeated(batch, joinResult, hashSetResults[0]);
+ } else {
+ // NOT Repeating.
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(CLASS_NAME + " batch #" + batchCounter + " non-repeated");
+ }
+
+ // We remember any matching rows in matches / matchSize. At the end of the loop,
+ // selected / batch.size will represent both matching and non-matching rows for outer join.
+ // Only deferred rows will have been removed from selected.
+ int selected[] = batch.selected;
+ boolean selectedInUse = batch.selectedInUse;
+
+ int hashSetResultCount = 0;
+ int allMatchCount = 0;
+ int spillCount = 0;
+ long saveKey = 0;
+
+ // We optimize performance by only looking up the first key in a series of equal keys.
+ boolean haveSaveKey = false;
+ JoinUtil.JoinResult saveJoinResult = JoinUtil.JoinResult.NOMATCH;
+
+ // Logical loop over the rows in the batch since the batch may have selected in use.
+ for (int logical = 0; logical < inputLogicalSize; logical++) {
+ int batchIndex = (selectedInUse ? selected[logical] : logical);
+
+ // Single-Column Long get key.
+ long currentKey;
+ boolean isNull;
+ if (!joinColVector.noNulls && joinColVector.isNull[batchIndex]) {
+ currentKey = 0;
+ isNull = true;
+ } else {
+ currentKey = vector[batchIndex];
+ isNull = false;
+ }
+
+ // Equal key series checking.
+ if (isNull || !haveSaveKey || currentKey != saveKey) {
+ // New key.
+ if (haveSaveKey) {
+ // Move on with our counts.
+ switch (saveJoinResult) {
+ case MATCH:
+ // We have extracted the existence from the hash set result, so we don't keep it.
+ break;
+ case SPILL:
+ // We keep the hash set result for its spill information.
+ hashSetResultCount++;
+ break;
+ case NOMATCH:
+ break;
+ }
+ }
+
+ if (isNull) {
+ saveJoinResult = JoinUtil.JoinResult.MATCH;
+ haveSaveKey = false;
+ } else {
+ // Regardless of our matching result, we keep that information to make multiple use
+ // of it for a possible series of equal keys.
+ haveSaveKey = true;
+ saveKey = currentKey;
+ if (useMinMax && (currentKey < min || currentKey > max)) {
+ saveJoinResult = JoinUtil.JoinResult.NOMATCH;
+ } else {
+ saveJoinResult = hashSet.contains(currentKey, hashSetResults[hashSetResultCount]);
+ }
+
+ // Reverse the match result for anti join.
+ // Key out of range for whole hash table, is a valid match for anti join.
+ saveJoinResult = inverseResultForAntiJoin(saveJoinResult);
+ }
+
+ // Common anti join result processing.
+ switch (saveJoinResult) {
+ case MATCH:
+ allMatchs[allMatchCount++] = batchIndex;
+ // VectorizedBatchUtil.debugDisplayOneRow(batch, batchIndex, CLASS_NAME + " MATCH isSingleValue " + equalKeySeriesIsSingleValue[equalKeySeriesCount] + " currentKey " + currentKey);
+ break;
+
+ case SPILL:
+ spills[spillCount] = batchIndex;
+ spillHashMapResultIndices[spillCount] = hashSetResultCount;
+ spillCount++;
+ break;
+
+ case NOMATCH:
+ // VectorizedBatchUtil.debugDisplayOneRow(batch, batchIndex, CLASS_NAME + " NOMATCH" + " currentKey " + currentKey);
+ break;
+ }
+ } else {
+ // Series of equal keys.
+ switch (saveJoinResult) {
+ case MATCH:
+ allMatchs[allMatchCount++] = batchIndex;
+ // VectorizedBatchUtil.debugDisplayOneRow(batch, batchIndex, CLASS_NAME + " MATCH duplicate");
+ break;
+
+ case SPILL:
+ spills[spillCount] = batchIndex;
+ spillHashMapResultIndices[spillCount] = hashSetResultCount;
+ spillCount++;
+ break;
+
+ case NOMATCH:
+ // VectorizedBatchUtil.debugDisplayOneRow(batch, batchIndex, CLASS_NAME + " NOMATCH duplicate");
+ break;
+ }
+ }
+ }
+
+ if (haveSaveKey) {
+ // Update our counts for the last key.
+ switch (saveJoinResult) {
+ case MATCH:
+ // We have extracted the existence from the hash set result, so we don't keep it.
+ break;
+ case SPILL:
+ // We keep the hash set result for its spill information.
+ hashSetResultCount++;
+ break;
+ case NOMATCH:
+ break;
+ }
+ }
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(CLASS_NAME +
+ " allMatchs " + intArrayToRangesString(allMatchs, allMatchCount) +
+ " spills " + intArrayToRangesString(spills, spillCount) +
+ " spillHashMapResultIndices " + intArrayToRangesString(spillHashMapResultIndices, spillCount) +
+ " hashMapResults " + Arrays.toString(Arrays.copyOfRange(hashSetResults, 0, hashSetResultCount)));
+ }
+
+ finishAnti(batch, allMatchCount, spillCount, hashSetResults);
+ }
+
+ if (batch.size > 0) {
+ // Forward any remaining selected rows.
+ forwardBigTableBatch(batch);
+ }
+
+ } catch (Exception e) {
+ throw new HiveException(e);
+ }
+ }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinAntiJoinMultiKeyOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinAntiJoinMultiKeyOperator.java
new file mode 100644
index 0000000..cbccb23
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinAntiJoinMultiKeyOperator.java
@@ -0,0 +1,392 @@
+/*
+ * 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.mapjoin;
+
+import org.apache.hadoop.hive.ql.CompilationOpContext;
+import org.apache.hadoop.hive.ql.exec.JoinUtil;
+import org.apache.hadoop.hive.ql.exec.vector.VectorSerializeRow;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizationContext;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.mapjoin.hashtable.VectorMapJoinBytesHashSet;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.plan.OperatorDesc;
+import org.apache.hadoop.hive.ql.plan.VectorDesc;
+import org.apache.hadoop.hive.serde2.ByteStream.Output;
+import org.apache.hadoop.hive.serde2.binarysortable.fast.BinarySortableSerializeWrite;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Arrays;
+
+// Multi-Key hash table import.
+// Multi-Key specific imports.
+
+// TODO : Duplicate codes need to merge with semi join.
+/*
+ * Specialized class for doing a vectorized map join that is an anti join on Multi-Key
+ * using hash set.
+ */
+public class VectorMapJoinAntiJoinMultiKeyOperator extends VectorMapJoinAntiJoinGenerateResultOperator {
+
+ private static final long serialVersionUID = 1L;
+
+ //------------------------------------------------------------------------------------------------
+
+ private static final String CLASS_NAME = VectorMapJoinAntiJoinMultiKeyOperator.class.getName();
+ private static final Logger LOG = LoggerFactory.getLogger(CLASS_NAME);
+
+ protected String getLoggingPrefix() {
+ return super.getLoggingPrefix(CLASS_NAME);
+ }
+
+ //------------------------------------------------------------------------------------------------
+
+ // (none)
+
+ // The above members are initialized by the constructor and must not be
+ // transient.
+ //---------------------------------------------------------------------------
+
+ // The hash map for this specialized class.
+ private transient VectorMapJoinBytesHashSet hashSet;
+
+ //---------------------------------------------------------------------------
+ // Multi-Key specific members.
+ //
+
+ // Object that can take a set of columns in row in a vectorized row batch and serialized it.
+ // Known to not have any nulls.
+ private transient VectorSerializeRow keyVectorSerializeWrite;
+
+ // The BinarySortable serialization of the current key.
+ private transient Output currentKeyOutput;
+
+ // The BinarySortable serialization of the saved key for a possible series of equal keys.
+ private transient Output saveKeyOutput;
+
+ //---------------------------------------------------------------------------
+ // Pass-thru constructors.
+ //
+
+ /** Kryo ctor. */
+ protected VectorMapJoinAntiJoinMultiKeyOperator() {
+ super();
+ }
+
+ public VectorMapJoinAntiJoinMultiKeyOperator(CompilationOpContext ctx) {
+ super(ctx);
+ }
+
+ public VectorMapJoinAntiJoinMultiKeyOperator(CompilationOpContext ctx, OperatorDesc conf,
+ VectorizationContext vContext, VectorDesc vectorDesc) throws HiveException {
+ super(ctx, conf, vContext, vectorDesc);
+ }
+
+ //---------------------------------------------------------------------------
+ // Process Multi-Key Anti Join on a vectorized row batch.
+ //
+
+ @Override
+ protected void commonSetup() throws HiveException {
+ super.commonSetup();
+
+ /*
+ * Initialize Multi-Key members for this specialized class.
+ */
+
+ keyVectorSerializeWrite = new VectorSerializeRow(BinarySortableSerializeWrite.with(
+ this.getConf().getKeyTblDesc().getProperties(), bigTableKeyColumnMap.length));
+ keyVectorSerializeWrite.init(bigTableKeyTypeInfos, bigTableKeyColumnMap);
+
+ currentKeyOutput = new Output();
+ saveKeyOutput = new Output();
+ }
+
+ @Override
+ public void hashTableSetup() throws HiveException {
+ super.hashTableSetup();
+
+ /*
+ * Get our Multi-Key hash set information for this specialized class.
+ */
+
+ hashSet = (VectorMapJoinBytesHashSet) vectorMapJoinHashTable;
+ }
+
+ @Override
+ public void processBatch(VectorizedRowBatch batch) throws HiveException {
+
+ try {
+
+ // Do the per-batch setup for an anti join.
+
+ // (Currently none)
+ // antiPerBatchSetup(batch);
+
+ // For anti joins, we may apply the filter(s) now.
+ for(VectorExpression ve : bigTableFilterExpressions) {
+ ve.evaluate(batch);
+ }
+
+ final int inputLogicalSize = batch.size;
+ if (inputLogicalSize == 0) {
+ return;
+ }
+
+ // Perform any key expressions. Results will go into scratch columns.
+ if (bigTableKeyExpressions != null) {
+ for (VectorExpression ve : bigTableKeyExpressions) {
+ ve.evaluate(batch);
+ }
+ }
+
+ /*
+ * Multi-Key specific declarations.
+ */
+
+ // None.
+
+ /*
+ * Multi-Key Long check for repeating.
+ */
+
+ // If all BigTable input columns to key expressions are isRepeating, then
+ // calculate key once; lookup once.
+ boolean allKeyInputColumnsRepeating;
+ if (bigTableKeyColumnMap.length == 0) {
+ allKeyInputColumnsRepeating = false;
+ } else {
+ allKeyInputColumnsRepeating = true;
+ for (int i = 0; i < bigTableKeyColumnMap.length; i++) {
+ if (!batch.cols[bigTableKeyColumnMap[i]].isRepeating) {
+ allKeyInputColumnsRepeating = false;
+ break;
+ }
+ }
+ }
+
+ if (allKeyInputColumnsRepeating) {
+
+ /*
+ * Repeating.
+ */
+
+ // All key input columns are repeating. Generate key once. Lookup once.
+ // Since the key is repeated, we must use entry 0 regardless of selectedInUse.
+
+ /*
+ * Multi-Key specific repeated lookup.
+ */
+
+ keyVectorSerializeWrite.setOutput(currentKeyOutput);
+ keyVectorSerializeWrite.serializeWrite(batch, 0);
+ JoinUtil.JoinResult joinResult;
+ if (keyVectorSerializeWrite.getHasAnyNulls()) {
+ // If right side is null, its a match for anti join.
+ joinResult = JoinUtil.JoinResult.MATCH;
+ } else {
+ byte[] keyBytes = currentKeyOutput.getData();
+ int keyLength = currentKeyOutput.getLength();
+ // LOG.debug(CLASS_NAME + " processOp all " + displayBytes(keyBytes, 0, keyLength));
+ joinResult = hashSet.contains(keyBytes, 0, keyLength, hashSetResults[0]);
+ // reverse the join result from hash table for anti join.
+ joinResult = inverseResultForAntiJoin(joinResult);
+ }
+
+ /*
+ * Common repeated join result processing.
+ */
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(CLASS_NAME + " batch #" + batchCounter + " repeated joinResult " + joinResult.name());
+ }
+ finishAntiRepeated(batch, joinResult, hashSetResults[0]);
+ } else {
+
+ /*
+ * NOT Repeating.
+ */
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(CLASS_NAME + " batch #" + batchCounter + " non-repeated");
+ }
+
+ // We remember any matching rows in matches / matchSize. At the end of the loop,
+ // selected / batch.size will represent both matching and non-matching rows for outer join.
+ // Only deferred rows will have been removed from selected.
+ int selected[] = batch.selected;
+ boolean selectedInUse = batch.selectedInUse;
+
+ int hashSetResultCount = 0;
+ int allMatchCount = 0;
+ int spillCount = 0;
+
+ /*
+ * Multi-Key specific variables.
+ */
+
+ Output temp;
+
+ // We optimize performance by only looking up the first key in a series of equal keys.
+ boolean haveSaveKey = false;
+ JoinUtil.JoinResult saveJoinResult = JoinUtil.JoinResult.NOMATCH;
+
+ // Logical loop over the rows in the batch since the batch may have selected in use.
+ for (int logical = 0; logical < inputLogicalSize; logical++) {
+ int batchIndex = (selectedInUse ? selected[logical] : logical);
+
+ /*
+ * Multi-Key get key.
+ */
+
+ // Generate binary sortable key for current row in vectorized row batch.
+ keyVectorSerializeWrite.setOutput(currentKeyOutput);
+ keyVectorSerializeWrite.serializeWrite(batch, batchIndex);
+ boolean isAnyNull = keyVectorSerializeWrite.getHasAnyNulls();
+
+ // LOG.debug(CLASS_NAME + " currentKey " +
+ // VectorizedBatchUtil.displayBytes(currentKeyOutput.getData(), 0, currentKeyOutput.getLength()));
+
+ /*
+ * Equal key series checking.
+ */
+
+ if (isAnyNull || !haveSaveKey || !saveKeyOutput.arraysEquals(currentKeyOutput)) {
+
+ // New key.
+
+ if (haveSaveKey) {
+ // Move on with our counts.
+ switch (saveJoinResult) {
+ case MATCH:
+ // We have extracted the existence from the hash set result, so we don't keep it.
+ break;
+ case SPILL:
+ // We keep the hash set result for its spill information.
+ hashSetResultCount++;
+ break;
+ case NOMATCH:
+ break;
+ }
+ }
+
+ if (isAnyNull) {
+ saveJoinResult = JoinUtil.JoinResult.MATCH;
+ haveSaveKey = false;
+ } else {
+ // Regardless of our matching result, we keep that information to make multiple use
+ // of it for a possible series of equal keys.
+ haveSaveKey = true;
+
+ /*
+ * Multi-Key specific save key and lookup.
+ */
+
+ temp = saveKeyOutput;
+ saveKeyOutput = currentKeyOutput;
+ currentKeyOutput = temp;
+
+ /*
+ * Multi-key specific lookup key.
+ */
+
+ byte[] keyBytes = saveKeyOutput.getData();
+ int keyLength = saveKeyOutput.getLength();
+ saveJoinResult = hashSet.contains(keyBytes, 0, keyLength, hashSetResults[hashSetResultCount]);
+ saveJoinResult = inverseResultForAntiJoin(saveJoinResult);
+ }
+
+ /*
+ * Common anti join result processing.
+ */
+
+ switch (saveJoinResult) {
+ case MATCH:
+ allMatchs[allMatchCount++] = batchIndex;
+ // VectorizedBatchUtil.debugDisplayOneRow(batch, batchIndex, CLASS_NAME + " MATCH isSingleValue " + equalKeySeriesIsSingleValue[equalKeySeriesCount] + " currentKey " + currentKey);
+ break;
+
+ case SPILL:
+ spills[spillCount] = batchIndex;
+ spillHashMapResultIndices[spillCount] = hashSetResultCount;
+ spillCount++;
+ break;
+
+ case NOMATCH:
+ // VectorizedBatchUtil.debugDisplayOneRow(batch, batchIndex, CLASS_NAME + " NOMATCH" + " currentKey " + currentKey);
+ break;
+ }
+ } else {
+ // Series of equal keys.
+
+ switch (saveJoinResult) {
+ case MATCH:
+ allMatchs[allMatchCount++] = batchIndex;
+ // VectorizedBatchUtil.debugDisplayOneRow(batch, batchIndex, CLASS_NAME + " MATCH duplicate");
+ break;
+
+ case SPILL:
+ spills[spillCount] = batchIndex;
+ spillHashMapResultIndices[spillCount] = hashSetResultCount;
+ spillCount++;
+ break;
+
+ case NOMATCH:
+ // VectorizedBatchUtil.debugDisplayOneRow(batch, batchIndex, CLASS_NAME + " NOMATCH duplicate");
+ break;
+ }
+ }
+ }
+
+ if (haveSaveKey) {
+ // Update our counts for the last key.
+ switch (saveJoinResult) {
+ case MATCH:
+ // We have extracted the existence from the hash set result, so we don't keep it.
+ break;
+ case SPILL:
+ // We keep the hash set result for its spill information.
+ hashSetResultCount++;
+ break;
+ case NOMATCH:
+ break;
+ }
+ }
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(CLASS_NAME +
+ " allMatchs " + intArrayToRangesString(allMatchs, allMatchCount) +
+ " spills " + intArrayToRangesString(spills, spillCount) +
+ " spillHashMapResultIndices " + intArrayToRangesString(spillHashMapResultIndices, spillCount) +
+ " hashMapResults " + Arrays.toString(Arrays.copyOfRange(hashSetResults, 0, hashSetResultCount)));
+ }
+
+ finishAnti(batch, allMatchCount, spillCount, hashSetResults);
+ }
+
+ if (batch.size > 0) {
+ // Forward any remaining selected rows.
+ forwardBigTableBatch(batch);
+ }
+ } catch (Exception e) {
+ throw new HiveException(e);
+ }
+ }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinAntiJoinStringOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinAntiJoinStringOperator.java
new file mode 100644
index 0000000..4e6f247
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinAntiJoinStringOperator.java
@@ -0,0 +1,363 @@
+/*
+ * 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.mapjoin;
+
+import org.apache.hadoop.hive.ql.CompilationOpContext;
+import org.apache.hadoop.hive.ql.exec.JoinUtil;
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizationContext;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.StringExpr;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.mapjoin.hashtable.VectorMapJoinBytesHashSet;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.plan.OperatorDesc;
+import org.apache.hadoop.hive.ql.plan.VectorDesc;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Arrays;
+
+// Single-Column String hash table import.
+// Single-Column String specific imports.
+
+// TODO : Duplicate codes need to merge with semi join.
+/*
+ * Specialized class for doing a vectorized map join that is an anti join on a Single-Column String
+ * using a hash set.
+ */
+public class VectorMapJoinAntiJoinStringOperator extends VectorMapJoinAntiJoinGenerateResultOperator {
+
+ private static final long serialVersionUID = 1L;
+
+ //------------------------------------------------------------------------------------------------
+
+ private static final String CLASS_NAME = VectorMapJoinAntiJoinStringOperator.class.getName();
+ private static final Logger LOG = LoggerFactory.getLogger(CLASS_NAME);
+
+ protected String getLoggingPrefix() {
+ return super.getLoggingPrefix(CLASS_NAME);
+ }
+
+ //------------------------------------------------------------------------------------------------
+
+ // (none)
+
+ // The above members are initialized by the constructor and must not be
+ // transient.
+ //---------------------------------------------------------------------------
+
+ // The hash map for this specialized class.
+ private transient VectorMapJoinBytesHashSet hashSet;
+
+ //---------------------------------------------------------------------------
+ // Single-Column String specific members.
+ //
+
+ // The column number for this one column join specialization.
+ private transient int singleJoinColumn;
+
+ //---------------------------------------------------------------------------
+ // Pass-thru constructors.
+ //
+
+ /** Kryo ctor. */
+ protected VectorMapJoinAntiJoinStringOperator() {
+ super();
+ }
+
+ public VectorMapJoinAntiJoinStringOperator(CompilationOpContext ctx) {
+ super(ctx);
+ }
+
+ public VectorMapJoinAntiJoinStringOperator(CompilationOpContext ctx, OperatorDesc conf,
+ VectorizationContext vContext, VectorDesc vectorDesc) throws HiveException {
+ super(ctx, conf, vContext, vectorDesc);
+ }
+
+ //---------------------------------------------------------------------------
+ // Process Single-Column String anti Join on a vectorized row batch.
+ //
+
+ @Override
+ protected void commonSetup() throws HiveException {
+ super.commonSetup();
+
+ /*
+ * Initialize Single-Column String members for this specialized class.
+ */
+
+ singleJoinColumn = bigTableKeyColumnMap[0];
+ }
+
+ @Override
+ public void hashTableSetup() throws HiveException {
+ super.hashTableSetup();
+
+ /*
+ * Get our Single-Column String hash set information for this specialized class.
+ */
+
+ hashSet = (VectorMapJoinBytesHashSet) vectorMapJoinHashTable;
+ }
+
+ @Override
+ public void processBatch(VectorizedRowBatch batch) throws HiveException {
+
+ try {
+
+ // Do the per-batch setup for an anti join.
+
+ // (Currently none)
+ // antiPerBatchSetup(batch);
+
+ // For anti joins, we may apply the filter(s) now.
+ for(VectorExpression ve : bigTableFilterExpressions) {
+ ve.evaluate(batch);
+ }
+
+ final int inputLogicalSize = batch.size;
+ if (inputLogicalSize == 0) {
+ return;
+ }
+
+ // Perform any key expressions. Results will go into scratch columns.
+ if (bigTableKeyExpressions != null) {
+ for (VectorExpression ve : bigTableKeyExpressions) {
+ ve.evaluate(batch);
+ }
+ }
+
+ /*
+ * Single-Column String specific declarations.
+ */
+
+ // The one join column for this specialized class.
+ BytesColumnVector joinColVector = (BytesColumnVector) batch.cols[singleJoinColumn];
+ byte[][] vector = joinColVector.vector;
+ int[] start = joinColVector.start;
+ int[] length = joinColVector.length;
+
+ /*
+ * Single-Column Long check for repeating.
+ */
+
+ // Check single column for repeating.
+ boolean allKeyInputColumnsRepeating = joinColVector.isRepeating;
+
+ if (allKeyInputColumnsRepeating) {
+
+ /*
+ * Repeating.
+ */
+
+ // All key input columns are repeating. Generate key once. Lookup once.
+ // Since the key is repeated, we must use entry 0 regardless of selectedInUse.
+
+ /*
+ * Single-Column String specific repeated lookup.
+ */
+
+ JoinUtil.JoinResult joinResult;
+ if (!joinColVector.noNulls && joinColVector.isNull[0]) {
+ joinResult = JoinUtil.JoinResult.MATCH;
+ } else {
+ byte[] keyBytes = vector[0];
+ int keyStart = start[0];
+ int keyLength = length[0];
+ joinResult = hashSet.contains(keyBytes, keyStart, keyLength, hashSetResults[0]);
+ joinResult = inverseResultForAntiJoin(joinResult);
+ }
+
+ /*
+ * Common repeated join result processing.
+ */
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(CLASS_NAME + " batch #" + batchCounter + " repeated joinResult " + joinResult.name());
+ }
+ finishAntiRepeated(batch, joinResult, hashSetResults[0]);
+ } else {
+
+ /*
+ * NOT Repeating.
+ */
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(CLASS_NAME + " batch #" + batchCounter + " non-repeated");
+ }
+
+ // We remember any matching rows in matchs / matchSize. At the end of the loop,
+ // selected / batch.size will represent both matching and non-matching rows for outer join.
+ // Only deferred rows will have been removed from selected.
+ int selected[] = batch.selected;
+ boolean selectedInUse = batch.selectedInUse;
+
+ int hashSetResultCount = 0;
+ int allMatchCount = 0;
+ int spillCount = 0;
+
+ /*
+ * Single-Column String specific variables.
+ */
+
+ int saveKeyBatchIndex = -1;
+
+ // We optimize performance by only looking up the first key in a series of equal keys.
+ boolean haveSaveKey = false;
+ JoinUtil.JoinResult saveJoinResult = JoinUtil.JoinResult.NOMATCH;
+
+ // Logical loop over the rows in the batch since the batch may have selected in use.
+ for (int logical = 0; logical < inputLogicalSize; logical++) {
+ int batchIndex = (selectedInUse ? selected[logical] : logical);
+
+ /*
+ * Single-Column String get key.
+ */
+
+ // Implicit -- use batchIndex.
+ boolean isNull = !joinColVector.noNulls && joinColVector.isNull[batchIndex];
+
+ /*
+ * Equal key series checking.
+ */
+
+ if (isNull || !haveSaveKey ||
+ !StringExpr.equal(vector[saveKeyBatchIndex], start[saveKeyBatchIndex], length[saveKeyBatchIndex],
+ vector[batchIndex], start[batchIndex], length[batchIndex])) {
+
+ // New key.
+
+ if (haveSaveKey) {
+ // Move on with our counts.
+ switch (saveJoinResult) {
+ case MATCH:
+ // We have extracted the existence from the hash set result, so we don't keep it.
+ break;
+ case SPILL:
+ // We keep the hash set result for its spill information.
+ hashSetResultCount++;
+ break;
+ case NOMATCH:
+ break;
+ }
+ }
+
+ if (isNull) {
+ saveJoinResult = JoinUtil.JoinResult.NOMATCH;
+ haveSaveKey = false;
+ } else {
+ // Regardless of our matching result, we keep that information to make multiple use
+ // of it for a possible series of equal keys.
+ haveSaveKey = true;
+
+ /*
+ * Single-Column String specific save key and lookup.
+ */
+
+ saveKeyBatchIndex = batchIndex;
+
+ /*
+ * Single-Column String specific lookup key.
+ */
+
+ byte[] keyBytes = vector[batchIndex];
+ int keyStart = start[batchIndex];
+ int keyLength = length[batchIndex];
+ saveJoinResult = hashSet.contains(keyBytes, keyStart, keyLength, hashSetResults[hashSetResultCount]);
+ saveJoinResult = inverseResultForAntiJoin(saveJoinResult);
+ }
+
+ /*
+ * Common anti join result processing.
+ */
+
+ switch (saveJoinResult) {
+ case MATCH:
+ allMatchs[allMatchCount++] = batchIndex;
+ // VectorizedBatchUtil.debugDisplayOneRow(batch, batchIndex, CLASS_NAME + " MATCH isSingleValue " + equalKeySeriesIsSingleValue[equalKeySeriesCount] + " currentKey " + currentKey);
+ break;
+
+ case SPILL:
+ spills[spillCount] = batchIndex;
+ spillHashMapResultIndices[spillCount] = hashSetResultCount;
+ spillCount++;
+ break;
+
+ case NOMATCH:
+ // VectorizedBatchUtil.debugDisplayOneRow(batch, batchIndex, CLASS_NAME + " NOMATCH" + " currentKey " + currentKey);
+ break;
+ }
+ } else {
+ // Series of equal keys.
+
+ switch (saveJoinResult) {
+ case MATCH:
+ allMatchs[allMatchCount++] = batchIndex;
+ // VectorizedBatchUtil.debugDisplayOneRow(batch, batchIndex, CLASS_NAME + " MATCH duplicate");
+ break;
+
+ case SPILL:
+ spills[spillCount] = batchIndex;
+ spillHashMapResultIndices[spillCount] = hashSetResultCount;
+ spillCount++;
+ break;
+
+ case NOMATCH:
+ // VectorizedBatchUtil.debugDisplayOneRow(batch, batchIndex, CLASS_NAME + " NOMATCH duplicate");
+ break;
+ }
+ }
+ }
+
+ if (haveSaveKey) {
+ // Update our counts for the last key.
+ switch (saveJoinResult) {
+ case MATCH:
+ // We have extracted the existence from the hash set result, so we don't keep it.
+ break;
+ case SPILL:
+ // We keep the hash set result for its spill information.
+ hashSetResultCount++;
+ break;
+ case NOMATCH:
+ break;
+ }
+ }
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(CLASS_NAME +
+ " allMatchs " + intArrayToRangesString(allMatchs, allMatchCount) +
+ " spills " + intArrayToRangesString(spills, spillCount) +
+ " spillHashMapResultIndices " + intArrayToRangesString(spillHashMapResultIndices, spillCount) +
+ " hashMapResults " + Arrays.toString(Arrays.copyOfRange(hashSetResults, 0, hashSetResultCount)));
+ }
+
+ finishAnti(batch, allMatchCount, spillCount, hashSetResults);
+ }
+
+ if (batch.size > 0) {
+ // Forward any remaining selected rows.
+ forwardBigTableBatch(batch);
+ }
+ } catch (Exception e) {
+ throw new HiveException(e);
+ }
+ }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java
index 2d3daeb..f43ba92 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConstantPropagateProcFactory.java
@@ -1418,7 +1418,7 @@ public final class ConstantPropagateProcFactory {
private boolean skipFolding(JoinDesc joinDesc) {
for (JoinCondDesc cond : joinDesc.getConds()) {
if (cond.getType() == JoinDesc.INNER_JOIN || cond.getType() == JoinDesc.UNIQUE_JOIN
- || cond.getType() == JoinDesc.LEFT_SEMI_JOIN) {
+ || cond.getType() == JoinDesc.LEFT_SEMI_JOIN || cond.getType() == JoinDesc.ANTI_JOIN) {
continue;
}
return true;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java
index 655b5f1..90b23e9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java
@@ -954,6 +954,7 @@ public class ConvertJoinMapJoin implements SemanticNodeProcessor {
switch (joinCondDesc.getType()) {
case JoinDesc.INNER_JOIN:
case JoinDesc.LEFT_SEMI_JOIN:
+ case JoinDesc.ANTI_JOIN:
case JoinDesc.UNIQUE_JOIN:
hasOuter = false;
break;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java
index 4f1c9b2..c4cb839 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/MapJoinProcessor.java
@@ -752,7 +752,8 @@ public class MapJoinProcessor extends Transform {
seenPostitions.add(condn.getRight());
if (joinType == JoinDesc.LEFT_OUTER_JOIN
- || joinType == JoinDesc.LEFT_SEMI_JOIN) {
+ || joinType == JoinDesc.LEFT_SEMI_JOIN
+ || joinType == JoinDesc.ANTI_JOIN) {
seenOuterJoin = true;
if(bigTableCandidates.size() == 0) {
bigTableCandidates.add(condn.getLeft());
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveCalciteUtil.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveCalciteUtil.java
index 88aaedd..d235e3b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveCalciteUtil.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveCalciteUtil.java
@@ -1233,4 +1233,27 @@ public class HiveCalciteUtil {
}
}
+ /**
+ * Checks if any of the expression given as list expressions are from right side of the join.
+ * This is used during anti join conversion.
+ *
+ * @param joinRel Join node whose right side has to be searched.
+ * @param expressions The list of expression to search.
+ * @return true if any of the expressions is from right side of join.
+ */
+ public static boolean hasAnyExpressionFromRightSide(RelNode joinRel, List<RexNode> expressions) {
+ List<RelDataTypeField> joinFields = joinRel.getRowType().getFieldList();
+ int nTotalFields = joinFields.size();
+ List<RelDataTypeField> leftFields = (joinRel.getInputs().get(0)).getRowType().getFieldList();
+ int nFieldsLeft = leftFields.size();
+ ImmutableBitSet rightBitmap = ImmutableBitSet.range(nFieldsLeft, nTotalFields);
+
+ for (RexNode node : expressions) {
+ ImmutableBitSet inputBits = RelOptUtil.InputFinder.bits(node);
+ if (rightBitmap.contains(inputBits)) {
+ return true;
+ }
+ }
+ return false;
+ }
}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelFactories.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelFactories.java
index 6be826e..312de0f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelFactories.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelFactories.java
@@ -46,6 +46,7 @@ import org.apache.calcite.sql.validate.SqlValidatorUtil;
import org.apache.calcite.tools.RelBuilderFactory;
import org.apache.calcite.util.ImmutableBitSet;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAggregate;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAntiJoin;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveExcept;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveFilter;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveIntersect;
@@ -170,6 +171,10 @@ public class HiveRelFactories {
final RelOptCluster cluster = left.getCluster();
return HiveSemiJoin.getSemiJoin(cluster, left.getTraitSet(), left, right, condition);
}
+ if (joinType == JoinRelType.ANTI) {
+ final RelOptCluster cluster = left.getCluster();
+ return HiveAntiJoin.getAntiJoin(cluster, left.getTraitSet(), left, right, condition);
+ }
return HiveJoin.getJoin(left.getCluster(), left, right, condition, joinType);
}
}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelOptMaterializationValidator.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelOptMaterializationValidator.java
index 1aa1731..90ece35 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelOptMaterializationValidator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelOptMaterializationValidator.java
@@ -40,6 +40,7 @@ import org.apache.calcite.util.Util;
import org.apache.hadoop.hive.metastore.TableType;
import org.apache.hadoop.hive.ql.metadata.Table;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAggregate;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAntiJoin;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveExcept;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveFilter;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveIntersect;
@@ -138,6 +139,8 @@ public class HiveRelOptMaterializationValidator extends HiveRelShuttleImpl {
return visit((HiveSemiJoin) node);
} else if (node instanceof HiveExcept) {
return visit((HiveExcept) node);
+ } else if (node instanceof HiveAntiJoin) {
+ return visit((HiveAntiJoin) node);
} else if (node instanceof HiveIntersect) {
return visit((HiveIntersect) node);
}
@@ -253,6 +256,13 @@ public class HiveRelOptMaterializationValidator extends HiveRelShuttleImpl {
return visitChildren(semiJoin);
}
+ private RelNode visit(HiveAntiJoin antiJoin) {
+ setAutomaticRewritingInvalidReason("Statement has unsupported join type: anti join.");
+ checkExpr(antiJoin.getCondition());
+ checkExpr(antiJoin.getJoinFilter());
+ return visitChildren(antiJoin);
+ }
+
// Note: Not currently part of the HiveRelNode interface
private RelNode visit(HiveExcept except) {
setAutomaticRewritingInvalidReason("Statement has unsupported operator: except.");
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelOptUtil.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelOptUtil.java
index 1e2c1e2..9d45c6c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelOptUtil.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRelOptUtil.java
@@ -747,6 +747,12 @@ public class HiveRelOptUtil extends RelOptUtil {
final RelNode nonFkInput = leftInputPotentialFK ? join.getRight() : join.getLeft();
final RewritablePKFKJoinInfo nonRewritable = RewritablePKFKJoinInfo.of(false, null);
+ // TODO : Need to handle Anti join.
+ // https://issues.apache.org/jira/browse/HIVE-23906
+ if (joinType == JoinRelType.ANTI) {
+ return nonRewritable;
+ }
+
if (joinType != JoinRelType.INNER && !join.isSemiJoin()) {
// If it is not an inner, we transform it as the metadata
// providers for expressions do not pull information through
@@ -854,6 +860,7 @@ public class HiveRelOptUtil extends RelOptUtil {
if (ecT.getEquivalenceClassesMap().containsKey(uniqueKeyColumnRef) &&
ecT.getEquivalenceClassesMap().get(uniqueKeyColumnRef).contains(foreignKeyColumnRef)) {
if (foreignKeyColumnType.isNullable()) {
+ //TODO : Handle Anti Join. https://issues.apache.org/jira/browse/HIVE-23906
if (joinType == JoinRelType.INNER || join.isSemiJoin()) {
// If it is nullable and it is an INNER, we just need a IS NOT NULL filter
RexNode originalCondOp = refToRex.get(foreignKeyColumnRef);
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveSubQRemoveRelBuilder.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveSubQRemoveRelBuilder.java
index a1d617b..f52ab60 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveSubQRemoveRelBuilder.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveSubQRemoveRelBuilder.java
@@ -1112,7 +1112,8 @@ public class HiveSubQRemoveRelBuilder {
}
public HiveSubQRemoveRelBuilder join(JoinRelType joinType, RexNode condition,
- Set<CorrelationId> variablesSet, boolean createSemiJoin) {
+ Set<CorrelationId> variablesSet, JoinRelType semiJoinType) {
+ assert semiJoinType == JoinRelType.SEMI || semiJoinType == JoinRelType.ANTI || semiJoinType == null;
Frame right = stack.pop();
final Frame left = stack.pop();
final RelNode join;
@@ -1138,12 +1139,12 @@ public class HiveSubQRemoveRelBuilder {
default:
postCondition = condition;
}
- if(createSemiJoin) {
+ if(semiJoinType != null) {
join = correlateFactory.createCorrelate(left.rel, right.rel, id,
- requiredColumns, JoinRelType.SEMI);
+ requiredColumns, semiJoinType);
} else {
join = correlateFactory.createCorrelate(left.rel, right.rel, id,
- requiredColumns, joinType);
+ requiredColumns, joinType);
}
} else {
join = joinFactory.createJoin(left.rel, right.rel, condition,
@@ -1161,7 +1162,7 @@ public class HiveSubQRemoveRelBuilder {
* variables. */
public HiveSubQRemoveRelBuilder join(JoinRelType joinType, RexNode condition,
Set<CorrelationId> variablesSet) {
- return join(joinType, condition, variablesSet, false);
+ return join(joinType, condition, variablesSet, null);
}
/** Creates a {@link org.apache.calcite.rel.core.Join} using USING syntax.
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveAntiJoin.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveAntiJoin.java
new file mode 100644
index 0000000..176cb70
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveAntiJoin.java
@@ -0,0 +1,97 @@
+/*
+ * 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.reloperators;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Sets;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexNode;
+import org.apache.hadoop.hive.ql.optimizer.calcite.CalciteSemanticException;
+import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelOptUtil;
+import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveRulesRegistry;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class HiveAntiJoin extends Join implements HiveRelNode {
+
+ // The joinFilter holds the residual filter which is used during post processing.
+ // These are the join conditions that are not part of the join key.
+ private final RexNode joinFilter;
+
+ public static HiveAntiJoin getAntiJoin(
+ RelOptCluster cluster,
+ RelTraitSet traitSet,
+ RelNode left,
+ RelNode right,
+ RexNode condition) {
+ try {
+ return new HiveAntiJoin(cluster, traitSet, left, right, condition);
+ } catch (CalciteSemanticException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ protected HiveAntiJoin(RelOptCluster cluster,
+ RelTraitSet traitSet,
+ RelNode left,
+ RelNode right,
+ RexNode condition) throws CalciteSemanticException {
+ super(cluster, traitSet, left, right, condition, JoinRelType.ANTI, Sets.newHashSet());
+ final List<RelDataTypeField> systemFieldList = ImmutableList.of();
+ List<List<RexNode>> joinKeyExprs = new ArrayList<List<RexNode>>();
+ List<Integer> filterNulls = new ArrayList<Integer>();
+ for (int i=0; i<this.getInputs().size(); i++) {
+ joinKeyExprs.add(new ArrayList<>());
+ }
+ this.joinFilter = HiveRelOptUtil.splitHiveJoinCondition(systemFieldList, this.getInputs(),
+ this.getCondition(), joinKeyExprs, filterNulls, null);
+ }
+
+ public RexNode getJoinFilter() {
+ return joinFilter;
+ }
+
+ @Override
+ public HiveAntiJoin copy(RelTraitSet traitSet, RexNode condition,
+ RelNode left, RelNode right, JoinRelType joinType, boolean semiJoinDone) {
+ try {
+ HiveAntiJoin antiJoin = new HiveAntiJoin(getCluster(), traitSet, left, right, condition);
+ // If available, copy state to registry for optimization rules
+ HiveRulesRegistry registry = antiJoin.getCluster().getPlanner().getContext().unwrap(HiveRulesRegistry.class);
+ if (registry != null) {
+ registry.copyPushedPredicates(this, antiJoin);
+ }
+ return antiJoin;
+ } catch (CalciteSemanticException e) {
+ // Semantic error not possible. Must be a bug. Convert to
+ // internal error.
+ throw new AssertionError(e);
+ }
+ }
+
+ @Override
+ public void implement(Implementor implementor) {
+ }
+
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveAntiSemiJoinRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveAntiSemiJoinRule.java
new file mode 100644
index 0000000..14a64c3
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveAntiSemiJoinRule.java
@@ -0,0 +1,160 @@
+/*
+ * 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 org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.hadoop.hive.ql.optimizer.calcite.HiveCalciteUtil;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAntiJoin;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Planner rule that converts a join plus filter to anti join.
+ */
+public class HiveAntiSemiJoinRule extends RelOptRule {
+ protected static final Logger LOG = LoggerFactory.getLogger(HiveAntiSemiJoinRule.class);
+ public static final HiveAntiSemiJoinRule INSTANCE = new HiveAntiSemiJoinRule();
+
+ // HiveProject(fld=[$0])
+ // HiveFilter(condition=[IS NULL($1)])
+ // HiveJoin(condition=[=($0, $1)], joinType=[left], algorithm=[none], cost=[not available])
+ //
+ // TO
+ //
+ // HiveProject(fld_tbl=[$0])
+ // HiveAntiJoin(condition=[=($0, $1)], joinType=[anti])
+ //
+ public HiveAntiSemiJoinRule() {
+ super(operand(Project.class, operand(Filter.class, operand(Join.class, RelOptRule.any()))),
+ "HiveJoinWithFilterToAntiJoinRule:filter");
+ }
+
+ // is null filter over a left join.
+ public void onMatch(final RelOptRuleCall call) {
+ final Project project = call.rel(0);
+ final Filter filter = call.rel(1);
+ final Join join = call.rel(2);
+ perform(call, project, filter, join);
+ }
+
+ protected void perform(RelOptRuleCall call, Project project, Filter filter, Join join) {
+ LOG.debug("Start Matching HiveAntiJoinRule");
+
+ //TODO : Need to support this scenario.
+ //https://issues.apache.org/jira/browse/HIVE-23991
+ if (join.getCondition().isAlwaysTrue()) {
+ return;
+ }
+
+ //We support conversion from left outer join only.
+ if (join.getJoinType() != JoinRelType.LEFT) {
+ return;
+ }
+
+ assert (filter != null);
+
+ List<RexNode> filterList = getResidualFilterNodes(filter, join);
+ if (filterList == null) {
+ return;
+ }
+
+ // If any projection is there from right side, then we can not convert to anti join.
+ boolean hasProjection = HiveCalciteUtil.hasAnyExpressionFromRightSide(join, project.getProjects());
+ if (hasProjection) {
+ return;
+ }
+
+ LOG.debug("Matched HiveAntiJoinRule");
+
+ // Build anti join with same left, right child and condition as original left outer join.
+ Join anti = HiveAntiJoin.getAntiJoin(join.getLeft().getCluster(), join.getLeft().getTraitSet(),
+ join.getLeft(), join.getRight(), join.getCondition());
+ RelNode newProject;
+ if (filterList.isEmpty()) {
+ newProject = project.copy(project.getTraitSet(), anti, project.getProjects(), project.getRowType());
+ } else {
+ // Collate the filter condition using AND as the filter was decomposed based
+ // on AND condition (RelOptUtil.conjunctions).
+ RexNode condition = filterList.size() == 1 ? filterList.get(0) :
+ join.getCluster().getRexBuilder().makeCall(SqlStdOperatorTable.AND, filterList);
+ Filter newFilter = filter.copy(filter.getTraitSet(), anti, condition);
+ newProject = project.copy(project.getTraitSet(), newFilter, project.getProjects(), project.getRowType());
+ }
+ call.transformTo(newProject);
+ }
+
+ /**
+ * Extracts the non-null filter conditions from given filter node.
+ *
+ * @param filter The filter condition to be checked.
+ * @param join Join node whose right side has to be searched.
+ * @return null : Anti join condition is not matched for filter.
+ * Empty list : No residual filter conditions present.
+ * Valid list containing the filter to be applied after join.
+ */
+ private List<RexNode> getResidualFilterNodes(Filter filter, Join join) {
+ // 1. If null filter is not present from right side then we can not convert to anti join.
+ // 2. If any non-null filter is present from right side, we can not convert it to anti join.
+ // 3. Keep other filters which needs to be executed after join.
+ // 4. The filter conditions are decomposed on AND conditions only.
+ //TODO If some conditions like (fld1 is null or fld2 is null) present, it will not be considered for conversion.
+ //https://issues.apache.org/jira/browse/HIVE-23992
+ List<RexNode> aboveFilters = RelOptUtil.conjunctions(filter.getCondition());
+ boolean hasNullFilterOnRightSide = false;
+ List<RexNode> filterList = new ArrayList<>();
+ for (RexNode filterNode : aboveFilters) {
+ if (filterNode.getKind() == SqlKind.IS_NULL) {
+ // Null filter from right side table can be removed and its a pre-condition for anti join conversion.
+ if (HiveCalciteUtil.hasAnyExpressionFromRightSide(join, Collections.singletonList(filterNode))) {
+ hasNullFilterOnRightSide = true;
+ } else {
+ filterList.add(filterNode);
+ }
+ } else {
+ if (HiveCalciteUtil.hasAnyExpressionFromRightSide(join, Collections.singletonList(filterNode))) {
+ // If some non null condition is present from right side, we can not convert the join to anti join as
+ // anti join does not project the fields from right side.
+ return null;
+ } else {
+ filterList.add(filterNode);
+ }
+ }
+ }
+
+ if (!hasNullFilterOnRightSide) {
+ return null;
+ }
+ return filterList;
+ }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinAddNotNullRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinAddNotNullRule.java
index b2ff255..3090294 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinAddNotNullRule.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinAddNotNullRule.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.HiveCalciteUtil;
import org.apache.hadoop.hive.ql.optimizer.calcite.HiveCalciteUtil.JoinLeafPredicateInfo;
import org.apache.hadoop.hive.ql.optimizer.calcite.HiveCalciteUtil.JoinPredicateInfo;
import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelFactories;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAntiJoin;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveJoin;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSemiJoin;
@@ -56,6 +57,9 @@ public final class HiveJoinAddNotNullRule extends RelOptRule {
public static final HiveJoinAddNotNullRule INSTANCE_SEMIJOIN =
new HiveJoinAddNotNullRule(HiveSemiJoin.class, HiveRelFactories.HIVE_FILTER_FACTORY);
+ public static final HiveJoinAddNotNullRule INSTANCE_ANTIJOIN =
+ new HiveJoinAddNotNullRule(HiveAntiJoin.class, HiveRelFactories.HIVE_FILTER_FACTORY);
+
private final FilterFactory filterFactory;
//~ Constructors -----------------------------------------------------------
@@ -74,7 +78,15 @@ public final class HiveJoinAddNotNullRule extends RelOptRule {
@Override
public void onMatch(RelOptRuleCall call) {
Join join = call.rel(0);
- if (join.getJoinType() == JoinRelType.FULL || join.getCondition().isAlwaysTrue()) {
+
+ // For anti join case add the not null on right side even if the condition is always true.
+ // This is done because during execution, anti join expect the right side to be empty and
+ // if we dont put null check on right, for null only right side table and condition always
+ // true, execution will produce 0 records as the post processing to filter out null value
+ // is not done for always true conditions during execution.
+ // eg select * from left_tbl where (select 1 from all_null_right limit 1) is null
+ if (join.getJoinType() == JoinRelType.FULL ||
+ (join.getJoinType() != JoinRelType.ANTI && join.getCondition().isAlwaysTrue())) {
return;
}
@@ -92,7 +104,7 @@ public final class HiveJoinAddNotNullRule extends RelOptRule {
Set<String> rightPushedPredicates = Sets.newHashSet(registry.getPushedPredicates(join, 1));
boolean genPredOnLeft = join.getJoinType() == JoinRelType.RIGHT || join.getJoinType() == JoinRelType.INNER || join.isSemiJoin();
- boolean genPredOnRight = join.getJoinType() == JoinRelType.LEFT || join.getJoinType() == JoinRelType.INNER || join.isSemiJoin();
+ boolean genPredOnRight = join.getJoinType() == JoinRelType.LEFT || join.getJoinType() == JoinRelType.INNER || join.isSemiJoin()|| join.getJoinType() == JoinRelType.ANTI;
RexNode newLeftPredicate = getNewPredicate(join, registry, joinPredInfo, leftPushedPredicates, genPredOnLeft, 0);
RexNode newRightPredicate = getNewPredicate(join, registry, joinPredInfo, rightPushedPredicates, genPredOnRight, 1);
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinConstraintsRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinConstraintsRule.java
index 8acecfc..0c533e3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinConstraintsRule.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinConstraintsRule.java
@@ -85,6 +85,11 @@ public class HiveJoinConstraintsRule extends RelOptRule {
final RelNode rightInput = join.getRight();
final RexNode cond = join.getCondition();
+ //TODO:https://issues.apache.org/jira/browse/HIVE-23920
+ if (joinType == JoinRelType.ANTI) {
+ return;
+ }
+
// 1) If it is an inner, check whether project only uses columns from one side.
// That side will need to be the FK side.
// If it is a left outer, left will be the FK side.
@@ -100,7 +105,8 @@ public class HiveJoinConstraintsRule extends RelOptRule {
// These boolean values represent corresponding left, right input which is potential FK
boolean leftInputPotentialFK = topRefs.intersects(leftBits);
boolean rightInputPotentialFK = topRefs.intersects(rightBits);
- if (leftInputPotentialFK && rightInputPotentialFK && (joinType == JoinRelType.INNER || joinType == JoinRelType.SEMI)) {
+ if (leftInputPotentialFK && rightInputPotentialFK &&
+ (joinType == JoinRelType.INNER || joinType == JoinRelType.SEMI)) {
// Both inputs are referenced. Before making a decision, try to swap
// references in join condition if it is an inner join, i.e. if a join
// condition column is referenced above the join, then we can just
@@ -183,6 +189,7 @@ public class HiveJoinConstraintsRule extends RelOptRule {
switch (joinType) {
case SEMI:
case INNER:
+ //case ANTI: //TODO:https://issues.apache.org/jira/browse/HIVE-23920
if (leftInputPotentialFK && rightInputPotentialFK) {
// Bails out as it references columns from both sides (or no columns)
// and there is nothing to transform
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinProjectTransposeRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinProjectTransposeRule.java
index 17984ac..c9742aa 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinProjectTransposeRule.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinProjectTransposeRule.java
@@ -24,6 +24,8 @@ import static org.apache.calcite.plan.RelOptRule.some;
import org.apache.calcite.plan.RelOptRuleCall;
import org.apache.calcite.plan.RelOptRuleOperand;
import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinRelType;
import org.apache.calcite.rel.rules.JoinProjectTransposeRule;
import org.apache.calcite.tools.RelBuilderFactory;
import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelFactories;
@@ -133,6 +135,12 @@ public final class HiveJoinProjectTransposeRule {
public void onMatch(RelOptRuleCall call) {
//TODO: this can be removed once CALCITE-3824 is released
+ Join joinRel = call.rel(0);
+
+ //TODO:https://issues.apache.org/jira/browse/HIVE-23921
+ if (joinRel.getJoinType() == JoinRelType.ANTI) {
+ return;
+ }
HiveProject proj;
if (hasLeftChild(call)) {
proj = call.rel(1);
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinPushTransitivePredicatesRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinPushTransitivePredicatesRule.java
index 3569265..3a2bf82 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinPushTransitivePredicatesRule.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinPushTransitivePredicatesRule.java
@@ -42,6 +42,7 @@ import org.apache.calcite.util.Util;
import org.apache.hadoop.hive.ql.exec.Description;
import org.apache.hadoop.hive.ql.optimizer.calcite.HiveCalciteUtil;
import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelFactories;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAntiJoin;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveJoin;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSemiJoin;
import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPNotNull;
@@ -70,6 +71,9 @@ public class HiveJoinPushTransitivePredicatesRule extends RelOptRule {
public static final HiveJoinPushTransitivePredicatesRule INSTANCE_SEMIJOIN =
new HiveJoinPushTransitivePredicatesRule(HiveSemiJoin.class, HiveRelFactories.HIVE_FILTER_FACTORY);
+ public static final HiveJoinPushTransitivePredicatesRule INSTANCE_ANTIJOIN =
+ new HiveJoinPushTransitivePredicatesRule(HiveAntiJoin.class, HiveRelFactories.HIVE_FILTER_FACTORY);
+
private final FilterFactory filterFactory;
public HiveJoinPushTransitivePredicatesRule(Class<? extends Join> clazz,
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveProjectJoinTransposeRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveProjectJoinTransposeRule.java
index 545255c..3fd76f5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveProjectJoinTransposeRule.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveProjectJoinTransposeRule.java
@@ -23,6 +23,7 @@ import org.apache.calcite.plan.RelOptRule;
import org.apache.calcite.plan.RelOptRuleCall;
import org.apache.calcite.rel.RelNode;
import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinRelType;
import org.apache.calcite.rel.core.Project;
import org.apache.calcite.rel.rules.PushProjector;
import org.apache.calcite.rel.type.RelDataTypeField;
@@ -76,7 +77,7 @@ public class HiveProjectJoinTransposeRule extends RelOptRule {
Project origProj = call.rel(0);
final Join join = call.rel(1);
- if (join.isSemiJoin()) {
+ if (join.getJoinType() == JoinRelType.SEMI || join.getJoinType() == JoinRelType.ANTI) {
return; // TODO: support SemiJoin
}
// locate all fields referenced in the projection and join condition;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRelDecorrelator.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRelDecorrelator.java
index ab56ce8..8f61da7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRelDecorrelator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRelDecorrelator.java
@@ -98,6 +98,7 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelFactories;
import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelOptUtil;
import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelShuttleImpl;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAggregate;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAntiJoin;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveFilter;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveIntersect;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveJoin;
@@ -1239,12 +1240,16 @@ public final class HiveRelDecorrelator implements ReflectiveVisitor {
valueGenerator = false;
}
- if(oldInput instanceof LogicalCorrelate
- && ((LogicalCorrelate) oldInput).getJoinType() == JoinRelType.SEMI
- && !cm.mapRefRelToCorRef.containsKey(rel)) {
+ boolean isSemiJoin = false;
+ if (oldInput instanceof LogicalCorrelate) {
+ isSemiJoin = ((LogicalCorrelate) oldInput).getJoinType() == JoinRelType.SEMI ||
+ ((LogicalCorrelate) oldInput).getJoinType() == JoinRelType.ANTI;
+ }
+
+ if(isSemiJoin && !cm.mapRefRelToCorRef.containsKey(rel)) {
// this conditions need to be pushed into semi-join since this condition
// corresponds to IN
- HiveSemiJoin join = ((HiveSemiJoin)frame.r);
+ Join join = ((Join)frame.r);
final List<RexNode> conditions = new ArrayList<>();
RexNode joinCond = join.getCondition();
conditions.add(joinCond);
@@ -1252,8 +1257,14 @@ public final class HiveRelDecorrelator implements ReflectiveVisitor {
final RexNode condition =
RexUtil.composeConjunction(rexBuilder, conditions, false);
- RelNode newRel = HiveSemiJoin.getSemiJoin(frame.r.getCluster(), frame.r.getTraitSet(),
- join.getLeft(), join.getRight(), condition);
+ RelNode newRel;
+ if (((LogicalCorrelate) oldInput).getJoinType() == JoinRelType.SEMI) {
+ newRel = HiveSemiJoin.getSemiJoin(frame.r.getCluster(), frame.r.getTraitSet(),
+ join.getLeft(), join.getRight(), condition);
+ } else {
+ newRel = HiveAntiJoin.getAntiJoin(frame.r.getCluster(), frame.r.getTraitSet(),
+ join.getLeft(), join.getRight(), condition);
+ }
return register(rel, newRel, frame.oldToNewOutputs, frame.corDefOutputs);
}
@@ -1311,9 +1322,13 @@ public final class HiveRelDecorrelator implements ReflectiveVisitor {
valueGenerator = false;
}
- if(oldInput instanceof LogicalCorrelate
- && ((LogicalCorrelate) oldInput).getJoinType() == JoinRelType.SEMI
- && !cm.mapRefRelToCorRef.containsKey(rel)) {
+ boolean isSemiJoin = false;
+ if (oldInput instanceof LogicalCorrelate) {
+ isSemiJoin = ((LogicalCorrelate) oldInput).getJoinType() == JoinRelType.SEMI ||
+ ((LogicalCorrelate) oldInput).getJoinType() == JoinRelType.ANTI;
+ }
+
+ if(isSemiJoin && !cm.mapRefRelToCorRef.containsKey(rel)) {
// this conditions need to be pushed into semi-join since this condition
// corresponds to IN
HiveSemiJoin join = ((HiveSemiJoin)frame.r);
@@ -1323,8 +1338,15 @@ public final class HiveRelDecorrelator implements ReflectiveVisitor {
conditions.add(decorrelateExpr(rel.getCondition(), valueGenerator));
final RexNode condition =
RexUtil.composeConjunction(rexBuilder, conditions, false);
- RelNode newRel = HiveSemiJoin.getSemiJoin(frame.r.getCluster(), frame.r.getTraitSet(),
- join.getLeft(), join.getRight(), condition);
+
+ RelNode newRel;
+ if (((LogicalCorrelate) oldInput).getJoinType() == JoinRelType.SEMI) {
+ newRel = HiveSemiJoin.getSemiJoin(frame.r.getCluster(), frame.r.getTraitSet(),
+ join.getLeft(), join.getRight(), condition);
+ } else {
+ newRel = HiveAntiJoin.getAntiJoin(frame.r.getCluster(), frame.r.getTraitSet(),
+ join.getLeft(), join.getRight(), condition);
+ }
return register(rel, newRel, frame.oldToNewOutputs, frame.corDefOutputs);
}
@@ -1447,14 +1469,18 @@ public final class HiveRelDecorrelator implements ReflectiveVisitor {
RelNode newJoin = null;
// this indicates original query was either correlated EXISTS or IN
- if(rel.getJoinType() == JoinRelType.SEMI) {
+ if(rel.getJoinType() == JoinRelType.SEMI || rel.getJoinType() == JoinRelType.ANTI) {
final List<Integer> leftKeys = new ArrayList<Integer>();
final List<Integer> rightKeys = new ArrayList<Integer>();
RelNode[] inputRels = new RelNode[] {leftFrame.r, rightFrame.r};
- newJoin = HiveSemiJoin.getSemiJoin(rel.getCluster(),
- rel.getCluster().traitSetOf(HiveRelNode.CONVENTION), leftFrame.r, rightFrame.r, condition);
-
+ if (rel.getJoinType() == JoinRelType.ANTI) {
+ newJoin = HiveAntiJoin.getAntiJoin(rel.getCluster(),
+ rel.getCluster().traitSetOf(HiveRelNode.CONVENTION), leftFrame.r, rightFrame.r, condition);
+ } else {
+ newJoin = HiveSemiJoin.getSemiJoin(rel.getCluster(),
+ rel.getCluster().traitSetOf(HiveRelNode.CONVENTION), leftFrame.r, rightFrame.r, condition);
+ }
} else {
// Right input positions are shifted by newLeftFieldCount.
for (int i = 0; i < oldRightFieldCount; i++) {
@@ -1610,7 +1636,7 @@ public final class HiveRelDecorrelator implements ReflectiveVisitor {
if(oldInput == null) {
if(currentRel.getInputs().size() == 1 && currentRel.getInput(0) instanceof LogicalCorrelate) {
final Frame newFrame = map.get(currentRel.getInput(0));
- if(newFrame.r instanceof HiveSemiJoin) {
+ if(newFrame.r instanceof HiveSemiJoin || newFrame.r instanceof HiveAntiJoin) {
int oldFieldSize = currentRel.getInput(0).getRowType().getFieldCount();
int newOrd = newFrame.r.getRowType().getFieldCount() + oldOrdinalNo - oldFieldSize;
return new RexInputRef(newOrd, oldInputRef.getType());
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRemoveGBYSemiJoinRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRemoveGBYSemiJoinRule.java
index 82704a2..abad7db 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRemoveGBYSemiJoinRule.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveRemoveGBYSemiJoinRule.java
@@ -20,17 +20,17 @@ import org.apache.calcite.plan.RelOptRule;
import org.apache.calcite.plan.RelOptRuleCall;
import org.apache.calcite.rel.RelNode;
import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.Join;
import org.apache.calcite.rel.core.JoinInfo;
import org.apache.calcite.rel.core.JoinRelType;
import org.apache.calcite.tools.RelBuilder;
import org.apache.calcite.util.ImmutableBitSet;
import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelFactories;
-import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSemiJoin;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
- * Planner rule that removes a {@code Aggregate} from a HiveSemiJoin
+ * Planner rule that removes a {@code Aggregate} from a HiveSemiJoin/HiveAntiJoin
* right input.
*/
public class HiveRemoveGBYSemiJoinRule extends RelOptRule {
@@ -41,7 +41,7 @@ public class HiveRemoveGBYSemiJoinRule extends RelOptRule {
public HiveRemoveGBYSemiJoinRule() {
super(
- operand(HiveSemiJoin.class,
+ operand(Join.class,
some(
operand(RelNode.class, any()),
operand(Aggregate.class, any()))),
@@ -49,9 +49,11 @@ public class HiveRemoveGBYSemiJoinRule extends RelOptRule {
}
@Override public void onMatch(RelOptRuleCall call) {
- final HiveSemiJoin semijoin= call.rel(0);
+ final Join join = call.rel(0);
- assert semijoin.getJoinType() == JoinRelType.SEMI;
+ if (join.getJoinType() != JoinRelType.SEMI && join.getJoinType() != JoinRelType.ANTI) {
+ return;
+ }
final RelNode left = call.rel(1);
final Aggregate rightAggregate= call.rel(2);
@@ -69,7 +71,7 @@ public class HiveRemoveGBYSemiJoinRule extends RelOptRule {
if(!rightAggregate.getAggCallList().isEmpty()) {
return;
}
- final JoinInfo joinInfo = semijoin.analyzeCondition();
+ final JoinInfo joinInfo = join.analyzeCondition();
boolean shouldTransform = joinInfo.rightSet().equals(
ImmutableBitSet.range(rightAggregate.getGroupCount()));
@@ -77,9 +79,15 @@ public class HiveRemoveGBYSemiJoinRule extends RelOptRule {
final RelBuilder relBuilder = call.builder();
RelNode newRightInput = relBuilder.project(relBuilder.push(rightAggregate.getInput()).
fields(rightAggregate.getGroupSet().asList())).build();
- RelNode newSemiJoin = call.builder().push(left).push(newRightInput)
- .semiJoin(semijoin.getCondition()).build();
- call.transformTo(newSemiJoin);
+ RelNode newJoin;
+ if (join.getJoinType() == JoinRelType.SEMI) {
+ newJoin = call.builder().push(left).push(newRightInput)
+ .semiJoin(join.getCondition()).build();
+ } else {
+ newJoin = call.builder().push(left).push(newRightInput)
+ .antiJoin(join.getCondition()).build();
+ }
+ call.transformTo(newJoin);
}
}
}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSubQueryRemoveRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSubQueryRemoveRule.java
index d2d6f70..a508b50 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSubQueryRemoveRule.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSubQueryRemoveRule.java
@@ -63,6 +63,8 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAggregate;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveFilter;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveProject;
+import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_CONVERT_ANTI_JOIN;
+
/**
* NOTE: this rule is replicated from Calcite's SubqueryRemoveRule
* Transform that converts IN, EXISTS and scalar sub-queries into joins.
@@ -405,7 +407,7 @@ public class HiveSubQueryRemoveRule extends RelOptRule {
builder.push(e.rel);
}
}
-
+ boolean isCandidateForAntiJoin = false;
// First, the cross join
switch (logic) {
case TRUE_FALSE_UNKNOWN:
@@ -414,6 +416,13 @@ public class HiveSubQueryRemoveRule extends RelOptRule {
// null keys we do not need to generate count(*), count(c)
if (e.getKind() == SqlKind.EXISTS) {
logic = RelOptUtil.Logic.TRUE_FALSE;
+ if (conf.getBoolVar(HiveConf.ConfVars.HIVE_CONVERT_ANTI_JOIN)) {
+ //TODO : As of now anti join is first converted to left outer join
+ // and then converted to anti join.
+ //logic = RelOptUtil.Logic.FALSE;
+
+ isCandidateForAntiJoin = true;
+ }
break;
}
builder.aggregate(builder.groupKey(), builder.count(false, "c"),
@@ -435,8 +444,13 @@ public class HiveSubQueryRemoveRule extends RelOptRule {
String trueLiteral = "literalTrue";
switch (logic) {
case TRUE:
+ case FALSE:
if (fields.isEmpty()) {
- builder.project(builder.alias(builder.literal(true), trueLiteral));
+ if (logic == RelOptUtil.Logic.TRUE) {
+ builder.project(builder.alias(builder.literal(true), trueLiteral));
+ } else {
+ builder.project(builder.alias(builder.literal(false), "literalFalse"));
+ }
if (!variablesSet.isEmpty() && (e.getKind() == SqlKind.EXISTS
|| e.getKind() == SqlKind.IN)) {
// avoid adding group by for correlated IN/EXISTS queries
@@ -459,7 +473,12 @@ public class HiveSubQueryRemoveRule extends RelOptRule {
default:
fields.add(builder.alias(builder.literal(true), trueLiteral));
builder.project(fields);
- builder.distinct();
+ // If, not-exists is first converted to left outer join with null
+ // filter and then to anti join, then the distinct clause is added
+ // later during semi/anti join processing at genMapGroupByForSemijoin.
+ if (!isCandidateForAntiJoin || variablesSet.isEmpty()) {
+ builder.distinct();
+ }
}
builder.as("dt");
final List<RexNode> conditions = new ArrayList<>();
@@ -468,8 +487,11 @@ public class HiveSubQueryRemoveRule extends RelOptRule {
}
switch (logic) {
case TRUE:
- builder.join(JoinRelType.INNER, builder.and(conditions), variablesSet, true);
+ builder.join(JoinRelType.INNER, builder.and(conditions), variablesSet, JoinRelType.SEMI);
return builder.literal(true);
+ case FALSE:
+ builder.join(JoinRelType.LEFT, builder.and(conditions), variablesSet, JoinRelType.ANTI);
+ return builder.literal(false);
}
builder.join(JoinRelType.LEFT, builder.and(conditions), variablesSet);
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/EstimateUniqueKeys.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/EstimateUniqueKeys.java
index e36259c..0da4eaa 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/EstimateUniqueKeys.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/EstimateUniqueKeys.java
@@ -35,6 +35,7 @@ import org.apache.calcite.util.ImmutableBitSet;
import com.google.common.collect.ImmutableSet;
import org.apache.calcite.util.Util;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAggregate;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAntiJoin;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveFilter;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveJoin;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveProject;
@@ -269,6 +270,12 @@ public final class EstimateUniqueKeys {
return getUniqueKeys(rel.getLeft());
}
+ private static Set<ImmutableBitSet> getUniqueKeys(HiveAntiJoin rel) {
+ // only return the unique keys from the LHS since a anti join only
+ // returns the LHS
+ return getUniqueKeys(rel.getLeft());
+ }
+
private static Set<ImmutableBitSet> getUniqueKeys(HiveAggregate rel) {
// group by keys form a unique key
return ImmutableSet.of(rel.getGroupSet());
@@ -297,6 +304,8 @@ public final class EstimateUniqueKeys {
return getUniqueKeys((HiveJoin) rel);
} else if (rel instanceof HiveSemiJoin) {
return getUniqueKeys((HiveSemiJoin) rel);
+ } else if (rel instanceof HiveAntiJoin) {
+ return getUniqueKeys((HiveAntiJoin) rel);
} else if (rel instanceof HiveAggregate) {
return getUniqueKeys((HiveAggregate) rel);
} else if (rel instanceof SetOp) {
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdDistinctRowCount.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdDistinctRowCount.java
index af96354..70afebb 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdDistinctRowCount.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdDistinctRowCount.java
@@ -39,6 +39,7 @@ import org.apache.calcite.util.ImmutableBitSet;
import org.apache.calcite.util.NumberUtil;
import org.apache.hadoop.hive.ql.optimizer.calcite.HiveCalciteUtil;
import org.apache.hadoop.hive.ql.optimizer.calcite.cost.HiveCost;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAntiJoin;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveJoin;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSemiJoin;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveTableScan;
@@ -79,6 +80,13 @@ public class HiveRelMdDistinctRowCount extends RelMdDistinctRowCount {
return super.getDistinctRowCount(rel, mq, groupKey, predicate);
}
+ public Double getDistinctRowCount(HiveAntiJoin rel, RelMetadataQuery mq, ImmutableBitSet groupKey,
+ RexNode predicate) {
+ //TODO : Currently calcite does not support this.
+ // https://issues.apache.org/jira/browse/HIVE-23933
+ return super.getDistinctRowCount(rel, mq, groupKey, predicate);
+ }
+
public Double getDistinctRowCount(HiveJoin rel, RelMetadataQuery mq, ImmutableBitSet groupKey,
RexNode predicate) {
return getJoinDistinctRowCount(mq, rel, rel.getJoinType(),
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdPredicates.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdPredicates.java
index 0527e2b..66c6888 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdPredicates.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdPredicates.java
@@ -416,6 +416,7 @@ public class HiveRelMdPredicates implements MetadataHandler<BuiltInMetadata.Pred
case INNER:
case LEFT:
case SEMI:
+ case ANTI:
infer(leftPreds, allExprsDigests, inferredPredicates,
nonFieldsPredicates, includeEqualityInference,
joinType == JoinRelType.LEFT ? rightFieldsBitSet
@@ -454,7 +455,8 @@ public class HiveRelMdPredicates implements MetadataHandler<BuiltInMetadata.Pred
}
}
- if ((joinType == JoinRelType.INNER || joinType == JoinRelType.SEMI) && !nonFieldsPredicates.isEmpty()) {
+ if ((joinType == JoinRelType.INNER || joinType == JoinRelType.SEMI) &&
+ !nonFieldsPredicates.isEmpty()) {
// Predicates without field references can be pushed to both inputs
final Set<String> leftPredsSet = new HashSet<String>(
Lists.transform(leftPreds, HiveCalciteUtil.REX_STR_FN));
@@ -480,6 +482,7 @@ public class HiveRelMdPredicates implements MetadataHandler<BuiltInMetadata.Pred
return RelOptPredicateList.of(rexBuilder, Iterables.concat(leftPreds, leftInferredPredicates),
leftInferredPredicates, rightInferredPredicates);
case LEFT:
+ case ANTI:
return RelOptPredicateList.of(rexBuilder,
leftPreds, EMPTY_LIST, rightInferredPredicates);
case RIGHT:
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdRowCount.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdRowCount.java
index 82e4cc1..a5829ec 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdRowCount.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdRowCount.java
@@ -53,6 +53,7 @@ import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelOptUtil;
import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelOptUtil.PKFKJoinInfo;
import org.apache.hadoop.hive.ql.optimizer.calcite.RelOptHiveTable;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAntiJoin;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveJoin;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSemiJoin;
import org.apache.hadoop.hive.ql.plan.ColStatistics;
@@ -118,6 +119,15 @@ public class HiveRelMdRowCount extends RelMdRowCount {
}
public Double getRowCount(HiveSemiJoin rel, RelMetadataQuery mq) {
+ return getRowCountInt(rel, mq);
+ }
+
+ public Double getRowCount(HiveAntiJoin rel, RelMetadataQuery mq) {
+ return getRowCountInt(rel, mq);
+ }
+
+ private Double getRowCountInt(Join rel, RelMetadataQuery mq) {
+ assert rel.getJoinType() == JoinRelType.SEMI || rel.getJoinType() == JoinRelType.ANTI;
PKFKRelationInfo pkfk = analyzeJoinForPKFK(rel, mq);
if (pkfk != null) {
double selectivity = pkfk.pkInfo.selectivity * pkfk.ndvScalingFactor;
@@ -125,9 +135,20 @@ public class HiveRelMdRowCount extends RelMdRowCount {
if (LOG.isDebugEnabled()) {
LOG.debug("Identified Primary - Foreign Key relation: {} {}", RelOptUtil.toString(rel), pkfk);
}
- return pkfk.fkInfo.rowCount * selectivity;
+ if (rel.getJoinType() == JoinRelType.ANTI) {
+ return pkfk.fkInfo.rowCount * (1 - selectivity);
+ } else {
+ return pkfk.fkInfo.rowCount * selectivity;
+ }
+ }
+
+ //TODO : Need to handle anti join in calcite
+ // https://issues.apache.org/jira/browse/HIVE-23933
+ if (rel.getJoinType() == JoinRelType.ANTI) {
+ return mq.getRowCount(rel.getLeft()) - super.getRowCount(rel, mq);
+ } else {
+ return super.getRowCount(rel, mq);
}
- return super.getRowCount(rel, mq);
}
@Override
@@ -248,7 +269,7 @@ public class HiveRelMdRowCount extends RelMdRowCount {
// @todo: remove this. 8/28/14 hb
// for now adding because RelOptUtil.classifyFilters has an assertion about
// column counts that is not true for semiJoins.
- if (joinRel.isSemiJoin()) {
+ if (joinRel.isSemiJoin() || joinRel.getJoinType() == JoinRelType.ANTI) {
return null;
}
@@ -355,7 +376,7 @@ public class HiveRelMdRowCount extends RelMdRowCount {
*/
public static Pair<PKFKRelationInfo, RexNode> constraintsBasedAnalyzeJoinForPKFK(Join join, RelMetadataQuery mq) {
- if (join.isSemiJoin()) {
+ if (join.isSemiJoin() || join.getJoinType() == JoinRelType.ANTI) {
// TODO: Support semijoin
return null;
}
@@ -390,9 +411,9 @@ public class HiveRelMdRowCount extends RelMdRowCount {
return null;
}
- boolean leftIsKey = (join.getJoinType() == JoinRelType.INNER || join.isSemiJoin() || join.getJoinType() == JoinRelType.RIGHT)
+ boolean leftIsKey = (join.getJoinType() == JoinRelType.INNER || join.isSemiJoin() || join.getJoinType() == JoinRelType.ANTI || join.getJoinType() == JoinRelType.RIGHT)
&& leftInputResult.isPkFkJoin;
- boolean rightIsKey = (join.getJoinType() == JoinRelType.INNER || join.isSemiJoin() || join.getJoinType() == JoinRelType.LEFT)
+ boolean rightIsKey = (join.getJoinType() == JoinRelType.INNER || join.isSemiJoin() || join.getJoinType() == JoinRelType.ANTI || join.getJoinType() == JoinRelType.LEFT)
&& rightInputResult.isPkFkJoin;
if (!leftIsKey && !rightIsKey) {
// Nothing to do here, bail out
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdSelectivity.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdSelectivity.java
index 1724ab1..2c36d8f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdSelectivity.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdSelectivity.java
@@ -64,8 +64,12 @@ public class HiveRelMdSelectivity extends RelMdSelectivity {
}
public Double getSelectivity(Join j, RelMetadataQuery mq, RexNode predicate) {
- if (j.getJoinType().equals(JoinRelType.INNER) || j.isSemiJoin()) {
- return computeInnerJoinSelectivity(j, mq, predicate);
+ if (j.getJoinType().equals(JoinRelType.INNER) || j.isSemiJoin() || j.getJoinType().equals(JoinRelType.ANTI)) {
+ Double selectivity = computeInnerJoinSelectivity(j, mq, predicate);
+ if (j.getJoinType().equals(JoinRelType.ANTI)) {
+ return 1 - selectivity;
+ }
+ return selectivity;
} else if (j.getJoinType().equals(JoinRelType.LEFT) ||
j.getJoinType().equals(JoinRelType.RIGHT)) {
double left = mq.getRowCount(j.getLeft());
@@ -142,7 +146,7 @@ public class HiveRelMdSelectivity extends RelMdSelectivity {
ndvEstimate = exponentialBackoff(peLst, colStatMap);
}
- if (j.isSemiJoin()) {
+ if (j.isSemiJoin() || (j.getJoinType().equals(JoinRelType.ANTI))) {
ndvEstimate = Math.min(mq.getRowCount(j.getLeft()),
ndvEstimate);
} else if (j instanceof HiveJoin) {
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdSize.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdSize.java
index d5a4e36..0d99f20 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdSize.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/HiveRelMdSize.java
@@ -20,6 +20,8 @@ package org.apache.hadoop.hive.ql.optimizer.calcite.stats;
import java.util.List;
import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinRelType;
import org.apache.calcite.rel.metadata.ReflectiveRelMetadataProvider;
import org.apache.calcite.rel.metadata.RelMdSize;
import org.apache.calcite.rel.metadata.RelMetadataProvider;
@@ -29,6 +31,7 @@ import org.apache.calcite.rel.type.RelDataTypeField;
import org.apache.calcite.util.BuiltInMethod;
import org.apache.calcite.util.ImmutableNullableList;
import org.apache.hadoop.hive.ql.optimizer.calcite.RelOptHiveTable;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAntiJoin;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveJoin;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSemiJoin;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveTableScan;
@@ -88,6 +91,15 @@ public class HiveRelMdSize extends RelMdSize {
}
public List<Double> averageColumnSizes(HiveSemiJoin rel, RelMetadataQuery mq) {
+ return averageColumnSizesInt(rel, mq);
+ }
+
+ public List<Double> averageColumnSizes(HiveAntiJoin rel, RelMetadataQuery mq) {
+ return averageColumnSizesInt(rel, mq);
+ }
+
+ private List<Double> averageColumnSizesInt(Join rel, RelMetadataQuery mq) {
+ assert rel.getJoinType() == JoinRelType.SEMI || rel.getJoinType() == JoinRelType.ANTI;
final RelNode left = rel.getLeft();
final List<Double> lefts =
mq.getAverageColumnSizes(left);
@@ -96,9 +108,7 @@ public class HiveRelMdSize extends RelMdSize {
}
final int fieldCount = rel.getRowType().getFieldCount();
Double[] sizes = new Double[fieldCount];
- if (lefts != null) {
- lefts.toArray(sizes);
- }
+ lefts.toArray(sizes);
return ImmutableNullableList.copyOf(sizes);
}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTBuilder.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTBuilder.java
index cbeec35..8abc28a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTBuilder.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTBuilder.java
@@ -175,6 +175,9 @@ public class ASTBuilder {
case FULL:
b = ASTBuilder.construct(HiveParser.TOK_FULLOUTERJOIN, "TOK_FULLOUTERJOIN");
break;
+ case ANTI:
+ b = ASTBuilder.construct(HiveParser.TOK_LEFTANTISEMIJOIN, "TOK_LEFTANTISEMIJOIN");
+ break;
}
b.add(left).add(right).add(cond);
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java
index cee9f8b..072a27d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java
@@ -409,7 +409,7 @@ public class ASTConverter {
QueryBlockInfo right = convertSource(join.getRight());
s = new Schema(left.schema, right.schema);
ASTNode cond = join.getCondition().accept(new RexVisitor(s, false, r.getCluster().getRexBuilder()));
- boolean semiJoin = join.isSemiJoin();
+ boolean semiJoin = join.isSemiJoin() || join.getJoinType() == JoinRelType.ANTI;
if (join.getRight() instanceof Join && !semiJoin) {
// should not be done for semijoin since it will change the semantics
// Invert join inputs; this is done because otherwise the SemanticAnalyzer
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java
index fe4ecf0..d414966 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java
@@ -47,6 +47,7 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.CalciteSemanticException;
import org.apache.hadoop.hive.ql.optimizer.calcite.HiveCalciteUtil;
import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelFactories;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAggregate;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAntiJoin;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveProject;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSortExchange;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSortLimit;
@@ -287,12 +288,13 @@ public class PlanModifierForASTConv {
// But we only need the additional project if the left child
// is another join too; if it is not, ASTConverter will swap
// the join inputs, leaving the join operator on the left.
- // we also do it if parent is HiveSemiJoin since ASTConverter won't
- // swap inputs then
+ // we also do it if parent is HiveSemiJoin or HiveAntiJoin since
+ // ASTConverter won't swap inputs then.
// This will help triggering multijoin recognition methods that
// are embedded in SemanticAnalyzer.
if (((Join) parent).getRight() == joinNode &&
- (((Join) parent).getLeft() instanceof Join || parent instanceof HiveSemiJoin) ) {
+ (((Join) parent).getLeft() instanceof Join || parent instanceof HiveSemiJoin
+ || parent instanceof HiveAntiJoin) ) {
validParent = false;
}
} else if (parent instanceof SetOp) {
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/opconventer/HiveOpConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/opconventer/HiveOpConverter.java
index bc2b742..905a4f3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/opconventer/HiveOpConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/opconventer/HiveOpConverter.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.hive.ql.exec.LimitOperator;
import org.apache.hadoop.hive.ql.exec.Operator;
import org.apache.hadoop.hive.ql.exec.TableScanOperator;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAggregate;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAntiJoin;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveFilter;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveJoin;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveMultiJoin;
@@ -98,7 +99,9 @@ public class HiveOpConverter {
} else if (rn instanceof HiveJoin) {
return new JoinVisitor(this).visit((HiveJoin) rn);
} else if (rn instanceof HiveSemiJoin) {
- return new JoinVisitor(this).visit((HiveSemiJoin) rn);
+ return new JoinVisitor(this).visit(rn);
+ } else if (rn instanceof HiveAntiJoin) {
+ return new JoinVisitor(this).visit(rn);
} else if (rn instanceof HiveFilter) {
return new HiveFilterVisitor(this).visit((HiveFilter) rn);
} else if (rn instanceof HiveSortLimit) {
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/opconventer/JoinVisitor.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/opconventer/JoinVisitor.java
index 8d9d5ae..3ad658c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/opconventer/JoinVisitor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/opconventer/JoinVisitor.java
@@ -39,6 +39,7 @@ import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
import org.apache.hadoop.hive.ql.exec.RowSchema;
import org.apache.hadoop.hive.ql.exec.Utilities;
import org.apache.hadoop.hive.ql.optimizer.calcite.HiveCalciteUtil;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAntiJoin;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveJoin;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveMultiJoin;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSemiJoin;
@@ -87,7 +88,8 @@ class JoinVisitor extends HiveRelNodeVisitor<RelNode> {
// 3. Virtual columns
Set<Integer> newVcolsInCalcite = new HashSet<Integer>();
newVcolsInCalcite.addAll(inputs[0].vcolsInCalcite);
- if (joinRel instanceof HiveMultiJoin || !((joinRel instanceof Join) && ((Join) joinRel).isSemiJoin())) {
+ if (joinRel instanceof HiveMultiJoin || !((joinRel instanceof Join) &&
+ ((((Join) joinRel).isSemiJoin()) || (((Join) joinRel).getJoinType() == JoinRelType.ANTI)))) {
int shift = inputs[0].inputs.get(0).getSchema().getSignature().size();
for (int i = 1; i < inputs.length; i++) {
newVcolsInCalcite.addAll(HiveCalciteUtil.shiftVColsSet(inputs[i].vcolsInCalcite, shift));
@@ -116,6 +118,8 @@ class JoinVisitor extends HiveRelNodeVisitor<RelNode> {
joinFilters = ((HiveMultiJoin)joinRel).getJoinFilters();
} else if (joinRel instanceof HiveSemiJoin){
joinFilters = ImmutableList.of(((HiveSemiJoin)joinRel).getJoinFilter());
+ } else if (joinRel instanceof HiveAntiJoin){
+ joinFilters = ImmutableList.of(((HiveAntiJoin)joinRel).getJoinFilter());
} else {
throw new SemanticException("Can't handle join type: " + joinRel.getClass().getName());
}
@@ -159,12 +163,24 @@ class JoinVisitor extends HiveRelNodeVisitor<RelNode> {
noOuterJoin = !hmj.isOuterJoin();
} else {
joinCondns = new JoinCondDesc[1];
- semiJoin = (join instanceof Join) && ((Join) join).isSemiJoin();
+ JoinRelType joinRelType = JoinRelType.INNER;
+ if (join instanceof Join) {
+ joinRelType = ((Join) join).getJoinType();
+ }
JoinType joinType;
- if (semiJoin) {
- joinType = JoinType.LEFTSEMI;
- } else {
- joinType = transformJoinType(((Join)join).getJoinType());
+ switch (joinRelType) {
+ case SEMI:
+ joinType = JoinType.LEFTSEMI;
+ semiJoin = true;
+ break;
+ case ANTI:
+ joinType = JoinType.ANTI;
+ semiJoin = true;
+ break;
+ default:
+ assert join instanceof Join;
+ joinType = transformJoinType(((Join)join).getJoinType());
+ semiJoin = false;
}
joinCondns[0] = new JoinCondDesc(new JoinCond(0, 1, joinType));
noOuterJoin = joinType != JoinType.FULLOUTER && joinType != JoinType.LEFTOUTER
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/CorrelationOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/CorrelationOptimizer.java
index c33f39d..2c35b9d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/CorrelationOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/CorrelationOptimizer.java
@@ -266,7 +266,8 @@ public class CorrelationOptimizer extends Transform {
if (pos == joinCond.getLeft()) {
if (type == JoinDesc.INNER_JOIN ||
type == JoinDesc.LEFT_OUTER_JOIN ||
- type == JoinDesc.LEFT_SEMI_JOIN) {
+ type == JoinDesc.LEFT_SEMI_JOIN ||
+ type == JoinDesc.ANTI_JOIN) {
Operator<? extends OperatorDesc> newCurrentRsOps = rsOps.get(joinCond.getRight());
analyzeReduceSinkOperatorsOfJoinOperator(joinConds, rsOps, newCurrentRsOps,
correlatedRsOps);
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 a690cd7..99b272c 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
@@ -45,6 +45,9 @@ import org.apache.hadoop.hive.ql.exec.vector.VectorizedInputFormatInterface;
import org.apache.hadoop.hive.ql.exec.vector.expressions.ConvertDecimal64ToDecimal;
import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorCoalesce;
import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.DecimalColDivideDecimalScalar;
+import org.apache.hadoop.hive.ql.exec.vector.mapjoin.VectorMapJoinAntiJoinLongOperator;
+import org.apache.hadoop.hive.ql.exec.vector.mapjoin.VectorMapJoinAntiJoinMultiKeyOperator;
+import org.apache.hadoop.hive.ql.exec.vector.mapjoin.VectorMapJoinAntiJoinStringOperator;
import org.apache.hadoop.hive.ql.exec.vector.reducesink.*;
import org.apache.hadoop.hive.ql.exec.vector.udf.VectorUDFArgDesc;
import org.apache.hadoop.hive.ql.io.AcidUtils;
@@ -3416,6 +3419,10 @@ public class Vectorizer implements PhysicalPlanResolver {
vectorMapJoinVariation = VectorMapJoinVariation.LEFT_SEMI;
hashTableKind = HashTableKind.HASH_SET;
break;
+ case JoinDesc.ANTI_JOIN:
+ vectorMapJoinVariation = VectorMapJoinVariation.LEFT_ANTI;
+ hashTableKind = HashTableKind.HASH_SET;
+ break;
default:
throw new HiveException("Unknown join type " + joinType);
}
@@ -3438,6 +3445,9 @@ public class Vectorizer implements PhysicalPlanResolver {
case LEFT_SEMI:
opClass = VectorMapJoinLeftSemiLongOperator.class;
break;
+ case LEFT_ANTI:
+ opClass = VectorMapJoinAntiJoinLongOperator.class;
+ break;
case OUTER:
opClass = VectorMapJoinOuterLongOperator.class;
break;
@@ -3459,6 +3469,9 @@ public class Vectorizer implements PhysicalPlanResolver {
case LEFT_SEMI:
opClass = VectorMapJoinLeftSemiStringOperator.class;
break;
+ case LEFT_ANTI:
+ opClass = VectorMapJoinAntiJoinStringOperator.class;
+ break;
case OUTER:
opClass = VectorMapJoinOuterStringOperator.class;
break;
@@ -3480,6 +3493,9 @@ public class Vectorizer implements PhysicalPlanResolver {
case LEFT_SEMI:
opClass = VectorMapJoinLeftSemiMultiKeyOperator.class;
break;
+ case LEFT_ANTI:
+ opClass = VectorMapJoinAntiJoinMultiKeyOperator.class;
+ break;
case OUTER:
opClass = VectorMapJoinOuterMultiKeyOperator.class;
break;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java
index 4318c4b..1d1f967 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java
@@ -2514,6 +2514,7 @@ public class StatsRulesProcFactory {
case JoinDesc.INNER_JOIN:
case JoinDesc.UNIQUE_JOIN:
case JoinDesc.LEFT_SEMI_JOIN:
+ case JoinDesc.ANTI_JOIN:
break;
}
colStats.setNumNulls(newNumNulls);
@@ -2606,6 +2607,17 @@ public class StatsRulesProcFactory {
// max # of rows = rows from left side
result = Math.min(rowCountParents.get(joinCond.getLeft()), result);
break;
+ case JoinDesc.ANTI_JOIN:
+ long leftRowCount = rowCountParents.get(joinCond.getLeft());
+ if (leftRowCount < result) {
+ // Ideally the inner join count should be less than the left row count. but if its not calculated
+ // properly then we can assume whole of left table will be selected.
+ result = leftRowCount;
+ } else {
+ // The number of result should be left reduced by the number of rows matching (ie inner join count).
+ result = leftRowCount - result;
+ }
+ break;
default:
LOG.debug("Unhandled join type in stats estimation: " + joinCond.getType());
break;
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 180b37e..679b6bd 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
@@ -181,6 +181,7 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.CalciteSemanticException.Unsu
import org.apache.hadoop.hive.ql.optimizer.calcite.cost.HiveAlgorithmsConf;
import org.apache.hadoop.hive.ql.optimizer.calcite.cost.HiveVolcanoPlanner;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAggregate;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAntiJoin;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveExcept;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveFilter;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveGroupingID;
@@ -227,6 +228,7 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveJoinConstraintsRule
import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveJoinProjectTransposeRule;
import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveJoinPushTransitivePredicatesRule;
import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveJoinToMultiJoinRule;
+import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveAntiSemiJoinRule;
import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HivePartitionPruneRule;
import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HivePointLookupOptimizerRule;
import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HivePreFilteringRule;
@@ -393,6 +395,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
HiveTableScan.class,
HiveAggregate.class,
+ HiveAntiJoin.class,
HiveExcept.class,
HiveFilter.class,
HiveIntersect.class,
@@ -1946,7 +1949,6 @@ public class CalcitePlanner extends SemanticAnalyzer {
LOG.debug("Plan After Join Reordering:\n"
+ RelOptUtil.toString(calciteOptimizedPlan, SqlExplainLevel.ALL_ATTRIBUTES));
}
-
return calciteOptimizedPlan;
}
@@ -2065,8 +2067,10 @@ public class CalcitePlanner extends SemanticAnalyzer {
}
rules.add(HiveJoinAddNotNullRule.INSTANCE_JOIN);
rules.add(HiveJoinAddNotNullRule.INSTANCE_SEMIJOIN);
+ rules.add(HiveJoinAddNotNullRule.INSTANCE_ANTIJOIN);
rules.add(HiveJoinPushTransitivePredicatesRule.INSTANCE_JOIN);
rules.add(HiveJoinPushTransitivePredicatesRule.INSTANCE_SEMIJOIN);
+ rules.add(HiveJoinPushTransitivePredicatesRule.INSTANCE_ANTIJOIN);
rules.add(HiveSortMergeRule.INSTANCE);
rules.add(HiveSortPullUpConstantsRule.SORT_LIMIT_INSTANCE);
rules.add(HiveSortPullUpConstantsRule.SORT_EXCHANGE_INSTANCE);
@@ -2130,6 +2134,16 @@ public class CalcitePlanner extends SemanticAnalyzer {
HiveRemoveSqCountCheck.INSTANCE);
}
+ // 10. Convert left outer join + null filter on right side table column to anti join. Add this
+ // rule after all the optimization for which calcite support for anti join is missing.
+ // Needs to be done before ProjectRemoveRule as it expect a project over filter.
+ // This is done before join re-ordering as join re-ordering is converting the left outer
+ // to right join in some cases before converting back again to left outer.
+ if (conf.getBoolVar(ConfVars.HIVE_CONVERT_ANTI_JOIN)) {
+ generatePartialProgram(program, false, HepMatchOrder.DEPTH_FIRST,
+ HiveAntiSemiJoinRule.INSTANCE);
+ }
+
// Trigger program
perfLogger.PerfLogBegin(this.getClass().getName(), PerfLogger.OPTIMIZER);
basePlan = executeProgram(basePlan, program.build(), mdProvider, executorProvider);
@@ -2881,6 +2895,10 @@ public class CalcitePlanner extends SemanticAnalyzer {
calciteJoinType = JoinRelType.SEMI;
leftSemiJoin = true;
break;
+ case ANTI:
+ calciteJoinType = JoinRelType.ANTI;
+ leftSemiJoin = true;
+ break;
case INNER:
default:
calciteJoinType = JoinRelType.INNER;
@@ -2914,10 +2932,18 @@ public class CalcitePlanner extends SemanticAnalyzer {
final RelDataType combinedRowType = SqlValidatorUtil.createJoinType(
cluster.getTypeFactory(), inputRels[0].getRowType(), inputRels[1].getRowType(),
null, ImmutableList.of());
- topRel = HiveSemiJoin.getSemiJoin(cluster, cluster.traitSetOf(HiveRelNode.CONVENTION),
- inputRels[0], inputRels[1],
- HiveCalciteUtil.fixNullability(cluster.getRexBuilder(),
- calciteJoinCond, RelOptUtil.getFieldTypeList(combinedRowType)));
+
+ if (hiveJoinType == JoinType.LEFTSEMI) {
+ topRel = HiveSemiJoin.getSemiJoin(cluster, cluster.traitSetOf(HiveRelNode.CONVENTION),
+ inputRels[0], inputRels[1],
+ HiveCalciteUtil.fixNullability(cluster.getRexBuilder(),
+ calciteJoinCond, RelOptUtil.getFieldTypeList(combinedRowType)));
+ } else {
+ topRel = HiveAntiJoin.getAntiJoin(cluster, cluster.traitSetOf(HiveRelNode.CONVENTION),
+ inputRels[0], inputRels[1],
+ HiveCalciteUtil.fixNullability(cluster.getRexBuilder(),
+ calciteJoinCond, RelOptUtil.getFieldTypeList(combinedRowType)));
+ }
// Create join RR: we need to check whether we need to update left RR in case
// previous call to projectNonColumnEquiConditions updated it
@@ -3019,6 +3045,9 @@ public class CalcitePlanner extends SemanticAnalyzer {
case HiveParser.TOK_LEFTSEMIJOIN:
hiveJoinType = JoinType.LEFTSEMI;
break;
+ case HiveParser.TOK_LEFTANTISEMIJOIN:
+ hiveJoinType = JoinType.ANTI;
+ break;
default:
hiveJoinType = JoinType.INNER;
break;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/JoinType.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/JoinType.java
index fd8bb53..18e0de2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/JoinType.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/JoinType.java
@@ -23,5 +23,5 @@ package org.apache.hadoop.hive.ql.parse;
*
*/
public enum JoinType {
- INNER, LEFTOUTER, RIGHTOUTER, FULLOUTER, UNIQUE, LEFTSEMI
+ INNER, LEFTOUTER, RIGHTOUTER, FULLOUTER, UNIQUE, LEFTSEMI, ANTI
};
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 8b1df4b..ce8bf73 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
@@ -1478,6 +1478,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
|| (node.getToken().getType() == HiveParser.TOK_CROSSJOIN)
|| isOuterJoinToken(node)
|| (node.getToken().getType() == HiveParser.TOK_LEFTSEMIJOIN)
+ || (node.getToken().getType() == HiveParser.TOK_LEFTANTISEMIJOIN)
|| (node.getToken().getType() == HiveParser.TOK_UNIQUEJOIN);
}
@@ -9777,6 +9778,10 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
joinTree.setNoSemiJoin(false);
condn[0] = new JoinCond(0, 1, JoinType.LEFTSEMI);
break;
+ case ANTI:
+ joinTree.setNoSemiJoin(false);
+ condn[0] = new JoinCond(0, 1, JoinType.ANTI);
+ break;
default:
condn[0] = new JoinCond(0, 1, JoinType.INNER);
joinTree.setNoOuterJoin(true);
@@ -9879,6 +9884,10 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
joinTree.setNoSemiJoin(false);
condn[0] = new JoinCond(0, 1, JoinType.LEFTSEMI);
break;
+ case HiveParser.TOK_LEFTANTISEMIJOIN:
+ joinTree.setNoSemiJoin(false);
+ condn[0] = new JoinCond(0, 1, JoinType.ANTI);
+ break;
default:
condn[0] = new JoinCond(0, 1, JoinType.INNER);
joinTree.setNoOuterJoin(true);
@@ -10092,7 +10101,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
for (ASTNode hintNode : hints) {
for (Node node : hintNode.getChildren()) {
ASTNode hint = (ASTNode) node;
- if (hint.getChild(0).getType() != HintParser.TOK_LEFTSEMIJOIN) {
+ if (hint.getChild(0).getType() != HintParser.TOK_LEFTSEMIJOIN &&
+ hint.getChild(0).getType() != HintParser.TOK_LEFTANTISEMIJOIN) {
continue;
}
if (result == null) {
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/JoinCondDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/JoinCondDesc.java
index 0eb03e9..75f9d81 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/JoinCondDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/JoinCondDesc.java
@@ -73,6 +73,9 @@ public class JoinCondDesc implements Serializable {
case LEFTSEMI:
type = JoinDesc.LEFT_SEMI_JOIN;
break;
+ case ANTI:
+ type = JoinDesc.ANTI_JOIN;
+ break;
default:
assert false;
}
@@ -140,6 +143,9 @@ public class JoinCondDesc implements Serializable {
case JoinDesc.LEFT_SEMI_JOIN:
sb.append("Left Semi Join ");
break;
+ case JoinDesc.ANTI_JOIN:
+ sb.append("Anti Join ");
+ break;
default:
sb.append("Unknown Join ");
break;
@@ -175,6 +181,9 @@ public class JoinCondDesc implements Serializable {
case JoinDesc.LEFT_SEMI_JOIN:
join.put("type", "Left Semi");
break;
+ case JoinDesc.ANTI_JOIN:
+ join.put("type", "Anti");
+ break;
default:
join.put("type", "Unknown Join");
break;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/JoinDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/JoinDesc.java
index be2d8e5..9025bc4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/JoinDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/JoinDesc.java
@@ -48,6 +48,7 @@ public class JoinDesc extends AbstractOperatorDesc {
public static final int FULL_OUTER_JOIN = 3;
public static final int UNIQUE_JOIN = 4;
public static final int LEFT_SEMI_JOIN = 5;
+ public static final int ANTI_JOIN = 6;
// used to handle skew join
private boolean handleSkewJoin = false;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorMapJoinDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorMapJoinDesc.java
index a0ee3a9..f32a230 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorMapJoinDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/VectorMapJoinDesc.java
@@ -89,7 +89,8 @@ public class VectorMapJoinDesc extends AbstractVectorDesc {
INNER_BIG_ONLY,
LEFT_SEMI,
OUTER,
- FULL_OUTER
+ FULL_OUTER,
+ LEFT_ANTI
}
private HashTableImplementationType hashTableImplementationType;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ppd/PredicateTransitivePropagate.java b/ql/src/java/org/apache/hadoop/hive/ql/ppd/PredicateTransitivePropagate.java
index 9666dd7..8be4239 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ppd/PredicateTransitivePropagate.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ppd/PredicateTransitivePropagate.java
@@ -203,6 +203,7 @@ public class PredicateTransitivePropagate extends Transform {
vector.add(right, left);
break;
case JoinDesc.LEFT_OUTER_JOIN:
+ case JoinDesc.ANTI_JOIN:
vector.add(left, right);
break;
case JoinDesc.RIGHT_OUTER_JOIN:
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ppd/SyntheticJoinPredicate.java b/ql/src/java/org/apache/hadoop/hive/ql/ppd/SyntheticJoinPredicate.java
index 7316df0..19b7be5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ppd/SyntheticJoinPredicate.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ppd/SyntheticJoinPredicate.java
@@ -339,6 +339,12 @@ public class SyntheticJoinPredicate extends Transform {
vector.add(right, left);
break;
case JoinDesc.LEFT_OUTER_JOIN:
+ case JoinDesc.ANTI_JOIN:
+ //TODO : In case of anti join, bloom filter can be created on left side also ("IN (keylist right table)").
+ // But the filter should be "not-in" ("NOT IN (keylist right table)") as we want to select the records from
+ // left side which are not present in the right side. But it may cause wrong result as
+ // bloom filter may have false positive and thus simply adding not is not correct,
+ // special handling is required for "NOT IN".
vector.add(right, left);
break;
case JoinDesc.RIGHT_OUTER_JOIN:
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/MapJoinTestConfig.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/MapJoinTestConfig.java
index a250533..37e41743 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/MapJoinTestConfig.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/MapJoinTestConfig.java
@@ -308,6 +308,9 @@ public class MapJoinTestConfig {
case LEFT_SEMI:
joinDescType = JoinDesc.LEFT_SEMI_JOIN;
break;
+ case LEFT_ANTI:
+ joinDescType = JoinDesc.ANTI_JOIN;
+ break;
case OUTER:
joinDescType = JoinDesc.LEFT_OUTER_JOIN;
break;
@@ -363,6 +366,7 @@ public class MapJoinTestConfig {
hashTableKind = HashTableKind.HASH_MULTISET;
break;
case LEFT_SEMI:
+ case LEFT_ANTI:
hashTableKind = HashTableKind.HASH_SET;
break;
case OUTER:
@@ -1244,4 +1248,4 @@ public class MapJoinTestConfig {
private static List<Operator<? extends OperatorDesc>> newOperatorList() {
return new ArrayList<Operator<? extends OperatorDesc>>();
}
-}
\ No newline at end of file
+}
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/MapJoinTestData.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/MapJoinTestData.java
index 2f5479e..81cc1b6 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/MapJoinTestData.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/MapJoinTestData.java
@@ -281,6 +281,7 @@ public class MapJoinTestData {
switch (testDesc.vectorMapJoinVariation) {
case INNER_BIG_ONLY:
case LEFT_SEMI:
+ case LEFT_ANTI:
testData.generateRandomSmallTableCounts(testDesc, random);
break;
case INNER:
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/TestMapJoinOperator.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/TestMapJoinOperator.java
index da74fd5..7e7b356 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/TestMapJoinOperator.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/TestMapJoinOperator.java
@@ -160,6 +160,7 @@ public class TestMapJoinOperator {
return (smallTableValueSize > 0);
case INNER_BIG_ONLY:
case LEFT_SEMI:
+ case LEFT_ANTI:
return (smallTableValueSize == 0);
case OUTER:
return true;
@@ -1590,6 +1591,7 @@ public class TestMapJoinOperator {
break;
case INNER_BIG_ONLY:
case LEFT_SEMI:
+ case LEFT_ANTI:
{
Object[] outputObjects = new Object[testDesc.outputColumnNames.length];
@@ -1792,6 +1794,8 @@ public class TestMapJoinOperator {
case FULL_OUTER:
executeTestFullOuter(testDesc, testData, title);
break;
+ case LEFT_ANTI: //TODO
+ break;
default:
throw new RuntimeException("Unexpected Vector MapJoin variation " +
testDesc.vectorMapJoinVariation);
@@ -2050,4 +2054,4 @@ public class TestMapJoinOperator {
" variation " + testDesc.vectorMapJoinVariation + option);
}
}
-}
\ No newline at end of file
+}
diff --git a/ql/src/test/queries/clientpositive/antijoin.q b/ql/src/test/queries/clientpositive/antijoin.q
new file mode 100644
index 0000000..2b41b25
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/antijoin.q
@@ -0,0 +1,67 @@
+--! qt:dataset:src
+--! qt:dataset:part
+SET hive.vectorized.execution.enabled=false;
+set hive.mapred.mode=nonstrict;
+SET hive.auto.convert.join=false;
+SET hive.auto.convert.anti.join=true;
+-- SORT_QUERY_RESULTS
+
+create table t1_n55 as select cast(key as int) key, value from src where key <= 10;
+
+select * from t1_n55 sort by key;
+
+create table t2_n33 as select cast(2*key as int) key, value from t1_n55;
+
+select * from t2_n33 sort by key;
+
+create table t3_n12 as select * from (select * from t1_n55 union all select * from t2_n33) b;
+select * from t3_n12 sort by key, value;
+
+explain select a.key, a.value from t1_n55 a where not exists (select 1 from t2_n33 b where a.key=b.key) sort by a.key, a.value;
+explain cbo select a.key, a.value from t1_n55 a where not exists (select 1 from t2_n33 b where a.key=b.key) sort by a.key, a.value;
+select a.key, a.value from t1_n55 a where not exists (select 1 from t2_n33 b where a.key=b.key) sort by a.key, a.value;
+
+explain select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key where b.key is null;
+explain cbo select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key where b.key is null;
+select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key where b.key is null;
+
+explain select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key join t3_n12 c on a.key=c.key where b.key is null sort by a.key, a.value;
+explain cbo select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key join t3_n12 c on a.key=c.key where b.key is null sort by a.key, a.value;
+select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key join t3_n12 c on a.key=c.key where b.key is null sort by a.key, a.value;
+
+SET hive.vectorized.execution.enabled=true;
+SET hive.auto.convert.join=true;
+explain select a.key, a.value from t1_n55 a where not exists (select 1 from t2_n33 b where a.key=b.key) sort by a.key, a.value;
+explain cbo select a.key, a.value from t1_n55 a where not exists (select 1 from t2_n33 b where a.key=b.key) sort by a.key, a.value;
+select a.key, a.value from t1_n55 a where not exists (select 1 from t2_n33 b where a.key=b.key) sort by a.key, a.value;
+
+explain select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key where b.key is null;
+explain cbo select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key where b.key is null;
+select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key where b.key is null;
+
+explain select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key join t3_n12 c on a.key=c.key where b.key is null sort by a.key, a.value;
+explain cbo select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key join t3_n12 c on a.key=c.key where b.key is null sort by a.key, a.value;
+select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key join t3_n12 c on a.key=c.key where b.key is null sort by a.key, a.value;
+
+-- single extra simple filter on right side.
+explain select a.key from t1_n55 a left join t2_n33 b on a.key = b.key where b.key is null and b.value > 'val_1';
+
+-- single extra simple filter on left side.
+explain select a.key from t1_n55 a left join t2_n33 b on a.key = b.key where b.key is null and a.key > 100;
+
+-- single extra non deterministic filter on right side.
+explain select a.key from t1_n55 a left join t2_n33 b on a.key = b.key where b.key is null and rand(length(b.value)) > 100;
+
+-- single extra deterministic filter on left side.
+explain select a.key from t1_n55 a left join t2_n33 b on a.key = b.key where b.key is null and rand(a.key) > 100;
+
+-- multiple extra deterministic filter on right side.
+explain select a.key from t1_n55 a left join t2_n33 b on a.key = b.key where b.key is null and rand(length(b.value)) < 100 and rand(length(b.value)) > 100;
+
+-- multiple extra deterministic filter on left side.
+explain select a.key from t1_n55 a left join t2_n33 b on a.key = b.key where b.key is null and rand(a.key) > 100 and rand(length(a.value)) > 100;
+
+-- filter with 'or' are not converted to anti join
+explain select a.key from t1_n55 a left join t2_n33 b on a.key = b.key where b.key is null or b.value is null;
+explain select a.key from t1_n55 a left join t2_n33 b on a.key = b.key where b.key is null or b.value = 'val_1';
+
diff --git a/ql/src/test/queries/clientpositive/explainuser_1.q b/ql/src/test/queries/clientpositive/explainuser_1.q
index 23bdb79..ce967f4 100644
--- a/ql/src/test/queries/clientpositive/explainuser_1.q
+++ b/ql/src/test/queries/clientpositive/explainuser_1.q
@@ -679,3 +679,46 @@ explain FROM T1_n119 a RIGHT OUTER JOIN T2_n70 c ON c.key+1=a.key select /*+ STR
explain FROM T1_n119 a FULL OUTER JOIN T2_n70 c ON c.key+1=a.key select /*+ STREAMTABLE(a) */ sum(hash(a.key)), sum(hash(a.val)), sum(hash(c.key));
explain select /*+ mapjoin(v)*/ sum(hash(k.key)), sum(hash(v.val)) from T1_n119 k left outer join T1_n119 v on k.key+1=v.key;
+
+set hive.auto.convert.anti.join=false;
+
+explain select *
+from src_cbo b
+where not exists
+ (select distinct a.key
+ from src_cbo a
+ where b.value = a.value and a.value > 'val_2'
+ )
+;
+
+explain select *
+from src_cbo b
+group by key, value
+having not exists
+ (select a.key
+ from src_cbo a
+ where b.value = a.value and a.key = b.key and a.value > 'val_12'
+ )
+;
+
+create view cv1_n5_anti as
+select *
+from src_cbo b
+where not exists
+ (select a.key
+ from src_cbo a
+ where b.value = a.value and a.key = b.key and a.value > 'val_9')
+;
+
+explain select * from cv1_n5_anti;
+
+explain select *
+from (select *
+ from src_cbo b
+ where not exists
+ (select a.key
+ from src_cbo a
+ where b.value = a.value and a.key = b.key and a.value > 'val_9')
+ ) a
+;
+set hive.auto.convert.anti.join=true;
\ No newline at end of file
diff --git a/ql/src/test/queries/clientpositive/external_jdbc_table_perf.q b/ql/src/test/queries/clientpositive/external_jdbc_table_perf.q
index 0320107..d277850 100644
--- a/ql/src/test/queries/clientpositive/external_jdbc_table_perf.q
+++ b/ql/src/test/queries/clientpositive/external_jdbc_table_perf.q
@@ -1750,6 +1750,199 @@ order by r, wq, ref, fee
limit 100;
+set hive.auto.convert.anti.join=false;
+
+explain
+SELECT cd_gender,
+ cd_marital_status,
+ cd_education_status,
+ Count(*) cnt1,
+ cd_purchase_estimate,
+ Count(*) cnt2,
+ cd_credit_rating,
+ Count(*) cnt3
+FROM customer c,
+ customer_address ca,
+ customer_demographics
+WHERE c.c_current_addr_sk = ca.ca_address_sk
+ AND ca_state IN ( 'CO', 'IL', 'MN' )
+ AND cd_demo_sk = c.c_current_cdemo_sk
+ AND EXISTS (SELECT *
+ FROM store_sales,
+ date_dim
+ WHERE c.c_customer_sk = ss_customer_sk
+ AND ss_sold_date_sk = d_date_sk
+ AND d_year = 1999
+ AND d_moy BETWEEN 1 AND 1 + 2)
+ AND ( NOT EXISTS (SELECT *
+ FROM web_sales,
+ date_dim
+ WHERE c.c_customer_sk = ws_bill_customer_sk
+ AND ws_sold_date_sk = d_date_sk
+ AND d_year = 1999
+ AND d_moy BETWEEN 1 AND 1 + 2)
+ AND NOT EXISTS (SELECT *
+ FROM catalog_sales,
+ date_dim
+ WHERE c.c_customer_sk = cs_ship_customer_sk
+ AND cs_sold_date_sk = d_date_sk
+ AND d_year = 1999
+ AND d_moy BETWEEN 1 AND 1 + 2) )
+GROUP BY cd_gender,
+ cd_marital_status,
+ cd_education_status,
+ cd_purchase_estimate,
+ cd_credit_rating
+ORDER BY cd_gender,
+ cd_marital_status,
+ cd_education_status,
+ cd_purchase_estimate,
+ cd_credit_rating
+LIMIT 100;
+SELECT cd_gender,
+ cd_marital_status,
+ cd_education_status,
+ Count(*) cnt1,
+ cd_purchase_estimate,
+ Count(*) cnt2,
+ cd_credit_rating,
+ Count(*) cnt3
+FROM customer c,
+ customer_address ca,
+ customer_demographics
+WHERE c.c_current_addr_sk = ca.ca_address_sk
+ AND ca_state IN ( 'CO', 'IL', 'MN' )
+ AND cd_demo_sk = c.c_current_cdemo_sk
+ AND EXISTS (SELECT *
+ FROM store_sales,
+ date_dim
+ WHERE c.c_customer_sk = ss_customer_sk
+ AND ss_sold_date_sk = d_date_sk
+ AND d_year = 1999
+ AND d_moy BETWEEN 1 AND 1 + 2)
+ AND ( NOT EXISTS (SELECT *
+ FROM web_sales,
+ date_dim
+ WHERE c.c_customer_sk = ws_bill_customer_sk
+ AND ws_sold_date_sk = d_date_sk
+ AND d_year = 1999
+ AND d_moy BETWEEN 1 AND 1 + 2)
+ AND NOT EXISTS (SELECT *
+ FROM catalog_sales,
+ date_dim
+ WHERE c.c_customer_sk = cs_ship_customer_sk
+ AND cs_sold_date_sk = d_date_sk
+ AND d_year = 1999
+ AND d_moy BETWEEN 1 AND 1 + 2) )
+GROUP BY cd_gender,
+ cd_marital_status,
+ cd_education_status,
+ cd_purchase_estimate,
+ cd_credit_rating
+ORDER BY cd_gender,
+ cd_marital_status,
+ cd_education_status,
+ cd_purchase_estimate,
+ cd_credit_rating
+LIMIT 100;
+
+
+
+explain
+SELECT cd_gender,
+ cd_marital_status,
+ cd_education_status,
+ Count(*) cnt1,
+ cd_purchase_estimate,
+ Count(*) cnt2,
+ cd_credit_rating,
+ Count(*) cnt3
+FROM customer c,
+ customer_address ca,
+ customer_demographics
+WHERE c.c_current_addr_sk = ca.ca_address_sk
+ AND ca_state IN ( 'CO', 'IL', 'MN' )
+ AND cd_demo_sk = c.c_current_cdemo_sk
+ AND EXISTS (SELECT *
+ FROM store_sales,
+ date_dim
+ WHERE c.c_customer_sk = ss_customer_sk
+ AND ss_sold_date_sk = d_date_sk
+ AND d_year = 1999
+ AND d_moy NOT BETWEEN 1 AND 1 + 2)
+ AND ( NOT EXISTS (SELECT *
+ FROM web_sales,
+ date_dim
+ WHERE c.c_customer_sk = ws_bill_customer_sk
+ AND ws_sold_date_sk = d_date_sk
+ AND d_year = 1999
+ AND d_moy NOT BETWEEN 1 AND 1 + 2)
+ AND NOT EXISTS (SELECT *
+ FROM catalog_sales,
+ date_dim
+ WHERE c.c_customer_sk = cs_ship_customer_sk
+ AND cs_sold_date_sk = d_date_sk
+ AND d_year = 1999
+ AND d_moy NOT BETWEEN 1 AND 1 + 2) )
+GROUP BY cd_gender,
+ cd_marital_status,
+ cd_education_status,
+ cd_purchase_estimate,
+ cd_credit_rating
+ORDER BY cd_gender,
+ cd_marital_status,
+ cd_education_status,
+ cd_purchase_estimate,
+ cd_credit_rating
+LIMIT 100;
+SELECT cd_gender,
+ cd_marital_status,
+ cd_education_status,
+ Count(*) cnt1,
+ cd_purchase_estimate,
+ Count(*) cnt2,
+ cd_credit_rating,
+ Count(*) cnt3
+FROM customer c,
+ customer_address ca,
+ customer_demographics
+WHERE c.c_current_addr_sk = ca.ca_address_sk
+ AND ca_state IN ( 'CO', 'IL', 'MN' )
+ AND cd_demo_sk = c.c_current_cdemo_sk
+ AND EXISTS (SELECT *
+ FROM store_sales,
+ date_dim
+ WHERE c.c_customer_sk = ss_customer_sk
+ AND ss_sold_date_sk = d_date_sk
+ AND d_year = 1999
+ AND d_moy NOT BETWEEN 1 AND 1 + 2)
+ AND ( NOT EXISTS (SELECT *
+ FROM web_sales,
+ date_dim
+ WHERE c.c_customer_sk = ws_bill_customer_sk
+ AND ws_sold_date_sk = d_date_sk
+ AND d_year = 1999
+ AND d_moy NOT BETWEEN 1 AND 1 + 2)
+ AND NOT EXISTS (SELECT *
+ FROM catalog_sales,
+ date_dim
+ WHERE c.c_customer_sk = cs_ship_customer_sk
+ AND cs_sold_date_sk = d_date_sk
+ AND d_year = 1999
+ AND d_moy NOT BETWEEN 1 AND 1 + 2) )
+GROUP BY cd_gender,
+ cd_marital_status,
+ cd_education_status,
+ cd_purchase_estimate,
+ cd_credit_rating
+ORDER BY cd_gender,
+ cd_marital_status,
+ cd_education_status,
+ cd_purchase_estimate,
+ cd_credit_rating
+LIMIT 100;
+set hive.auto.convert.anti.join=true;
+
DROP TABLE catalog_sales;
DROP TABLE catalog_returns;
DROP TABLE store_sales;
diff --git a/ql/src/test/queries/clientpositive/lineage3.q b/ql/src/test/queries/clientpositive/lineage3.q
index e8ef4fe..8d5bd30 100644
--- a/ql/src/test/queries/clientpositive/lineage3.q
+++ b/ql/src/test/queries/clientpositive/lineage3.q
@@ -205,4 +205,11 @@ insert into dest_dp3 partition (y=2, m, d) select first, word, month m, day d wh
insert into dest_dp2 partition (y=1, m) select f, w, m
insert into dest_dp1 partition (year=0) select f, w;
+set hive.auto.convert.anti.join=false;
+select * from src1 a
+where not exists
+ (select cint from alltypesorc b
+ where a.key = b.ctinyint + 300)
+and key > 300;
+
reset hive.metastore.disallow.incompatible.col.type.changes;
diff --git a/ql/src/test/queries/clientpositive/subquery_in_having.q b/ql/src/test/queries/clientpositive/subquery_in_having.q
index 732b3e5..9e72184 100644
--- a/ql/src/test/queries/clientpositive/subquery_in_having.q
+++ b/ql/src/test/queries/clientpositive/subquery_in_having.q
@@ -147,6 +147,22 @@ where NOT EXISTS (select key from src_null_n4 where src_null_n4.value <> b.value
group by key, value
having count(*) not in (select count(*) from src_null_n4 s1 where s1.key > '9' and s1.value <> b.value group by s1.key );
+set hive.auto.convert.anti.join=false;
+
+select key, value, count(*)
+from src_null_n4 b
+where NOT EXISTS (select key from src_null_n4 where src_null_n4.value <> b.value)
+group by key, value
+having count(*) not in (select count(*) from src_null_n4 s1 where s1.key > '9' and s1.value <> b.value group by s1.key );
+
+
+explain
+select key, value, count(*)
+from src_null_n4 b
+where NOT EXISTS (select key from src_null_n4 where src_null_n4.value <> b.value)
+group by key, value
+having count(*) not in (select count(*) from src_null_n4 s1 where s1.key > '9' and s1.value <> b.value group by s1.key );
+
select key, value, count(*)
from src_null_n4 b
where NOT EXISTS (select key from src_null_n4 where src_null_n4.value <> b.value)
diff --git a/ql/src/test/queries/clientpositive/subquery_multi.q b/ql/src/test/queries/clientpositive/subquery_multi.q
index 03459ac..7389384 100644
--- a/ql/src/test/queries/clientpositive/subquery_multi.q
+++ b/ql/src/test/queries/clientpositive/subquery_multi.q
@@ -143,6 +143,48 @@ select * from part ws1 where
and ws1.p_retailprice <> ws2.p_retailprice)
and not exists(select * from part_null wr1 where ws1.p_type = wr1.p_name);
+set hive.auto.convert.anti.join=false;
+
+explain select * from part_null where p_size IN (select p_size from part_null) AND NOT EXISTS (select c from tempty);
+select * from part_null where p_size IN (select p_size from part_null) AND NOT EXISTS (select c from tempty);
+
+explain select * from part_null where p_name IN (select p_name from part_null) AND NOT EXISTS (select c from tempty);
+select * from part_null where p_name IN (select p_name from part_null) AND NOT EXISTS (select c from tempty);
+
+
+explain
+select key, value, count(*)
+from src b
+where b.key in (select key from src where src.value = b.value)
+group by key, value
+having count(*) in (select count(*) from src s1 where s1.key > '9' and not exists (select * from src s2 where s1.value = s2.value) group by s1.key )
+ ;
+select key, value, count(*)
+from src b
+where b.key in (select key from src where src.value = b.value)
+group by key, value
+having count(*) in (select count(*) from src s1 where s1.key > '9' and not exists (select * from src s2 where s1.value = s2.value) group by s1.key ) ;
+
+
+explain select count(*) from src
+ where src.key in (select key from src s1 where s1.key > '9')
+ or src.value is not null
+ or not exists(select key from src);
+
+select count(*) from src
+ where src.key in (select key from src s1 where s1.key > '9')
+ or src.value is not null
+ or not exists(select key from src);
+
+-- EXISTS and NOT EXISTS with non-equi predicate
+explain select * from part ws1 where
+ exists (select * from part ws2 where ws1.p_type= ws2.p_type
+ and ws1.p_retailprice <> ws2.p_retailprice)
+ and not exists(select * from part_null wr1 where ws1.p_type = wr1.p_name);
+select * from part ws1 where
+ exists (select * from part ws2 where ws1.p_type= ws2.p_type
+ and ws1.p_retailprice <> ws2.p_retailprice)
+ and not exists(select * from part_null wr1 where ws1.p_type = wr1.p_name);
drop table tnull;
drop table tempty;
diff --git a/ql/src/test/queries/clientpositive/subquery_scalar.q b/ql/src/test/queries/clientpositive/subquery_scalar.q
index 64973a1..598b671 100644
--- a/ql/src/test/queries/clientpositive/subquery_scalar.q
+++ b/ql/src/test/queries/clientpositive/subquery_scalar.q
@@ -148,6 +148,10 @@ select sum(l_extendedprice) from lineitem, part where p_partkey = l_partkey and
explain select * from part_null_n0 where p_name IN (select p_name from part where part.p_type = part_null_n0.p_type AND p_brand NOT LIKE (select min(p_brand) from part pp where part.p_type = pp.p_type));
select * from part_null_n0 where p_name IN (select p_name from part where part.p_type = part_null_n0.p_type AND p_brand NOT LIKE (select min(p_brand) from part pp where part.p_type = pp.p_type));
+-- non corr, is null , is not converted to anti join.
+explain select * from part where (select i from tnull_n0 limit 1) is null;
+select * from part where (select i from tnull_n0 limit 1) is null;
+
drop table tnull_n0;
drop table part_null_n0;
drop table tempty_n0;
@@ -288,8 +292,6 @@ explain cbo with avg_sales as
where ss_sold_date_sk = d_date_sk
and d_year between 1999 and 2001 ) x)
select * from store_sales where ss_list_price > (select average_sales from avg_sales);
-
-
DROP TABLE store_sales;
DROP TABLE date_dim;
diff --git a/ql/src/test/results/clientpositive/llap/antijoin.q.out b/ql/src/test/results/clientpositive/llap/antijoin.q.out
new file mode 100644
index 0000000..7b5c288
--- /dev/null
+++ b/ql/src/test/results/clientpositive/llap/antijoin.q.out
@@ -0,0 +1,1701 @@
+PREHOOK: query: create table t1_n55 as select cast(key as int) key, value from src where key <= 10
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@src
+PREHOOK: Output: database:default
+PREHOOK: Output: default@t1_n55
+POSTHOOK: query: create table t1_n55 as select cast(key as int) key, value from src where key <= 10
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@src
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@t1_n55
+POSTHOOK: Lineage: t1_n55.key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: t1_n55.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+PREHOOK: query: select * from t1_n55 sort by key
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+#### A masked pattern was here ####
+POSTHOOK: query: select * from t1_n55 sort by key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+#### A masked pattern was here ####
+0 val_0
+0 val_0
+0 val_0
+10 val_10
+2 val_2
+4 val_4
+5 val_5
+5 val_5
+5 val_5
+8 val_8
+9 val_9
+PREHOOK: query: create table t2_n33 as select cast(2*key as int) key, value from t1_n55
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@t1_n55
+PREHOOK: Output: database:default
+PREHOOK: Output: default@t2_n33
+POSTHOOK: query: create table t2_n33 as select cast(2*key as int) key, value from t1_n55
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@t2_n33
+POSTHOOK: Lineage: t2_n33.key EXPRESSION [(t1_n55)t1_n55.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: t2_n33.value SIMPLE [(t1_n55)t1_n55.FieldSchema(name:value, type:string, comment:null), ]
+PREHOOK: query: select * from t2_n33 sort by key
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+POSTHOOK: query: select * from t2_n33 sort by key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+0 val_0
+0 val_0
+0 val_0
+10 val_5
+10 val_5
+10 val_5
+16 val_8
+18 val_9
+20 val_10
+4 val_2
+8 val_4
+PREHOOK: query: create table t3_n12 as select * from (select * from t1_n55 union all select * from t2_n33) b
+PREHOOK: type: CREATETABLE_AS_SELECT
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+PREHOOK: Output: database:default
+PREHOOK: Output: default@t3_n12
+POSTHOOK: query: create table t3_n12 as select * from (select * from t1_n55 union all select * from t2_n33) b
+POSTHOOK: type: CREATETABLE_AS_SELECT
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@t3_n12
+POSTHOOK: Lineage: t3_n12.key EXPRESSION [(t1_n55)t1_n55.FieldSchema(name:key, type:int, comment:null), (t2_n33)t2_n33.FieldSchema(name:key, type:int, comment:null), ]
+POSTHOOK: Lineage: t3_n12.value EXPRESSION [(t1_n55)t1_n55.FieldSchema(name:value, type:string, comment:null), (t2_n33)t2_n33.FieldSchema(name:value, type:string, comment:null), ]
+PREHOOK: query: select * from t3_n12 sort by key, value
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t3_n12
+#### A masked pattern was here ####
+POSTHOOK: query: select * from t3_n12 sort by key, value
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t3_n12
+#### A masked pattern was here ####
+0 val_0
+0 val_0
+0 val_0
+0 val_0
+0 val_0
+0 val_0
+10 val_10
+10 val_5
+10 val_5
+10 val_5
+16 val_8
+18 val_9
+2 val_2
+20 val_10
+4 val_2
+4 val_4
+5 val_5
+5 val_5
+5 val_5
+8 val_4
+8 val_8
+9 val_9
+PREHOOK: query: explain select a.key, a.value from t1_n55 a where not exists (select 1 from t2_n33 b where a.key=b.key) sort by a.key, a.value
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+POSTHOOK: query: explain select a.key, a.value from t1_n55 a where not exists (select 1 from t2_n33 b where a.key=b.key) sort by a.key, a.value
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+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), Map 4 (SIMPLE_EDGE)
+ Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: a
+ Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: key (type: int), value (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: int)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col1 (type: string)
+ Execution mode: llap
+ LLAP IO: all inputs
+ Map 4
+ Map Operator Tree:
+ TableScan
+ alias: b
+ filterExpr: key is not null (type: boolean)
+ Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator
+ predicate: key is not null (type: boolean)
+ Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: key (type: int)
+ outputColumnNames: _col0
+ Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+ Group By Operator
+ keys: _col0 (type: int)
+ minReductionHashAggr: 0.5454545
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: int)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+ Execution mode: llap
+ LLAP IO: all inputs
+ Reducer 2
+ Execution mode: llap
+ Reduce Operator Tree:
+ Merge Join Operator
+ condition map:
+ Anti Join 0 to 1
+ keys:
+ 0 _col0 (type: int)
+ 1 _col0 (type: int)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 4 Data size: 372 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: int), _col1 (type: string)
+ null sort order: zz
+ sort order: ++
+ Statistics: Num rows: 4 Data size: 372 Basic stats: COMPLETE Column stats: COMPLETE
+ Reducer 3
+ Execution mode: llap
+ Reduce Operator Tree:
+ Select Operator
+ expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 4 Data size: 372 Basic stats: COMPLETE Column stats: COMPLETE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 4 Data size: 372 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: explain cbo select a.key, a.value from t1_n55 a where not exists (select 1 from t2_n33 b where a.key=b.key) sort by a.key, a.value
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo select a.key, a.value from t1_n55 a where not exists (select 1 from t2_n33 b where a.key=b.key) sort by a.key, a.value
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+CBO PLAN:
+HiveSortExchange(distribution=[any], collation=[[0, 1]])
+ HiveProject(key=[$0], value=[$1])
+ HiveAntiJoin(condition=[=($0, $3)], joinType=[anti])
+ HiveProject(key=[$0], value=[$1])
+ HiveTableScan(table=[[default, t1_n55]], table:alias=[a])
+ HiveProject(literalTrue=[true], key=[$0])
+ HiveFilter(condition=[IS NOT NULL($0)])
+ HiveTableScan(table=[[default, t2_n33]], table:alias=[b])
+
+PREHOOK: query: select a.key, a.value from t1_n55 a where not exists (select 1 from t2_n33 b where a.key=b.key) sort by a.key, a.value
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+POSTHOOK: query: select a.key, a.value from t1_n55 a where not exists (select 1 from t2_n33 b where a.key=b.key) sort by a.key, a.value
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+2 val_2
+5 val_5
+5 val_5
+5 val_5
+9 val_9
+PREHOOK: query: explain select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key where b.key is null
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+POSTHOOK: query: explain select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key where b.key is null
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+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), Map 3 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: a
+ Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: key (type: int), value (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: int)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col1 (type: string)
+ Execution mode: llap
+ LLAP IO: all inputs
+ Map 3
+ Map Operator Tree:
+ TableScan
+ alias: b
+ filterExpr: key is not null (type: boolean)
+ Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator
+ predicate: key is not null (type: boolean)
+ Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: key (type: int)
+ outputColumnNames: _col0
+ Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+ Group By Operator
+ keys: _col0 (type: int)
+ minReductionHashAggr: 0.5454545
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: int)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+ Execution mode: llap
+ LLAP IO: all inputs
+ Reducer 2
+ Execution mode: llap
+ Reduce Operator Tree:
+ Merge Join Operator
+ condition map:
+ Anti Join 0 to 1
+ keys:
+ 0 _col0 (type: int)
+ 1 _col0 (type: int)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 4 Data size: 372 Basic stats: COMPLETE Column stats: COMPLETE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 4 Data size: 372 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: explain cbo select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key where b.key is null
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key where b.key is null
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+CBO PLAN:
+HiveAntiJoin(condition=[=($0, $2)], joinType=[anti])
+ HiveProject(key=[$0], value=[$1])
+ HiveTableScan(table=[[default, t1_n55]], table:alias=[a])
+ HiveProject(key=[$0])
+ HiveFilter(condition=[IS NOT NULL($0)])
+ HiveTableScan(table=[[default, t2_n33]], table:alias=[b])
+
+PREHOOK: query: select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key where b.key is null
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+POSTHOOK: query: select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key where b.key is null
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+2 val_2
+5 val_5
+5 val_5
+5 val_5
+9 val_9
+PREHOOK: query: explain select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key join t3_n12 c on a.key=c.key where b.key is null sort by a.key, a.value
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+PREHOOK: Input: default@t3_n12
+#### A masked pattern was here ####
+POSTHOOK: query: explain select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key join t3_n12 c on a.key=c.key where b.key is null sort by a.key, a.value
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+POSTHOOK: Input: default@t3_n12
+#### A masked pattern was here ####
+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), Map 5 (SIMPLE_EDGE)
+ Reducer 3 <- Map 6 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+ Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: a
+ filterExpr: key is not null (type: boolean)
+ Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator
+ predicate: key is not null (type: boolean)
+ Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: key (type: int), value (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: int)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col1 (type: string)
+ Execution mode: llap
+ LLAP IO: all inputs
+ Map 5
+ Map Operator Tree:
+ TableScan
+ alias: b
+ filterExpr: key is not null (type: boolean)
+ Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator
+ predicate: key is not null (type: boolean)
+ Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: key (type: int)
+ outputColumnNames: _col0
+ Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+ Group By Operator
+ keys: _col0 (type: int)
+ minReductionHashAggr: 0.5454545
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: int)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+ Execution mode: llap
+ LLAP IO: all inputs
+ Map 6
+ Map Operator Tree:
+ TableScan
+ alias: c
+ filterExpr: key is not null (type: boolean)
+ Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator
+ predicate: key is not null (type: boolean)
+ Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: key (type: int)
+ outputColumnNames: _col0
+ Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: int)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
+ Execution mode: llap
+ LLAP IO: all inputs
+ Reducer 2
+ Execution mode: llap
+ Reduce Operator Tree:
+ Merge Join Operator
+ condition map:
+ Anti Join 0 to 1
+ keys:
+ 0 _col0 (type: int)
+ 1 _col0 (type: int)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 4 Data size: 372 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: int)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 4 Data size: 372 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col1 (type: string)
+ Reducer 3
+ Execution mode: llap
+ Reduce Operator Tree:
+ Merge Join Operator
+ condition map:
+ Inner Join 0 to 1
+ keys:
+ 0 _col0 (type: int)
+ 1 _col0 (type: int)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 8 Data size: 744 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: int), _col1 (type: string)
+ null sort order: zz
+ sort order: ++
+ Statistics: Num rows: 8 Data size: 744 Basic stats: COMPLETE Column stats: COMPLETE
+ Reducer 4
+ Execution mode: llap
+ Reduce Operator Tree:
+ Select Operator
+ expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 8 Data size: 744 Basic stats: COMPLETE Column stats: COMPLETE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 8 Data size: 744 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: explain cbo select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key join t3_n12 c on a.key=c.key where b.key is null sort by a.key, a.value
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+PREHOOK: Input: default@t3_n12
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key join t3_n12 c on a.key=c.key where b.key is null sort by a.key, a.value
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+POSTHOOK: Input: default@t3_n12
+#### A masked pattern was here ####
+CBO PLAN:
+HiveSortExchange(distribution=[any], collation=[[0, 1]])
+ HiveProject(key=[$1], value=[$2])
+ HiveJoin(condition=[=($1, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+ HiveProject(key=[$0])
+ HiveFilter(condition=[IS NOT NULL($0)])
+ HiveTableScan(table=[[default, t3_n12]], table:alias=[c])
+ HiveAntiJoin(condition=[=($0, $2)], joinType=[anti])
+ HiveProject(key=[$0], value=[$1])
+ HiveFilter(condition=[IS NOT NULL($0)])
+ HiveTableScan(table=[[default, t1_n55]], table:alias=[a])
+ HiveProject(key=[$0])
+ HiveFilter(condition=[IS NOT NULL($0)])
+ HiveTableScan(table=[[default, t2_n33]], table:alias=[b])
+
+PREHOOK: query: select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key join t3_n12 c on a.key=c.key where b.key is null sort by a.key, a.value
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+PREHOOK: Input: default@t3_n12
+#### A masked pattern was here ####
+POSTHOOK: query: select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key join t3_n12 c on a.key=c.key where b.key is null sort by a.key, a.value
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+POSTHOOK: Input: default@t3_n12
+#### A masked pattern was here ####
+2 val_2
+5 val_5
+5 val_5
+5 val_5
+5 val_5
+5 val_5
+5 val_5
+5 val_5
+5 val_5
+5 val_5
+9 val_9
+PREHOOK: query: explain select a.key, a.value from t1_n55 a where not exists (select 1 from t2_n33 b where a.key=b.key) sort by a.key, a.value
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+POSTHOOK: query: explain select a.key, a.value from t1_n55 a where not exists (select 1 from t2_n33 b where a.key=b.key) sort by a.key, a.value
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+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:
+ Map 1 <- Map 3 (BROADCAST_EDGE)
+ Reducer 2 <- Map 1 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: a
+ Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: key (type: int), value (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+ Map Join Operator
+ condition map:
+ Anti Join 0 to 1
+ keys:
+ 0 _col0 (type: int)
+ 1 _col0 (type: int)
+ outputColumnNames: _col0, _col1
+ input vertices:
+ 1 Map 3
+ Statistics: Num rows: 4 Data size: 372 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: int), _col1 (type: string)
+ null sort order: zz
+ sort order: ++
+ Statistics: Num rows: 4 Data size: 372 Basic stats: COMPLETE Column stats: COMPLETE
+ Execution mode: vectorized, llap
+ LLAP IO: all inputs
+ Map 3
+ Map Operator Tree:
+ TableScan
+ alias: b
+ filterExpr: key is not null (type: boolean)
+ Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator
+ predicate: key is not null (type: boolean)
+ Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: key (type: int)
+ outputColumnNames: _col0
+ Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+ Group By Operator
+ keys: _col0 (type: int)
+ minReductionHashAggr: 0.5454545
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: int)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+ Execution mode: vectorized, llap
+ LLAP IO: all inputs
+ Reducer 2
+ Execution mode: vectorized, llap
+ Reduce Operator Tree:
+ Select Operator
+ expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 4 Data size: 372 Basic stats: COMPLETE Column stats: COMPLETE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 4 Data size: 372 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: explain cbo select a.key, a.value from t1_n55 a where not exists (select 1 from t2_n33 b where a.key=b.key) sort by a.key, a.value
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo select a.key, a.value from t1_n55 a where not exists (select 1 from t2_n33 b where a.key=b.key) sort by a.key, a.value
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+CBO PLAN:
+HiveSortExchange(distribution=[any], collation=[[0, 1]])
+ HiveProject(key=[$0], value=[$1])
+ HiveAntiJoin(condition=[=($0, $3)], joinType=[anti])
+ HiveProject(key=[$0], value=[$1])
+ HiveTableScan(table=[[default, t1_n55]], table:alias=[a])
+ HiveProject(literalTrue=[true], key=[$0])
+ HiveFilter(condition=[IS NOT NULL($0)])
+ HiveTableScan(table=[[default, t2_n33]], table:alias=[b])
+
+PREHOOK: query: select a.key, a.value from t1_n55 a where not exists (select 1 from t2_n33 b where a.key=b.key) sort by a.key, a.value
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+POSTHOOK: query: select a.key, a.value from t1_n55 a where not exists (select 1 from t2_n33 b where a.key=b.key) sort by a.key, a.value
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+2 val_2
+5 val_5
+5 val_5
+5 val_5
+9 val_9
+PREHOOK: query: explain select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key where b.key is null
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+POSTHOOK: query: explain select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key where b.key is null
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+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:
+ Map 1 <- Map 2 (BROADCAST_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: a
+ Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: key (type: int), value (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+ Map Join Operator
+ condition map:
+ Anti Join 0 to 1
+ keys:
+ 0 _col0 (type: int)
+ 1 _col0 (type: int)
+ outputColumnNames: _col0, _col1
+ input vertices:
+ 1 Map 2
+ Statistics: Num rows: 4 Data size: 372 Basic stats: COMPLETE Column stats: COMPLETE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 4 Data size: 372 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
+ Execution mode: vectorized, llap
+ LLAP IO: all inputs
+ Map 2
+ Map Operator Tree:
+ TableScan
+ alias: b
+ filterExpr: key is not null (type: boolean)
+ Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator
+ predicate: key is not null (type: boolean)
+ Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: key (type: int)
+ outputColumnNames: _col0
+ Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+ Group By Operator
+ keys: _col0 (type: int)
+ minReductionHashAggr: 0.5454545
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: int)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+ Execution mode: vectorized, llap
+ LLAP IO: all inputs
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: explain cbo select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key where b.key is null
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key where b.key is null
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+CBO PLAN:
+HiveAntiJoin(condition=[=($0, $2)], joinType=[anti])
+ HiveProject(key=[$0], value=[$1])
+ HiveTableScan(table=[[default, t1_n55]], table:alias=[a])
+ HiveProject(key=[$0])
+ HiveFilter(condition=[IS NOT NULL($0)])
+ HiveTableScan(table=[[default, t2_n33]], table:alias=[b])
+
+PREHOOK: query: select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key where b.key is null
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+POSTHOOK: query: select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key where b.key is null
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+2 val_2
+5 val_5
+5 val_5
+5 val_5
+9 val_9
+PREHOOK: query: explain select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key join t3_n12 c on a.key=c.key where b.key is null sort by a.key, a.value
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+PREHOOK: Input: default@t3_n12
+#### A masked pattern was here ####
+POSTHOOK: query: explain select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key join t3_n12 c on a.key=c.key where b.key is null sort by a.key, a.value
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+POSTHOOK: Input: default@t3_n12
+#### A masked pattern was here ####
+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:
+ Map 1 <- Map 2 (BROADCAST_EDGE)
+ Map 3 <- Map 1 (BROADCAST_EDGE)
+ Reducer 4 <- Map 3 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: a
+ filterExpr: key is not null (type: boolean)
+ Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator
+ predicate: key is not null (type: boolean)
+ Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: key (type: int), value (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+ Map Join Operator
+ condition map:
+ Anti Join 0 to 1
+ keys:
+ 0 _col0 (type: int)
+ 1 _col0 (type: int)
+ outputColumnNames: _col0, _col1
+ input vertices:
+ 1 Map 2
+ Statistics: Num rows: 4 Data size: 372 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: int)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 4 Data size: 372 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col1 (type: string)
+ Execution mode: vectorized, llap
+ LLAP IO: all inputs
+ Map 2
+ Map Operator Tree:
+ TableScan
+ alias: b
+ filterExpr: key is not null (type: boolean)
+ Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator
+ predicate: key is not null (type: boolean)
+ Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: key (type: int)
+ outputColumnNames: _col0
+ Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+ Group By Operator
+ keys: _col0 (type: int)
+ minReductionHashAggr: 0.5454545
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: int)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+ Execution mode: vectorized, llap
+ LLAP IO: all inputs
+ Map 3
+ Map Operator Tree:
+ TableScan
+ alias: c
+ filterExpr: key is not null (type: boolean)
+ probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_46_container, bigKeyColName:key, smallTablePos:0, keyRatio:0.09090909090909091
+ Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator
+ predicate: key is not null (type: boolean)
+ Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: key (type: int)
+ outputColumnNames: _col0
+ Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ keys:
+ 0 _col0 (type: int)
+ 1 _col0 (type: int)
+ outputColumnNames: _col0, _col1
+ input vertices:
+ 0 Map 1
+ Statistics: Num rows: 8 Data size: 744 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: int), _col1 (type: string)
+ null sort order: zz
+ sort order: ++
+ Statistics: Num rows: 8 Data size: 744 Basic stats: COMPLETE Column stats: COMPLETE
+ Execution mode: vectorized, llap
+ LLAP IO: all inputs
+ Reducer 4
+ Execution mode: vectorized, llap
+ Reduce Operator Tree:
+ Select Operator
+ expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 8 Data size: 744 Basic stats: COMPLETE Column stats: COMPLETE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 8 Data size: 744 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: explain cbo select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key join t3_n12 c on a.key=c.key where b.key is null sort by a.key, a.value
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+PREHOOK: Input: default@t3_n12
+#### A masked pattern was here ####
+POSTHOOK: query: explain cbo select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key join t3_n12 c on a.key=c.key where b.key is null sort by a.key, a.value
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+POSTHOOK: Input: default@t3_n12
+#### A masked pattern was here ####
+CBO PLAN:
+HiveSortExchange(distribution=[any], collation=[[0, 1]])
+ HiveProject(key=[$1], value=[$2])
+ HiveJoin(condition=[=($1, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+ HiveProject(key=[$0])
+ HiveFilter(condition=[IS NOT NULL($0)])
+ HiveTableScan(table=[[default, t3_n12]], table:alias=[c])
+ HiveAntiJoin(condition=[=($0, $2)], joinType=[anti])
+ HiveProject(key=[$0], value=[$1])
+ HiveFilter(condition=[IS NOT NULL($0)])
+ HiveTableScan(table=[[default, t1_n55]], table:alias=[a])
+ HiveProject(key=[$0])
+ HiveFilter(condition=[IS NOT NULL($0)])
+ HiveTableScan(table=[[default, t2_n33]], table:alias=[b])
+
+PREHOOK: query: select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key join t3_n12 c on a.key=c.key where b.key is null sort by a.key, a.value
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+PREHOOK: Input: default@t3_n12
+#### A masked pattern was here ####
+POSTHOOK: query: select a.key, a.value from t1_n55 a left join t2_n33 b on a.key=b.key join t3_n12 c on a.key=c.key where b.key is null sort by a.key, a.value
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+POSTHOOK: Input: default@t3_n12
+#### A masked pattern was here ####
+2 val_2
+5 val_5
+5 val_5
+5 val_5
+5 val_5
+5 val_5
+5 val_5
+5 val_5
+5 val_5
+5 val_5
+9 val_9
+Warning: Map Join MAPJOIN[15][bigTable=?] in task 'Map 1' is a cross product
+PREHOOK: query: explain select a.key from t1_n55 a left join t2_n33 b on a.key = b.key where b.key is null and b.value > 'val_1'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+POSTHOOK: query: explain select a.key from t1_n55 a left join t2_n33 b on a.key = b.key where b.key is null and b.value > 'val_1'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+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:
+ Map 1 <- Map 2 (BROADCAST_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: a
+ filterExpr: (key is null and key is not null) (type: boolean)
+ Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator
+ predicate: (key is null and key is not null) (type: boolean)
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ Limit
+ Number of rows: 0
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ keys:
+ 0
+ 1
+ input vertices:
+ 1 Map 2
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: null (type: int)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 4 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
+ Execution mode: vectorized, llap
+ LLAP IO: all inputs
+ Map 2
+ Map Operator Tree:
+ TableScan
+ alias: b
+ filterExpr: (key is null and (value > 'val_1') and key is not null) (type: boolean)
+ Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator
+ predicate: (key is null and (value > 'val_1') and key is not null) (type: boolean)
+ Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ null sort order:
+ sort order:
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ Execution mode: vectorized, llap
+ LLAP IO: all inputs
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: explain select a.key from t1_n55 a left join t2_n33 b on a.key = b.key where b.key is null and a.key > 100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+POSTHOOK: query: explain select a.key from t1_n55 a left join t2_n33 b on a.key = b.key where b.key is null and a.key > 100
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+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:
+ Map 1 <- Map 2 (BROADCAST_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: a
+ filterExpr: (key > 100) (type: boolean)
+ Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator
+ predicate: (key > 100) (type: boolean)
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: key (type: int)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ Map Join Operator
+ condition map:
+ Anti Join 0 to 1
+ keys:
+ 0 _col0 (type: int)
+ 1 _col0 (type: int)
+ outputColumnNames: _col0
+ input vertices:
+ 1 Map 2
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 4 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
+ Execution mode: vectorized, llap
+ LLAP IO: all inputs
+ Map 2
+ Map Operator Tree:
+ TableScan
+ alias: b
+ filterExpr: (key > 100) (type: boolean)
+ Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator
+ predicate: (key > 100) (type: boolean)
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: key (type: int)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ Group By Operator
+ keys: _col0 (type: int)
+ minReductionHashAggr: 0.0
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: int)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ Execution mode: vectorized, llap
+ LLAP IO: all inputs
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: explain select a.key from t1_n55 a left join t2_n33 b on a.key = b.key where b.key is null and rand(length(b.value)) > 100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+POSTHOOK: query: explain select a.key from t1_n55 a left join t2_n33 b on a.key = b.key where b.key is null and rand(length(b.value)) > 100
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+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:
+ Map 1 <- Map 2 (BROADCAST_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: a
+ Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: key (type: int)
+ outputColumnNames: _col0
+ Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+ Map Join Operator
+ condition map:
+ Left Outer Join 0 to 1
+ keys:
+ 0 _col0 (type: int)
+ 1 _col0 (type: int)
+ outputColumnNames: _col0, _col1, _col2
+ input vertices:
+ 1 Map 2
+ Statistics: Num rows: 17 Data size: 1649 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator
+ predicate: (_col1 is null and (rand(length(_col2)) > 100.0D)) (type: boolean)
+ Statistics: Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: _col0 (type: int)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 4 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
+ Execution mode: vectorized, llap
+ LLAP IO: all inputs
+ Map 2
+ Map Operator Tree:
+ TableScan
+ alias: b
+ filterExpr: key is not null (type: boolean)
+ Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator
+ predicate: key is not null (type: boolean)
+ Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: key (type: int), value (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: int)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col1 (type: string)
+ Execution mode: vectorized, llap
+ LLAP IO: all inputs
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: explain select a.key from t1_n55 a left join t2_n33 b on a.key = b.key where b.key is null and rand(a.key) > 100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+POSTHOOK: query: explain select a.key from t1_n55 a left join t2_n33 b on a.key = b.key where b.key is null and rand(a.key) > 100
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+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:
+ Map 1 <- Map 2 (BROADCAST_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: a
+ Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: key (type: int)
+ outputColumnNames: _col0
+ Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+ Map Join Operator
+ condition map:
+ Anti Join 0 to 1
+ keys:
+ 0 _col0 (type: int)
+ 1 _col0 (type: int)
+ outputColumnNames: _col0
+ input vertices:
+ 1 Map 2
+ Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator
+ predicate: (rand(_col0) > 100.0D) (type: boolean)
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 4 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
+ Execution mode: vectorized, llap
+ LLAP IO: all inputs
+ Map 2
+ Map Operator Tree:
+ TableScan
+ alias: b
+ filterExpr: key is not null (type: boolean)
+ Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator
+ predicate: key is not null (type: boolean)
+ Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: key (type: int)
+ outputColumnNames: _col0
+ Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+ Group By Operator
+ keys: _col0 (type: int)
+ minReductionHashAggr: 0.5454545
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: int)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+ Execution mode: vectorized, llap
+ LLAP IO: all inputs
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: explain select a.key from t1_n55 a left join t2_n33 b on a.key = b.key where b.key is null and rand(length(b.value)) < 100 and rand(length(b.value)) > 100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+POSTHOOK: query: explain select a.key from t1_n55 a left join t2_n33 b on a.key = b.key where b.key is null and rand(length(b.value)) < 100 and rand(length(b.value)) > 100
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+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:
+ Map 1 <- Map 2 (BROADCAST_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: a
+ Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: key (type: int)
+ outputColumnNames: _col0
+ Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+ Map Join Operator
+ condition map:
+ Left Outer Join 0 to 1
+ keys:
+ 0 _col0 (type: int)
+ 1 _col0 (type: int)
+ outputColumnNames: _col0, _col1, _col2
+ input vertices:
+ 1 Map 2
+ Statistics: Num rows: 17 Data size: 1649 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator
+ predicate: (_col1 is null and (rand(length(_col2)) < 100.0D) and (rand(length(_col2)) > 100.0D)) (type: boolean)
+ Statistics: Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: _col0 (type: int)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 4 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
+ Execution mode: vectorized, llap
+ LLAP IO: all inputs
+ Map 2
+ Map Operator Tree:
+ TableScan
+ alias: b
+ filterExpr: key is not null (type: boolean)
+ Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator
+ predicate: key is not null (type: boolean)
+ Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: key (type: int), value (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: int)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col1 (type: string)
+ Execution mode: vectorized, llap
+ LLAP IO: all inputs
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: explain select a.key from t1_n55 a left join t2_n33 b on a.key = b.key where b.key is null and rand(a.key) > 100 and rand(length(a.value)) > 100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+POSTHOOK: query: explain select a.key from t1_n55 a left join t2_n33 b on a.key = b.key where b.key is null and rand(a.key) > 100 and rand(length(a.value)) > 100
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+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:
+ Map 1 <- Map 2 (BROADCAST_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: a
+ Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: key (type: int), value (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+ Map Join Operator
+ condition map:
+ Anti Join 0 to 1
+ keys:
+ 0 _col0 (type: int)
+ 1 _col0 (type: int)
+ outputColumnNames: _col0, _col1
+ input vertices:
+ 1 Map 2
+ Statistics: Num rows: 4 Data size: 372 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator
+ predicate: ((rand(_col0) > 100.0D) and (rand(length(_col1)) > 100.0D)) (type: boolean)
+ Statistics: Num rows: 1 Data size: 93 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: _col0 (type: int)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 4 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
+ Execution mode: vectorized, llap
+ LLAP IO: all inputs
+ Map 2
+ Map Operator Tree:
+ TableScan
+ alias: b
+ filterExpr: key is not null (type: boolean)
+ Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator
+ predicate: key is not null (type: boolean)
+ Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: key (type: int)
+ outputColumnNames: _col0
+ Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+ Group By Operator
+ keys: _col0 (type: int)
+ minReductionHashAggr: 0.5454545
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: int)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
+ Execution mode: vectorized, llap
+ LLAP IO: all inputs
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: explain select a.key from t1_n55 a left join t2_n33 b on a.key = b.key where b.key is null or b.value is null
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+POSTHOOK: query: explain select a.key from t1_n55 a left join t2_n33 b on a.key = b.key where b.key is null or b.value is null
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+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:
+ Map 1 <- Map 2 (BROADCAST_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: a
+ Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: key (type: int)
+ outputColumnNames: _col0
+ Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+ Map Join Operator
+ condition map:
+ Left Outer Join 0 to 1
+ keys:
+ 0 _col0 (type: int)
+ 1 _col0 (type: int)
+ outputColumnNames: _col0, _col1, _col2
+ input vertices:
+ 1 Map 2
+ Statistics: Num rows: 17 Data size: 1649 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator
+ predicate: (_col1 is null or _col2 is null) (type: boolean)
+ Statistics: Num rows: 1 Data size: 97 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: _col0 (type: int)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 4 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
+ Execution mode: vectorized, llap
+ LLAP IO: all inputs
+ Map 2
+ Map Operator Tree:
+ TableScan
+ alias: b
+ filterExpr: key is not null (type: boolean)
+ Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator
+ predicate: key is not null (type: boolean)
+ Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: key (type: int), value (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: int)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col1 (type: string)
+ Execution mode: vectorized, llap
+ LLAP IO: all inputs
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: explain select a.key from t1_n55 a left join t2_n33 b on a.key = b.key where b.key is null or b.value = 'val_1'
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1_n55
+PREHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+POSTHOOK: query: explain select a.key from t1_n55 a left join t2_n33 b on a.key = b.key where b.key is null or b.value = 'val_1'
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1_n55
+POSTHOOK: Input: default@t2_n33
+#### A masked pattern was here ####
+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:
+ Map 1 <- Map 2 (BROADCAST_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: a
+ Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: key (type: int)
+ outputColumnNames: _col0
+ Statistics: Num rows: 11 Data size: 44 Basic stats: COMPLETE Column stats: COMPLETE
+ Map Join Operator
+ condition map:
+ Left Outer Join 0 to 1
+ keys:
+ 0 _col0 (type: int)
+ 1 _col0 (type: int)
+ outputColumnNames: _col0, _col1, _col2
+ input vertices:
+ 1 Map 2
+ Statistics: Num rows: 17 Data size: 1649 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator
+ predicate: (_col1 is null or (_col2 = 'val_1')) (type: boolean)
+ Statistics: Num rows: 2 Data size: 194 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: _col0 (type: int)
+ outputColumnNames: _col0
+ Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 2 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
+ Execution mode: vectorized, llap
+ LLAP IO: all inputs
+ Map 2
+ Map Operator Tree:
+ TableScan
+ alias: b
+ filterExpr: key is not null (type: boolean)
+ Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator
+ predicate: key is not null (type: boolean)
+ Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: key (type: int), value (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: int)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col1 (type: string)
+ Execution mode: vectorized, llap
+ LLAP IO: all inputs
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
diff --git a/ql/src/test/results/clientpositive/llap/explainuser_1.q.out b/ql/src/test/results/clientpositive/llap/explainuser_1.q.out
index 548fc65..cce2f97 100644
--- a/ql/src/test/results/clientpositive/llap/explainuser_1.q.out
+++ b/ql/src/test/results/clientpositive/llap/explainuser_1.q.out
@@ -2092,39 +2092,33 @@ Stage-0
limit:-1
Stage-1
Reducer 2 llap
- File Output Operator [FS_19]
- Select Operator [SEL_18] (rows=500 width=178)
- Output:["_col0","_col1"]
- Filter Operator [FIL_17] (rows=500 width=178)
- predicate:_col2 is null
- Merge Join Operator [MERGEJOIN_27] (rows=566 width=178)
- Conds:RS_14._col1=RS_15._col1(Left Outer),Output:["_col0","_col1","_col2"]
- <-Map 1 [SIMPLE_EDGE] llap
- SHUFFLE [RS_14]
- PartitionCols:_col1
- Select Operator [SEL_1] (rows=500 width=178)
- Output:["_col0","_col1"]
- TableScan [TS_0] (rows=500 width=178)
- default@src_cbo,b,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
- <-Reducer 3 [SIMPLE_EDGE] llap
- SHUFFLE [RS_15]
- PartitionCols:_col1
- Select Operator [SEL_13] (rows=41 width=95)
- Output:["_col0","_col1"]
- Group By Operator [GBY_12] (rows=41 width=91)
- Output:["_col0"],keys:_col1
- Select Operator [SEL_8] (rows=83 width=178)
- Output:["_col1"]
- Group By Operator [GBY_7] (rows=83 width=178)
- Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
- <-Map 1 [SIMPLE_EDGE] llap
- SHUFFLE [RS_6]
- PartitionCols:_col0
- Group By Operator [GBY_5] (rows=83 width=178)
- Output:["_col0","_col1"],keys:value, key
- Filter Operator [FIL_21] (rows=166 width=178)
- predicate:(value > 'val_2')
- Please refer to the previous TableScan [TS_0]
+ File Output Operator [FS_15]
+ Merge Join Operator [MERGEJOIN_23] (rows=434 width=178)
+ Conds:RS_11._col1=RS_12._col0(Anti),Output:["_col0","_col1"]
+ <-Map 1 [SIMPLE_EDGE] llap
+ SHUFFLE [RS_11]
+ PartitionCols:_col1
+ Select Operator [SEL_1] (rows=500 width=178)
+ Output:["_col0","_col1"]
+ TableScan [TS_0] (rows=500 width=178)
+ default@src_cbo,b,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
+ <-Reducer 3 [SIMPLE_EDGE] llap
+ SHUFFLE [RS_12]
+ PartitionCols:_col0
+ Group By Operator [GBY_10] (rows=41 width=91)
+ Output:["_col0"],keys:_col0
+ Select Operator [SEL_8] (rows=83 width=91)
+ Output:["_col0"]
+ Group By Operator [GBY_7] (rows=83 width=178)
+ Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
+ <-Map 1 [SIMPLE_EDGE] llap
+ SHUFFLE [RS_6]
+ PartitionCols:_col0, _col1
+ Group By Operator [GBY_5] (rows=83 width=178)
+ Output:["_col0","_col1"],keys:value, key
+ Filter Operator [FIL_17] (rows=166 width=178)
+ predicate:(value > 'val_2')
+ Please refer to the previous TableScan [TS_0]
PREHOOK: query: explain select *
from src_cbo b
@@ -2151,45 +2145,41 @@ POSTHOOK: Input: default@src_cbo
Plan optimized by CBO.
Vertex dependency in root stage
-Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 3 (SIMPLE_EDGE)
+Reducer 2 <- Map 1 (SIMPLE_EDGE)
+Reducer 3 <- Map 1 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
Stage-0
Fetch Operator
limit:-1
Stage-1
- Reducer 2 llap
- File Output Operator [FS_18]
- Select Operator [SEL_17] (rows=250 width=178)
- Output:["_col0","_col1"]
- Filter Operator [FIL_16] (rows=250 width=179)
- predicate:_col2 is null
- Merge Join Operator [MERGEJOIN_31] (rows=333 width=179)
- Conds:GBY_4._col0, _col1=DUMMY_STORE_32._col1, _col2(Left Outer),Output:["_col0","_col1","_col2"]
- <-Dummy Store [DUMMY_STORE_32]
- Select Operator [SEL_12] (rows=83 width=182)
- Output:["_col0","_col1","_col2"]
- Group By Operator [GBY_11] (rows=83 width=178)
- Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
- <-Group By Operator [GBY_4] (rows=250 width=178)
- Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
- <-Map 1 [SIMPLE_EDGE] llap
- SHUFFLE [RS_3]
- PartitionCols:_col0, _col1
- Group By Operator [GBY_2] (rows=250 width=178)
- Output:["_col0","_col1"],keys:key, value
- Select Operator [SEL_1] (rows=500 width=178)
- Output:["key","value"]
- TableScan [TS_0] (rows=500 width=178)
- default@src_cbo,b,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
- <-Map 3 [SIMPLE_EDGE] llap
- SHUFFLE [RS_10]
- PartitionCols:_col0, _col1
- Group By Operator [GBY_9] (rows=83 width=178)
- Output:["_col0","_col1"],keys:key, value
- Filter Operator [FIL_20] (rows=166 width=178)
- predicate:((value > 'val_12') and key is not null)
- TableScan [TS_6] (rows=500 width=178)
- default@src_cbo,a,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
+ Reducer 3 llap
+ File Output Operator [FS_15]
+ Merge Join Operator [MERGEJOIN_28] (rows=167 width=178)
+ Conds:RS_11._col0, _col1=RS_12._col0, _col1(Anti),Output:["_col0","_col1"]
+ <-Map 1 [SIMPLE_EDGE] llap
+ SHUFFLE [RS_12]
+ PartitionCols:_col0, _col1
+ Group By Operator [GBY_10] (rows=83 width=178)
+ Output:["_col0","_col1"],keys:_col0, _col1
+ Select Operator [SEL_8] (rows=166 width=178)
+ Output:["_col0","_col1"]
+ Filter Operator [FIL_17] (rows=166 width=178)
+ predicate:((value > 'val_12') and key is not null)
+ TableScan [TS_0] (rows=500 width=178)
+ default@src_cbo,b,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
+ <-Reducer 2 [SIMPLE_EDGE] llap
+ SHUFFLE [RS_11]
+ PartitionCols:_col0, _col1
+ Group By Operator [GBY_4] (rows=250 width=178)
+ Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
+ <-Map 1 [SIMPLE_EDGE] llap
+ SHUFFLE [RS_3]
+ PartitionCols:_col0, _col1
+ Group By Operator [GBY_2] (rows=250 width=178)
+ Output:["_col0","_col1"],keys:key, value
+ Select Operator [SEL_1] (rows=500 width=178)
+ Output:["key","value"]
+ Please refer to the previous TableScan [TS_0]
PREHOOK: query: create view cv1_n5 as
select *
@@ -6695,3 +6685,297 @@ Stage-0
TableScan [TS_0] (rows=1 width=184)
default@t1_n119,k,Tbl:COMPLETE,Col:NONE,Output:["key"]
+PREHOOK: query: explain select *
+from src_cbo b
+where not exists
+ (select distinct a.key
+ from src_cbo a
+ where b.value = a.value and a.value > 'val_2'
+ )
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src_cbo
+#### A masked pattern was here ####
+POSTHOOK: query: explain select *
+from src_cbo b
+where not exists
+ (select distinct a.key
+ from src_cbo a
+ where b.value = a.value and a.value > 'val_2'
+ )
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src_cbo
+#### A masked pattern was here ####
+Plan optimized by CBO.
+
+Vertex dependency in root stage
+Map 1 <- Reducer 6 (BROADCAST_EDGE)
+Reducer 3 <- Map 2 (SIMPLE_EDGE)
+Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
+Reducer 5 <- Reducer 4 (SIMPLE_EDGE)
+Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
+
+Stage-0
+ Fetch Operator
+ limit:-1
+ Stage-1
+ Map 1 llap
+ File Output Operator [FS_21]
+ Select Operator [SEL_20] (rows=500 width=178)
+ Output:["_col0","_col1"]
+ Filter Operator [FIL_19] (rows=500 width=179)
+ predicate:_col2 is null
+ Map Join Operator [MAPJOIN_29] (rows=666 width=179)
+ Conds:SEL_1._col1=RS_17._col1(Left Outer),Output:["_col0","_col1","_col2"]
+ <-Reducer 6 [BROADCAST_EDGE] llap
+ BROADCAST [RS_17]
+ PartitionCols:_col1
+ Select Operator [SEL_15] (rows=102 width=95)
+ Output:["_col0","_col1"]
+ Group By Operator [GBY_14] (rows=102 width=91)
+ Output:["_col0"],keys:KEY._col0
+ <-Reducer 5 [SIMPLE_EDGE] llap
+ SHUFFLE [RS_13]
+ PartitionCols:_col0
+ Group By Operator [GBY_12] (rows=166 width=91)
+ Output:["_col0"],keys:KEY._col0
+ <-Reducer 4 [SIMPLE_EDGE] llap
+ SHUFFLE [RS_11]
+ PartitionCols:rand()
+ Select Operator [SEL_9] (rows=166 width=178)
+ Output:["_col1"]
+ Group By Operator [GBY_8] (rows=166 width=178)
+ Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
+ <-Reducer 3 [SIMPLE_EDGE] llap
+ SHUFFLE [RS_7]
+ PartitionCols:_col0, _col1
+ Group By Operator [GBY_6] (rows=166 width=178)
+ Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
+ <-Map 2 [SIMPLE_EDGE] llap
+ SHUFFLE [RS_5]
+ PartitionCols:rand()
+ Filter Operator [FIL_23] (rows=166 width=178)
+ predicate:(value > 'val_2')
+ TableScan [TS_2] (rows=500 width=178)
+ default@src_cbo,a,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
+ <-Select Operator [SEL_1] (rows=500 width=178)
+ Output:["_col0","_col1"]
+ TableScan [TS_0] (rows=500 width=178)
+ default@src_cbo,b,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
+
+PREHOOK: query: explain select *
+from src_cbo b
+group by key, value
+having not exists
+ (select a.key
+ from src_cbo a
+ where b.value = a.value and a.key = b.key and a.value > 'val_12'
+ )
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src_cbo
+#### A masked pattern was here ####
+POSTHOOK: query: explain select *
+from src_cbo b
+group by key, value
+having not exists
+ (select a.key
+ from src_cbo a
+ where b.value = a.value and a.key = b.key and a.value > 'val_12'
+ )
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src_cbo
+#### A masked pattern was here ####
+Plan optimized by CBO.
+
+Vertex dependency in root stage
+Reducer 2 <- Map 1 (SIMPLE_EDGE)
+Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 5 (BROADCAST_EDGE)
+Reducer 4 <- Map 1 (SIMPLE_EDGE)
+Reducer 5 <- Reducer 4 (SIMPLE_EDGE)
+
+Stage-0
+ Fetch Operator
+ limit:-1
+ Stage-1
+ Reducer 3 llap
+ File Output Operator [FS_20]
+ Select Operator [SEL_19] (rows=500 width=178)
+ Output:["_col0","_col1"]
+ Filter Operator [FIL_18] (rows=500 width=179)
+ predicate:_col2 is null
+ Map Join Operator [MAPJOIN_33] (rows=762 width=179)
+ Conds:GBY_5._col0, _col1=RS_16._col1, _col2(Left Outer),Output:["_col0","_col1","_col2"]
+ <-Reducer 5 [BROADCAST_EDGE] llap
+ BROADCAST [RS_16]
+ PartitionCols:_col1, _col2
+ Select Operator [SEL_14] (rows=166 width=182)
+ Output:["_col0","_col1","_col2"]
+ Group By Operator [GBY_13] (rows=166 width=178)
+ Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
+ <-Reducer 4 [SIMPLE_EDGE] llap
+ SHUFFLE [RS_12]
+ PartitionCols:_col0, _col1
+ Group By Operator [GBY_11] (rows=166 width=178)
+ Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
+ <-Map 1 [SIMPLE_EDGE] llap
+ SHUFFLE [RS_10]
+ PartitionCols:rand()
+ Filter Operator [FIL_22] (rows=166 width=178)
+ predicate:((value > 'val_12') and key is not null)
+ TableScan [TS_0] (rows=500 width=178)
+ default@src_cbo,b,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
+ <-Group By Operator [GBY_5] (rows=500 width=178)
+ Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
+ <-Reducer 2 [SIMPLE_EDGE] llap
+ SHUFFLE [RS_4]
+ PartitionCols:_col0, _col1
+ Group By Operator [GBY_3] (rows=500 width=178)
+ Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
+ <-Map 1 [SIMPLE_EDGE] llap
+ SHUFFLE [RS_2]
+ PartitionCols:rand()
+ Select Operator [SEL_1] (rows=500 width=178)
+ Output:["key","value"]
+ Please refer to the previous TableScan [TS_0]
+
+PREHOOK: query: create view cv1_n5_anti as
+select *
+from src_cbo b
+where not exists
+ (select a.key
+ from src_cbo a
+ where b.value = a.value and a.key = b.key and a.value > 'val_9')
+PREHOOK: type: CREATEVIEW
+PREHOOK: Input: default@src_cbo
+PREHOOK: Output: database:default
+PREHOOK: Output: default@cv1_n5_anti
+POSTHOOK: query: create view cv1_n5_anti as
+select *
+from src_cbo b
+where not exists
+ (select a.key
+ from src_cbo a
+ where b.value = a.value and a.key = b.key and a.value > 'val_9')
+POSTHOOK: type: CREATEVIEW
+POSTHOOK: Input: default@src_cbo
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@cv1_n5_anti
+POSTHOOK: Lineage: cv1_n5_anti.key SIMPLE [(src_cbo)b.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: cv1_n5_anti.value SIMPLE [(src_cbo)b.FieldSchema(name:value, type:string, comment:null), ]
+PREHOOK: query: explain select * from cv1_n5_anti
+PREHOOK: type: QUERY
+PREHOOK: Input: default@cv1_n5_anti
+PREHOOK: Input: default@src_cbo
+#### A masked pattern was here ####
+POSTHOOK: query: explain select * from cv1_n5_anti
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@cv1_n5_anti
+POSTHOOK: Input: default@src_cbo
+#### A masked pattern was here ####
+Plan optimized by CBO.
+
+Vertex dependency in root stage
+Map 1 <- Reducer 4 (BROADCAST_EDGE)
+Reducer 3 <- Map 2 (SIMPLE_EDGE)
+Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
+
+Stage-0
+ Fetch Operator
+ limit:-1
+ Stage-1
+ Map 1 llap
+ File Output Operator [FS_15]
+ Select Operator [SEL_14] (rows=500 width=178)
+ Output:["_col0","_col1"]
+ Filter Operator [FIL_13] (rows=500 width=179)
+ predicate:_col2 is null
+ Map Join Operator [MAPJOIN_28] (rows=762 width=179)
+ Conds:SEL_1._col0, _col1=RS_11._col1, _col2(Left Outer),Output:["_col0","_col1","_col2"]
+ <-Reducer 4 [BROADCAST_EDGE] llap
+ BROADCAST [RS_11]
+ PartitionCols:_col1, _col2
+ Select Operator [SEL_9] (rows=166 width=182)
+ Output:["_col0","_col1","_col2"]
+ Group By Operator [GBY_8] (rows=166 width=178)
+ Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
+ <-Reducer 3 [SIMPLE_EDGE] llap
+ SHUFFLE [RS_7]
+ PartitionCols:_col0, _col1
+ Group By Operator [GBY_6] (rows=166 width=178)
+ Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
+ <-Map 2 [SIMPLE_EDGE] llap
+ SHUFFLE [RS_5]
+ PartitionCols:rand()
+ Filter Operator [FIL_17] (rows=166 width=178)
+ predicate:((value > 'val_9') and key is not null)
+ TableScan [TS_2] (rows=500 width=178)
+ default@src_cbo,a,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"],properties:{"insideView":"TRUE"}
+ <-Select Operator [SEL_1] (rows=500 width=178)
+ Output:["_col0","_col1"]
+ TableScan [TS_0] (rows=500 width=178)
+ default@src_cbo,b,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"],properties:{"insideView":"TRUE"}
+
+PREHOOK: query: explain select *
+from (select *
+ from src_cbo b
+ where not exists
+ (select a.key
+ from src_cbo a
+ where b.value = a.value and a.key = b.key and a.value > 'val_9')
+ ) a
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src_cbo
+#### A masked pattern was here ####
+POSTHOOK: query: explain select *
+from (select *
+ from src_cbo b
+ where not exists
+ (select a.key
+ from src_cbo a
+ where b.value = a.value and a.key = b.key and a.value > 'val_9')
+ ) a
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src_cbo
+#### A masked pattern was here ####
+Plan optimized by CBO.
+
+Vertex dependency in root stage
+Map 1 <- Reducer 4 (BROADCAST_EDGE)
+Reducer 3 <- Map 2 (SIMPLE_EDGE)
+Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
+
+Stage-0
+ Fetch Operator
+ limit:-1
+ Stage-1
+ Map 1 llap
+ File Output Operator [FS_15]
+ Select Operator [SEL_14] (rows=500 width=178)
+ Output:["_col0","_col1"]
+ Filter Operator [FIL_13] (rows=500 width=179)
+ predicate:_col2 is null
+ Map Join Operator [MAPJOIN_28] (rows=762 width=179)
+ Conds:SEL_1._col0, _col1=RS_11._col1, _col2(Left Outer),Output:["_col0","_col1","_col2"]
+ <-Reducer 4 [BROADCAST_EDGE] llap
+ BROADCAST [RS_11]
+ PartitionCols:_col1, _col2
+ Select Operator [SEL_9] (rows=166 width=182)
+ Output:["_col0","_col1","_col2"]
+ Group By Operator [GBY_8] (rows=166 width=178)
+ Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
+ <-Reducer 3 [SIMPLE_EDGE] llap
+ SHUFFLE [RS_7]
+ PartitionCols:_col0, _col1
+ Group By Operator [GBY_6] (rows=166 width=178)
+ Output:["_col0","_col1"],keys:KEY._col0, KEY._col1
+ <-Map 2 [SIMPLE_EDGE] llap
+ SHUFFLE [RS_5]
+ PartitionCols:rand()
+ Filter Operator [FIL_17] (rows=166 width=178)
+ predicate:((value > 'val_9') and key is not null)
+ TableScan [TS_2] (rows=500 width=178)
+ default@src_cbo,a,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
+ <-Select Operator [SEL_1] (rows=500 width=178)
+ Output:["_col0","_col1"]
+ TableScan [TS_0] (rows=500 width=178)
+ default@src_cbo,b,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
+
diff --git a/ql/src/test/results/clientpositive/llap/external_jdbc_table_perf.q.out b/ql/src/test/results/clientpositive/llap/external_jdbc_table_perf.q.out
index b7d3c9a..dd7581f 100644
--- a/ql/src/test/results/clientpositive/llap/external_jdbc_table_perf.q.out
+++ b/ql/src/test/results/clientpositive/llap/external_jdbc_table_perf.q.out
@@ -1811,7 +1811,6 @@ WHERE "ws_bill_customer_sk" IS NOT NULL AND "ws_sold_date_sk" IS NOT NULL) AS "t
INNER JOIN (SELECT "d_date_sk"
FROM "DATE_DIM"
WHERE "d_year" = 1999 AND "d_moy" BETWEEN 1 AND 3 AND "d_date_sk" IS NOT NULL) AS "t2" ON "t0"."ws_sold_date_sk" = "t2"."d_date_sk"
-GROUP BY "t0"."ws_bill_customer_sk"
hive.sql.query.fieldNames literalTrue,ws_bill_customer_sk0
hive.sql.query.fieldTypes boolean,int
hive.sql.query.split false
@@ -1841,22 +1840,26 @@ WHERE "cs_ship_customer_sk" IS NOT NULL AND "cs_sold_date_sk" IS NOT NULL) AS "t
INNER JOIN (SELECT "d_date_sk"
FROM "DATE_DIM"
WHERE "d_year" = 1999 AND "d_moy" BETWEEN 1 AND 3 AND "d_date_sk" IS NOT NULL) AS "t2" ON "t0"."cs_sold_date_sk" = "t2"."d_date_sk"
-GROUP BY "t0"."cs_ship_customer_sk"
hive.sql.query.fieldNames literalTrue,cs_ship_customer_sk0
hive.sql.query.fieldTypes boolean,int
hive.sql.query.split false
- Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: literaltrue (type: boolean), cs_ship_customer_sk0 (type: int)
- outputColumnNames: _col0, _col1
- Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
- Reduce Output Operator
- key expressions: _col1 (type: int)
- null sort order: z
- sort order: +
- Map-reduce partition columns: _col1 (type: int)
- Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col0 (type: boolean)
+ expressions: cs_ship_customer_sk0 (type: int)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+ Group By Operator
+ keys: _col0 (type: int)
+ minReductionHashAggr: 0.99
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: int)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Execution mode: vectorized, llap
LLAP IO: no inputs
Reducer 2
@@ -1907,39 +1910,32 @@ GROUP BY "t0"."cs_ship_customer_sk"
Reduce Operator Tree:
Merge Join Operator
condition map:
- Left Outer Join 0 to 1
+ Anti Join 0 to 1
keys:
0 _col0 (type: int)
- 1 _col1 (type: int)
- outputColumnNames: _col32, _col33, _col34, _col35, _col36, _col42
+ 1 _col0 (type: int)
+ outputColumnNames: _col32, _col33, _col34, _col35, _col36
Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
- Filter Operator
- predicate: _col42 is null (type: boolean)
+ Top N Key Operator
+ sort order: +++++
+ keys: _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
+ null sort order: zzzzz
Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
- Top N Key Operator
- sort order: +++++
+ top n: 100
+ Group By Operator
+ aggregations: count()
keys: _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
- null sort order: zzzzz
+ minReductionHashAggr: 0.99
+ mode: hash
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
- top n: 100
- Select Operator
- expressions: _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
- outputColumnNames: _col32, _col33, _col34, _col35, _col36
+ Reduce Output Operator
+ key expressions: _col0 (type: char(1)), _col1 (type: char(1)), _col2 (type: char(20)), _col3 (type: int), _col4 (type: char(10))
+ null sort order: zzzzz
+ sort order: +++++
+ Map-reduce partition columns: _col0 (type: char(1)), _col1 (type: char(1)), _col2 (type: char(20)), _col3 (type: int), _col4 (type: char(10))
Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
- Group By Operator
- aggregations: count()
- keys: _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
- minReductionHashAggr: 0.99
- mode: hash
- outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
- Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
- Reduce Output Operator
- key expressions: _col0 (type: char(1)), _col1 (type: char(1)), _col2 (type: char(20)), _col3 (type: int), _col4 (type: char(10))
- null sort order: zzzzz
- sort order: +++++
- Map-reduce partition columns: _col0 (type: char(1)), _col1 (type: char(1)), _col2 (type: char(20)), _col3 (type: int), _col4 (type: char(10))
- Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col5 (type: bigint)
+ value expressions: _col5 (type: bigint)
Reducer 5
Execution mode: vectorized, llap
Reduce Operator Tree:
@@ -2299,7 +2295,6 @@ WHERE "ws_bill_customer_sk" IS NOT NULL AND "ws_sold_date_sk" IS NOT NULL) AS "t
INNER JOIN (SELECT "d_date_sk"
FROM "DATE_DIM"
WHERE "d_moy" BETWEEN 1 AND 3 AND "d_year" = 1999 AND "d_date_sk" IS NOT NULL) AS "t2" ON "t0"."ws_sold_date_sk" = "t2"."d_date_sk"
-GROUP BY "t0"."ws_bill_customer_sk"
hive.sql.query.fieldNames literalTrue,ws_bill_customer_sk0
hive.sql.query.fieldTypes boolean,int
hive.sql.query.split false
@@ -2329,22 +2324,26 @@ WHERE "cs_ship_customer_sk" IS NOT NULL AND "cs_sold_date_sk" IS NOT NULL) AS "t
INNER JOIN (SELECT "d_date_sk"
FROM "DATE_DIM"
WHERE "d_moy" BETWEEN 1 AND 3 AND "d_year" = 1999 AND "d_date_sk" IS NOT NULL) AS "t2" ON "t0"."cs_sold_date_sk" = "t2"."d_date_sk"
-GROUP BY "t0"."cs_ship_customer_sk"
hive.sql.query.fieldNames literalTrue,cs_ship_customer_sk0
hive.sql.query.fieldTypes boolean,int
hive.sql.query.split false
- Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: literaltrue (type: boolean), cs_ship_customer_sk0 (type: int)
- outputColumnNames: _col0, _col1
- Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
- Reduce Output Operator
- key expressions: _col1 (type: int)
- null sort order: z
- sort order: +
- Map-reduce partition columns: _col1 (type: int)
- Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col0 (type: boolean)
+ expressions: cs_ship_customer_sk0 (type: int)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+ Group By Operator
+ keys: _col0 (type: int)
+ minReductionHashAggr: 0.99
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: int)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
Execution mode: vectorized, llap
LLAP IO: no inputs
Reducer 2
@@ -2395,39 +2394,32 @@ GROUP BY "t0"."cs_ship_customer_sk"
Reduce Operator Tree:
Merge Join Operator
condition map:
- Left Outer Join 0 to 1
+ Anti Join 0 to 1
keys:
0 _col0 (type: int)
- 1 _col1 (type: int)
- outputColumnNames: _col32, _col33, _col34, _col35, _col36, _col42
+ 1 _col0 (type: int)
+ outputColumnNames: _col32, _col33, _col34, _col35, _col36
Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
- Filter Operator
- predicate: _col42 is null (type: boolean)
+ Top N Key Operator
+ sort order: +++++
+ keys: _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
+ null sort order: zzzzz
Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
- Top N Key Operator
- sort order: +++++
+ top n: 100
+ Group By Operator
+ aggregations: count()
keys: _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
- null sort order: zzzzz
+ minReductionHashAggr: 0.99
+ mode: hash
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
- top n: 100
- Select Operator
- expressions: _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
- outputColumnNames: _col32, _col33, _col34, _col35, _col36
+ Reduce Output Operator
+ key expressions: _col0 (type: char(1)), _col1 (type: char(1)), _col2 (type: char(20)), _col3 (type: int), _col4 (type: char(10))
+ null sort order: zzzzz
+ sort order: +++++
+ Map-reduce partition columns: _col0 (type: char(1)), _col1 (type: char(1)), _col2 (type: char(20)), _col3 (type: int), _col4 (type: char(10))
Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
- Group By Operator
- aggregations: count()
- keys: _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
- minReductionHashAggr: 0.99
- mode: hash
- outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
- Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
- Reduce Output Operator
- key expressions: _col0 (type: char(1)), _col1 (type: char(1)), _col2 (type: char(20)), _col3 (type: int), _col4 (type: char(10))
- null sort order: zzzzz
- sort order: +++++
- Map-reduce partition columns: _col0 (type: char(1)), _col1 (type: char(1)), _col2 (type: char(20)), _col3 (type: int), _col4 (type: char(10))
- Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col5 (type: bigint)
+ value expressions: _col5 (type: bigint)
Reducer 5
Execution mode: vectorized, llap
Reduce Operator Tree:
@@ -5818,6 +5810,974 @@ POSTHOOK: Input: default@reason
POSTHOOK: Input: default@web_returns
POSTHOOK: Input: default@web_sales
#### A masked pattern was here ####
+PREHOOK: query: explain
+SELECT cd_gender,
+ cd_marital_status,
+ cd_education_status,
+ Count(*) cnt1,
+ cd_purchase_estimate,
+ Count(*) cnt2,
+ cd_credit_rating,
+ Count(*) cnt3
+FROM customer c,
+ customer_address ca,
+ customer_demographics
+WHERE c.c_current_addr_sk = ca.ca_address_sk
+ AND ca_state IN ( 'CO', 'IL', 'MN' )
+ AND cd_demo_sk = c.c_current_cdemo_sk
+ AND EXISTS (SELECT *
+ FROM store_sales,
+ date_dim
+ WHERE c.c_customer_sk = ss_customer_sk
+ AND ss_sold_date_sk = d_date_sk
+ AND d_year = 1999
+ AND d_moy BETWEEN 1 AND 1 + 2)
+ AND ( NOT EXISTS (SELECT *
+ FROM web_sales,
+ date_dim
+ WHERE c.c_customer_sk = ws_bill_customer_sk
+ AND ws_sold_date_sk = d_date_sk
+ AND d_year = 1999
+ AND d_moy BETWEEN 1 AND 1 + 2)
+ AND NOT EXISTS (SELECT *
+ FROM catalog_sales,
+ date_dim
+ WHERE c.c_customer_sk = cs_ship_customer_sk
+ AND cs_sold_date_sk = d_date_sk
+ AND d_year = 1999
+ AND d_moy BETWEEN 1 AND 1 + 2) )
+GROUP BY cd_gender,
+ cd_marital_status,
+ cd_education_status,
+ cd_purchase_estimate,
+ cd_credit_rating
+ORDER BY cd_gender,
+ cd_marital_status,
+ cd_education_status,
+ cd_purchase_estimate,
+ cd_credit_rating
+LIMIT 100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@catalog_sales
+PREHOOK: Input: default@customer
+PREHOOK: Input: default@customer_address
+PREHOOK: Input: default@customer_demographics
+PREHOOK: Input: default@date_dim
+PREHOOK: Input: default@store_sales
+PREHOOK: Input: default@web_sales
+#### A masked pattern was here ####
+POSTHOOK: query: explain
+SELECT cd_gender,
+ cd_marital_status,
+ cd_education_status,
+ Count(*) cnt1,
+ cd_purchase_estimate,
+ Count(*) cnt2,
+ cd_credit_rating,
+ Count(*) cnt3
+FROM customer c,
+ customer_address ca,
+ customer_demographics
+WHERE c.c_current_addr_sk = ca.ca_address_sk
+ AND ca_state IN ( 'CO', 'IL', 'MN' )
+ AND cd_demo_sk = c.c_current_cdemo_sk
+ AND EXISTS (SELECT *
+ FROM store_sales,
+ date_dim
+ WHERE c.c_customer_sk = ss_customer_sk
+ AND ss_sold_date_sk = d_date_sk
+ AND d_year = 1999
+ AND d_moy BETWEEN 1 AND 1 + 2)
+ AND ( NOT EXISTS (SELECT *
+ FROM web_sales,
+ date_dim
+ WHERE c.c_customer_sk = ws_bill_customer_sk
+ AND ws_sold_date_sk = d_date_sk
+ AND d_year = 1999
+ AND d_moy BETWEEN 1 AND 1 + 2)
+ AND NOT EXISTS (SELECT *
+ FROM catalog_sales,
+ date_dim
+ WHERE c.c_customer_sk = cs_ship_customer_sk
+ AND cs_sold_date_sk = d_date_sk
+ AND d_year = 1999
+ AND d_moy BETWEEN 1 AND 1 + 2) )
+GROUP BY cd_gender,
+ cd_marital_status,
+ cd_education_status,
+ cd_purchase_estimate,
+ cd_credit_rating
+ORDER BY cd_gender,
+ cd_marital_status,
+ cd_education_status,
+ cd_purchase_estimate,
+ cd_credit_rating
+LIMIT 100
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@catalog_sales
+POSTHOOK: Input: default@customer
+POSTHOOK: Input: default@customer_address
+POSTHOOK: Input: default@customer_demographics
+POSTHOOK: Input: default@date_dim
+POSTHOOK: Input: default@store_sales
+POSTHOOK: Input: default@web_sales
+#### A masked pattern was here ####
+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), Map 7 (SIMPLE_EDGE)
+ Reducer 3 <- Map 8 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+ Reducer 4 <- Map 9 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+ Reducer 5 <- Reducer 4 (SIMPLE_EDGE)
+ Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: c
+ properties:
+ hive.sql.query SELECT "t0"."c_customer_sk", "t0"."c_customer_id", "t0"."c_current_cdemo_sk", "t0"."c_current_hdemo_sk", "t0"."c_current_addr_sk", "t0"."c_first_shipto_date_sk", "t0"."c_first_sales_date_sk", "t0"."c_salutation", "t0"."c_first_name", "t0"."c_last_name", "t0"."c_preferred_cust_flag", "t0"."c_birth_day", "t0"."c_birth_month", "t0"."c_birth_year", "t0"."c_birth_country", "t0"."c_login", "t0"."c_email_address", "t0"."c_last_review_date", "t4"."ca_address_sk [...]
+FROM (SELECT "c_customer_sk", "c_customer_id", "c_current_cdemo_sk", "c_current_hdemo_sk", "c_current_addr_sk", "c_first_shipto_date_sk", "c_first_sales_date_sk", "c_salutation", "c_first_name", "c_last_name", "c_preferred_cust_flag", "c_birth_day", "c_birth_month", "c_birth_year", "c_birth_country", "c_login", "c_email_address", "c_last_review_date"
+FROM "CUSTOMER"
+WHERE "c_current_addr_sk" IS NOT NULL AND "c_current_cdemo_sk" IS NOT NULL AND "c_customer_sk" IS NOT NULL) AS "t0"
+INNER JOIN (SELECT "cd_demo_sk", "cd_gender", "cd_marital_status", "cd_education_status", "cd_purchase_estimate", "cd_credit_rating", "cd_dep_count", "cd_dep_employed_count", "cd_dep_college_count"
+FROM "CUSTOMER_DEMOGRAPHICS"
+WHERE "cd_demo_sk" IS NOT NULL) AS "t2" ON "t0"."c_current_cdemo_sk" = "t2"."cd_demo_sk"
+INNER JOIN (SELECT "ca_address_sk", "ca_address_id", "ca_street_number", "ca_street_name", "ca_street_type", "ca_suite_number", "ca_city", "ca_county", "ca_state", "ca_zip", "ca_country", "ca_gmt_offset", "ca_location_type"
+FROM "CUSTOMER_ADDRESS"
+WHERE "ca_state" IN ('CO', 'IL', 'MN') AND "ca_address_sk" IS NOT NULL) AS "t4" ON "t0"."c_current_addr_sk" = "t4"."ca_address_sk"
+ hive.sql.query.fieldNames c_customer_sk,c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ca_address_sk,ca_address_id,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_locati [...]
+ hive.sql.query.fieldTypes int,char(16),int,int,int,int,int,char(10),char(20),char(30),char(1),int,int,int,varchar(20),char(13),char(50),char(10),int,char(16),char(10),varchar(60),char(15),char(10),varchar(60),varchar(30),char(2),char(10),varchar(20),decimal(5,2),char(20),int,char(1),char(1),char(20),int,char(10),int,int,int
+ hive.sql.query.split false
+ Statistics: Num rows: 1 Data size: 376 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: c_customer_sk (type: int), cd_gender (type: char(1)), cd_marital_status (type: char(1)), cd_education_status (type: char(20)), cd_purchase_estimate (type: int), cd_credit_rating (type: char(10))
+ outputColumnNames: _col0, _col32, _col33, _col34, _col35, _col36
+ Statistics: Num rows: 1 Data size: 376 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: int)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 1 Data size: 376 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
+ Execution mode: vectorized, llap
+ LLAP IO: no inputs
+ Map 7
+ Map Operator Tree:
+ TableScan
+ alias: store_sales
+ properties:
+ hive.sql.query SELECT "t0"."ss_customer_sk" AS "ss_customer_sk0"
+FROM (SELECT "ss_sold_date_sk", "ss_sold_time_sk", "ss_item_sk", "ss_customer_sk", "ss_cdemo_sk", "ss_hdemo_sk", "ss_addr_sk", "ss_store_sk", "ss_promo_sk", "ss_ticket_number", "ss_quantity", "ss_wholesale_cost", "ss_list_price", "ss_sales_price", "ss_ext_discount_amt", "ss_ext_sales_price", "ss_ext_wholesale_cost", "ss_ext_list_price", "ss_ext_tax", "ss_coupon_amt", "ss_net_paid", "ss_net_paid_inc_tax", "ss_net_profit"
+FROM "STORE_SALES"
+WHERE "ss_customer_sk" IS NOT NULL AND "ss_sold_date_sk" IS NOT NULL) AS "t0"
+INNER JOIN (SELECT "d_date_sk"
+FROM "DATE_DIM"
+WHERE "d_year" = 1999 AND "d_moy" BETWEEN 1 AND 3 AND "d_date_sk" IS NOT NULL) AS "t2" ON "t0"."ss_sold_date_sk" = "t2"."d_date_sk"
+ hive.sql.query.fieldNames ss_customer_sk0
+ hive.sql.query.fieldTypes int
+ hive.sql.query.split false
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: ss_customer_sk0 (type: int)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+ Group By Operator
+ keys: _col0 (type: int)
+ minReductionHashAggr: 0.99
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: int)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+ Execution mode: vectorized, llap
+ LLAP IO: no inputs
+ Map 8
+ Map Operator Tree:
+ TableScan
+ alias: web_sales
+ properties:
+ hive.sql.query SELECT TRUE AS "literalTrue", "t0"."ws_bill_customer_sk" AS "ws_bill_customer_sk0"
+FROM (SELECT "ws_sold_date_sk", "ws_sold_time_sk", "ws_ship_date_sk", "ws_item_sk", "ws_bill_customer_sk", "ws_bill_cdemo_sk", "ws_bill_hdemo_sk", "ws_bill_addr_sk", "ws_ship_customer_sk", "ws_ship_cdemo_sk", "ws_ship_hdemo_sk", "ws_ship_addr_sk", "ws_web_page_sk", "ws_web_site_sk", "ws_ship_mode_sk", "ws_warehouse_sk", "ws_promo_sk", "ws_order_number", "ws_quantity", "ws_wholesale_cost", "ws_list_price", "ws_sales_price", "ws_ext_discount_amt", "ws_ext_sales_price", "ws_ext_wholesale_co [...]
+FROM "WEB_SALES"
+WHERE "ws_bill_customer_sk" IS NOT NULL AND "ws_sold_date_sk" IS NOT NULL) AS "t0"
+INNER JOIN (SELECT "d_date_sk"
+FROM "DATE_DIM"
+WHERE "d_year" = 1999 AND "d_moy" BETWEEN 1 AND 3 AND "d_date_sk" IS NOT NULL) AS "t2" ON "t0"."ws_sold_date_sk" = "t2"."d_date_sk"
+ hive.sql.query.fieldNames literalTrue,ws_bill_customer_sk0
+ hive.sql.query.fieldTypes boolean,int
+ hive.sql.query.split false
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: literaltrue (type: boolean), ws_bill_customer_sk0 (type: int)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col1 (type: int)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col1 (type: int)
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: boolean)
+ Execution mode: vectorized, llap
+ LLAP IO: no inputs
+ Map 9
+ Map Operator Tree:
+ TableScan
+ alias: catalog_sales
+ properties:
+ hive.sql.query SELECT TRUE AS "literalTrue", "t0"."cs_ship_customer_sk" AS "cs_ship_customer_sk0"
+FROM (SELECT "cs_sold_date_sk", "cs_sold_time_sk", "cs_ship_date_sk", "cs_bill_customer_sk", "cs_bill_cdemo_sk", "cs_bill_hdemo_sk", "cs_bill_addr_sk", "cs_ship_customer_sk", "cs_ship_cdemo_sk", "cs_ship_hdemo_sk", "cs_ship_addr_sk", "cs_call_center_sk", "cs_catalog_page_sk", "cs_ship_mode_sk", "cs_warehouse_sk", "cs_item_sk", "cs_promo_sk", "cs_order_number", "cs_quantity", "cs_wholesale_cost", "cs_list_price", "cs_sales_price", "cs_ext_discount_amt", "cs_ext_sales_price", "cs_ext_whole [...]
+FROM "CATALOG_SALES"
+WHERE "cs_ship_customer_sk" IS NOT NULL AND "cs_sold_date_sk" IS NOT NULL) AS "t0"
+INNER JOIN (SELECT "d_date_sk"
+FROM "DATE_DIM"
+WHERE "d_year" = 1999 AND "d_moy" BETWEEN 1 AND 3 AND "d_date_sk" IS NOT NULL) AS "t2" ON "t0"."cs_sold_date_sk" = "t2"."d_date_sk"
+ hive.sql.query.fieldNames literalTrue,cs_ship_customer_sk0
+ hive.sql.query.fieldTypes boolean,int
+ hive.sql.query.split false
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: cs_ship_customer_sk0 (type: int)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+ Group By Operator
+ keys: _col0 (type: int)
+ minReductionHashAggr: 0.99
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: int)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+ Execution mode: vectorized, llap
+ LLAP IO: no inputs
+ Reducer 2
+ Execution mode: llap
+ Reduce Operator Tree:
+ Merge Join Operator
+ condition map:
+ Left Semi Join 0 to 1
+ keys:
+ 0 _col0 (type: int)
+ 1 _col0 (type: int)
+ outputColumnNames: _col0, _col32, _col33, _col34, _col35, _col36
+ Statistics: Num rows: 1 Data size: 413 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: int)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 1 Data size: 413 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
+ Reducer 3
+ Execution mode: llap
+ Reduce Operator Tree:
+ Merge Join Operator
+ condition map:
+ Left Outer Join 0 to 1
+ keys:
+ 0 _col0 (type: int)
+ 1 _col1 (type: int)
+ outputColumnNames: _col0, _col32, _col33, _col34, _col35, _col36, _col40
+ Statistics: Num rows: 1 Data size: 454 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: _col40 is null (type: boolean)
+ Statistics: Num rows: 1 Data size: 454 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: int), _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
+ outputColumnNames: _col0, _col32, _col33, _col34, _col35, _col36
+ Statistics: Num rows: 1 Data size: 454 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: int)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 1 Data size: 454 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
+ Reducer 4
+ Execution mode: llap
+ Reduce Operator Tree:
+ Merge Join Operator
+ condition map:
+ Anti Join 0 to 1
+ keys:
+ 0 _col0 (type: int)
+ 1 _col0 (type: int)
+ outputColumnNames: _col32, _col33, _col34, _col35, _col36
+ Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+ Top N Key Operator
+ sort order: +++++
+ keys: _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
+ null sort order: zzzzz
+ Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+ top n: 100
+ Group By Operator
+ aggregations: count()
+ keys: _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
+ minReductionHashAggr: 0.99
+ mode: hash
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+ Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: char(1)), _col1 (type: char(1)), _col2 (type: char(20)), _col3 (type: int), _col4 (type: char(10))
+ null sort order: zzzzz
+ sort order: +++++
+ Map-reduce partition columns: _col0 (type: char(1)), _col1 (type: char(1)), _col2 (type: char(20)), _col3 (type: int), _col4 (type: char(10))
+ Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col5 (type: bigint)
+ Reducer 5
+ Execution mode: vectorized, llap
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations: count(VALUE._col0)
+ keys: KEY._col0 (type: char(1)), KEY._col1 (type: char(1)), KEY._col2 (type: char(20)), KEY._col3 (type: int), KEY._col4 (type: char(10))
+ mode: mergepartial
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+ Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: char(1)), _col1 (type: char(1)), _col2 (type: char(20)), _col5 (type: bigint), _col3 (type: int), _col4 (type: char(10))
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col6
+ Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: char(1)), _col1 (type: char(1)), _col2 (type: char(20)), _col4 (type: int), _col6 (type: char(10))
+ null sort order: zzzzz
+ sort order: +++++
+ Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col3 (type: bigint)
+ Reducer 6
+ Execution mode: vectorized, llap
+ Reduce Operator Tree:
+ Select Operator
+ expressions: KEY.reducesinkkey0 (type: char(1)), KEY.reducesinkkey1 (type: char(1)), KEY.reducesinkkey2 (type: char(20)), VALUE._col0 (type: bigint), KEY.reducesinkkey3 (type: int), VALUE._col0 (type: bigint), KEY.reducesinkkey4 (type: char(10)), VALUE._col0 (type: bigint)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
+ Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+ Limit
+ Number of rows: 100
+ Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 499 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: 100
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: SELECT cd_gender,
+ cd_marital_status,
+ cd_education_status,
+ Count(*) cnt1,
+ cd_purchase_estimate,
+ Count(*) cnt2,
+ cd_credit_rating,
+ Count(*) cnt3
+FROM customer c,
+ customer_address ca,
+ customer_demographics
+WHERE c.c_current_addr_sk = ca.ca_address_sk
+ AND ca_state IN ( 'CO', 'IL', 'MN' )
+ AND cd_demo_sk = c.c_current_cdemo_sk
+ AND EXISTS (SELECT *
+ FROM store_sales,
+ date_dim
+ WHERE c.c_customer_sk = ss_customer_sk
+ AND ss_sold_date_sk = d_date_sk
+ AND d_year = 1999
+ AND d_moy BETWEEN 1 AND 1 + 2)
+ AND ( NOT EXISTS (SELECT *
+ FROM web_sales,
+ date_dim
+ WHERE c.c_customer_sk = ws_bill_customer_sk
+ AND ws_sold_date_sk = d_date_sk
+ AND d_year = 1999
+ AND d_moy BETWEEN 1 AND 1 + 2)
+ AND NOT EXISTS (SELECT *
+ FROM catalog_sales,
+ date_dim
+ WHERE c.c_customer_sk = cs_ship_customer_sk
+ AND cs_sold_date_sk = d_date_sk
+ AND d_year = 1999
+ AND d_moy BETWEEN 1 AND 1 + 2) )
+GROUP BY cd_gender,
+ cd_marital_status,
+ cd_education_status,
+ cd_purchase_estimate,
+ cd_credit_rating
+ORDER BY cd_gender,
+ cd_marital_status,
+ cd_education_status,
+ cd_purchase_estimate,
+ cd_credit_rating
+LIMIT 100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@catalog_sales
+PREHOOK: Input: default@customer
+PREHOOK: Input: default@customer_address
+PREHOOK: Input: default@customer_demographics
+PREHOOK: Input: default@date_dim
+PREHOOK: Input: default@store_sales
+PREHOOK: Input: default@web_sales
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT cd_gender,
+ cd_marital_status,
+ cd_education_status,
+ Count(*) cnt1,
+ cd_purchase_estimate,
+ Count(*) cnt2,
+ cd_credit_rating,
+ Count(*) cnt3
+FROM customer c,
+ customer_address ca,
+ customer_demographics
+WHERE c.c_current_addr_sk = ca.ca_address_sk
+ AND ca_state IN ( 'CO', 'IL', 'MN' )
+ AND cd_demo_sk = c.c_current_cdemo_sk
+ AND EXISTS (SELECT *
+ FROM store_sales,
+ date_dim
+ WHERE c.c_customer_sk = ss_customer_sk
+ AND ss_sold_date_sk = d_date_sk
+ AND d_year = 1999
+ AND d_moy BETWEEN 1 AND 1 + 2)
+ AND ( NOT EXISTS (SELECT *
+ FROM web_sales,
+ date_dim
+ WHERE c.c_customer_sk = ws_bill_customer_sk
+ AND ws_sold_date_sk = d_date_sk
+ AND d_year = 1999
+ AND d_moy BETWEEN 1 AND 1 + 2)
+ AND NOT EXISTS (SELECT *
+ FROM catalog_sales,
+ date_dim
+ WHERE c.c_customer_sk = cs_ship_customer_sk
+ AND cs_sold_date_sk = d_date_sk
+ AND d_year = 1999
+ AND d_moy BETWEEN 1 AND 1 + 2) )
+GROUP BY cd_gender,
+ cd_marital_status,
+ cd_education_status,
+ cd_purchase_estimate,
+ cd_credit_rating
+ORDER BY cd_gender,
+ cd_marital_status,
+ cd_education_status,
+ cd_purchase_estimate,
+ cd_credit_rating
+LIMIT 100
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@catalog_sales
+POSTHOOK: Input: default@customer
+POSTHOOK: Input: default@customer_address
+POSTHOOK: Input: default@customer_demographics
+POSTHOOK: Input: default@date_dim
+POSTHOOK: Input: default@store_sales
+POSTHOOK: Input: default@web_sales
+#### A masked pattern was here ####
+PREHOOK: query: explain
+SELECT cd_gender,
+ cd_marital_status,
+ cd_education_status,
+ Count(*) cnt1,
+ cd_purchase_estimate,
+ Count(*) cnt2,
+ cd_credit_rating,
+ Count(*) cnt3
+FROM customer c,
+ customer_address ca,
+ customer_demographics
+WHERE c.c_current_addr_sk = ca.ca_address_sk
+ AND ca_state IN ( 'CO', 'IL', 'MN' )
+ AND cd_demo_sk = c.c_current_cdemo_sk
+ AND EXISTS (SELECT *
+ FROM store_sales,
+ date_dim
+ WHERE c.c_customer_sk = ss_customer_sk
+ AND ss_sold_date_sk = d_date_sk
+ AND d_year = 1999
+ AND d_moy NOT BETWEEN 1 AND 1 + 2)
+ AND ( NOT EXISTS (SELECT *
+ FROM web_sales,
+ date_dim
+ WHERE c.c_customer_sk = ws_bill_customer_sk
+ AND ws_sold_date_sk = d_date_sk
+ AND d_year = 1999
+ AND d_moy NOT BETWEEN 1 AND 1 + 2)
+ AND NOT EXISTS (SELECT *
+ FROM catalog_sales,
+ date_dim
+ WHERE c.c_customer_sk = cs_ship_customer_sk
+ AND cs_sold_date_sk = d_date_sk
+ AND d_year = 1999
+ AND d_moy NOT BETWEEN 1 AND 1 + 2) )
+GROUP BY cd_gender,
+ cd_marital_status,
+ cd_education_status,
+ cd_purchase_estimate,
+ cd_credit_rating
+ORDER BY cd_gender,
+ cd_marital_status,
+ cd_education_status,
+ cd_purchase_estimate,
+ cd_credit_rating
+LIMIT 100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@catalog_sales
+PREHOOK: Input: default@customer
+PREHOOK: Input: default@customer_address
+PREHOOK: Input: default@customer_demographics
+PREHOOK: Input: default@date_dim
+PREHOOK: Input: default@store_sales
+PREHOOK: Input: default@web_sales
+#### A masked pattern was here ####
+POSTHOOK: query: explain
+SELECT cd_gender,
+ cd_marital_status,
+ cd_education_status,
+ Count(*) cnt1,
+ cd_purchase_estimate,
+ Count(*) cnt2,
+ cd_credit_rating,
+ Count(*) cnt3
+FROM customer c,
+ customer_address ca,
+ customer_demographics
+WHERE c.c_current_addr_sk = ca.ca_address_sk
+ AND ca_state IN ( 'CO', 'IL', 'MN' )
+ AND cd_demo_sk = c.c_current_cdemo_sk
+ AND EXISTS (SELECT *
+ FROM store_sales,
+ date_dim
+ WHERE c.c_customer_sk = ss_customer_sk
+ AND ss_sold_date_sk = d_date_sk
+ AND d_year = 1999
+ AND d_moy NOT BETWEEN 1 AND 1 + 2)
+ AND ( NOT EXISTS (SELECT *
+ FROM web_sales,
+ date_dim
+ WHERE c.c_customer_sk = ws_bill_customer_sk
+ AND ws_sold_date_sk = d_date_sk
+ AND d_year = 1999
+ AND d_moy NOT BETWEEN 1 AND 1 + 2)
+ AND NOT EXISTS (SELECT *
+ FROM catalog_sales,
+ date_dim
+ WHERE c.c_customer_sk = cs_ship_customer_sk
+ AND cs_sold_date_sk = d_date_sk
+ AND d_year = 1999
+ AND d_moy NOT BETWEEN 1 AND 1 + 2) )
+GROUP BY cd_gender,
+ cd_marital_status,
+ cd_education_status,
+ cd_purchase_estimate,
+ cd_credit_rating
+ORDER BY cd_gender,
+ cd_marital_status,
+ cd_education_status,
+ cd_purchase_estimate,
+ cd_credit_rating
+LIMIT 100
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@catalog_sales
+POSTHOOK: Input: default@customer
+POSTHOOK: Input: default@customer_address
+POSTHOOK: Input: default@customer_demographics
+POSTHOOK: Input: default@date_dim
+POSTHOOK: Input: default@store_sales
+POSTHOOK: Input: default@web_sales
+#### A masked pattern was here ####
+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), Map 7 (SIMPLE_EDGE)
+ Reducer 3 <- Map 8 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+ Reducer 4 <- Map 9 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+ Reducer 5 <- Reducer 4 (SIMPLE_EDGE)
+ Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: c
+ properties:
+ hive.sql.query SELECT "t0"."c_customer_sk", "t0"."c_customer_id", "t0"."c_current_cdemo_sk", "t0"."c_current_hdemo_sk", "t0"."c_current_addr_sk", "t0"."c_first_shipto_date_sk", "t0"."c_first_sales_date_sk", "t0"."c_salutation", "t0"."c_first_name", "t0"."c_last_name", "t0"."c_preferred_cust_flag", "t0"."c_birth_day", "t0"."c_birth_month", "t0"."c_birth_year", "t0"."c_birth_country", "t0"."c_login", "t0"."c_email_address", "t0"."c_last_review_date", "t4"."ca_address_sk [...]
+FROM (SELECT "c_customer_sk", "c_customer_id", "c_current_cdemo_sk", "c_current_hdemo_sk", "c_current_addr_sk", "c_first_shipto_date_sk", "c_first_sales_date_sk", "c_salutation", "c_first_name", "c_last_name", "c_preferred_cust_flag", "c_birth_day", "c_birth_month", "c_birth_year", "c_birth_country", "c_login", "c_email_address", "c_last_review_date"
+FROM "CUSTOMER"
+WHERE "c_current_addr_sk" IS NOT NULL AND "c_current_cdemo_sk" IS NOT NULL AND "c_customer_sk" IS NOT NULL) AS "t0"
+INNER JOIN (SELECT "cd_demo_sk", "cd_gender", "cd_marital_status", "cd_education_status", "cd_purchase_estimate", "cd_credit_rating", "cd_dep_count", "cd_dep_employed_count", "cd_dep_college_count"
+FROM "CUSTOMER_DEMOGRAPHICS"
+WHERE "cd_demo_sk" IS NOT NULL) AS "t2" ON "t0"."c_current_cdemo_sk" = "t2"."cd_demo_sk"
+INNER JOIN (SELECT "ca_address_sk", "ca_address_id", "ca_street_number", "ca_street_name", "ca_street_type", "ca_suite_number", "ca_city", "ca_county", "ca_state", "ca_zip", "ca_country", "ca_gmt_offset", "ca_location_type"
+FROM "CUSTOMER_ADDRESS"
+WHERE "ca_state" IN ('CO', 'IL', 'MN') AND "ca_address_sk" IS NOT NULL) AS "t4" ON "t0"."c_current_addr_sk" = "t4"."ca_address_sk"
+ hive.sql.query.fieldNames c_customer_sk,c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ca_address_sk,ca_address_id,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_locati [...]
+ hive.sql.query.fieldTypes int,char(16),int,int,int,int,int,char(10),char(20),char(30),char(1),int,int,int,varchar(20),char(13),char(50),char(10),int,char(16),char(10),varchar(60),char(15),char(10),varchar(60),varchar(30),char(2),char(10),varchar(20),decimal(5,2),char(20),int,char(1),char(1),char(20),int,char(10),int,int,int
+ hive.sql.query.split false
+ Statistics: Num rows: 1 Data size: 376 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: c_customer_sk (type: int), cd_gender (type: char(1)), cd_marital_status (type: char(1)), cd_education_status (type: char(20)), cd_purchase_estimate (type: int), cd_credit_rating (type: char(10))
+ outputColumnNames: _col0, _col32, _col33, _col34, _col35, _col36
+ Statistics: Num rows: 1 Data size: 376 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: int)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 1 Data size: 376 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
+ Execution mode: vectorized, llap
+ LLAP IO: no inputs
+ Map 7
+ Map Operator Tree:
+ TableScan
+ alias: store_sales
+ properties:
+ hive.sql.query SELECT "t0"."ss_customer_sk" AS "ss_customer_sk0"
+FROM (SELECT "ss_sold_date_sk", "ss_sold_time_sk", "ss_item_sk", "ss_customer_sk", "ss_cdemo_sk", "ss_hdemo_sk", "ss_addr_sk", "ss_store_sk", "ss_promo_sk", "ss_ticket_number", "ss_quantity", "ss_wholesale_cost", "ss_list_price", "ss_sales_price", "ss_ext_discount_amt", "ss_ext_sales_price", "ss_ext_wholesale_cost", "ss_ext_list_price", "ss_ext_tax", "ss_coupon_amt", "ss_net_paid", "ss_net_paid_inc_tax", "ss_net_profit"
+FROM "STORE_SALES"
+WHERE "ss_customer_sk" IS NOT NULL AND "ss_sold_date_sk" IS NOT NULL) AS "t0"
+INNER JOIN (SELECT "d_date_sk"
+FROM "DATE_DIM"
+WHERE "d_moy" BETWEEN 1 AND 3 AND "d_year" = 1999 AND "d_date_sk" IS NOT NULL) AS "t2" ON "t0"."ss_sold_date_sk" = "t2"."d_date_sk"
+ hive.sql.query.fieldNames ss_customer_sk0
+ hive.sql.query.fieldTypes int
+ hive.sql.query.split false
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: ss_customer_sk0 (type: int)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+ Group By Operator
+ keys: _col0 (type: int)
+ minReductionHashAggr: 0.99
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: int)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+ Execution mode: vectorized, llap
+ LLAP IO: no inputs
+ Map 8
+ Map Operator Tree:
+ TableScan
+ alias: web_sales
+ properties:
+ hive.sql.query SELECT TRUE AS "literalTrue", "t0"."ws_bill_customer_sk" AS "ws_bill_customer_sk0"
+FROM (SELECT "ws_sold_date_sk", "ws_sold_time_sk", "ws_ship_date_sk", "ws_item_sk", "ws_bill_customer_sk", "ws_bill_cdemo_sk", "ws_bill_hdemo_sk", "ws_bill_addr_sk", "ws_ship_customer_sk", "ws_ship_cdemo_sk", "ws_ship_hdemo_sk", "ws_ship_addr_sk", "ws_web_page_sk", "ws_web_site_sk", "ws_ship_mode_sk", "ws_warehouse_sk", "ws_promo_sk", "ws_order_number", "ws_quantity", "ws_wholesale_cost", "ws_list_price", "ws_sales_price", "ws_ext_discount_amt", "ws_ext_sales_price", "ws_ext_wholesale_co [...]
+FROM "WEB_SALES"
+WHERE "ws_bill_customer_sk" IS NOT NULL AND "ws_sold_date_sk" IS NOT NULL) AS "t0"
+INNER JOIN (SELECT "d_date_sk"
+FROM "DATE_DIM"
+WHERE "d_moy" BETWEEN 1 AND 3 AND "d_year" = 1999 AND "d_date_sk" IS NOT NULL) AS "t2" ON "t0"."ws_sold_date_sk" = "t2"."d_date_sk"
+ hive.sql.query.fieldNames literalTrue,ws_bill_customer_sk0
+ hive.sql.query.fieldTypes boolean,int
+ hive.sql.query.split false
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: literaltrue (type: boolean), ws_bill_customer_sk0 (type: int)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col1 (type: int)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col1 (type: int)
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: boolean)
+ Execution mode: vectorized, llap
+ LLAP IO: no inputs
+ Map 9
+ Map Operator Tree:
+ TableScan
+ alias: catalog_sales
+ properties:
+ hive.sql.query SELECT TRUE AS "literalTrue", "t0"."cs_ship_customer_sk" AS "cs_ship_customer_sk0"
+FROM (SELECT "cs_sold_date_sk", "cs_sold_time_sk", "cs_ship_date_sk", "cs_bill_customer_sk", "cs_bill_cdemo_sk", "cs_bill_hdemo_sk", "cs_bill_addr_sk", "cs_ship_customer_sk", "cs_ship_cdemo_sk", "cs_ship_hdemo_sk", "cs_ship_addr_sk", "cs_call_center_sk", "cs_catalog_page_sk", "cs_ship_mode_sk", "cs_warehouse_sk", "cs_item_sk", "cs_promo_sk", "cs_order_number", "cs_quantity", "cs_wholesale_cost", "cs_list_price", "cs_sales_price", "cs_ext_discount_amt", "cs_ext_sales_price", "cs_ext_whole [...]
+FROM "CATALOG_SALES"
+WHERE "cs_ship_customer_sk" IS NOT NULL AND "cs_sold_date_sk" IS NOT NULL) AS "t0"
+INNER JOIN (SELECT "d_date_sk"
+FROM "DATE_DIM"
+WHERE "d_moy" BETWEEN 1 AND 3 AND "d_year" = 1999 AND "d_date_sk" IS NOT NULL) AS "t2" ON "t0"."cs_sold_date_sk" = "t2"."d_date_sk"
+ hive.sql.query.fieldNames literalTrue,cs_ship_customer_sk0
+ hive.sql.query.fieldTypes boolean,int
+ hive.sql.query.split false
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: cs_ship_customer_sk0 (type: int)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+ Group By Operator
+ keys: _col0 (type: int)
+ minReductionHashAggr: 0.99
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: int)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+ Execution mode: vectorized, llap
+ LLAP IO: no inputs
+ Reducer 2
+ Execution mode: llap
+ Reduce Operator Tree:
+ Merge Join Operator
+ condition map:
+ Left Semi Join 0 to 1
+ keys:
+ 0 _col0 (type: int)
+ 1 _col0 (type: int)
+ outputColumnNames: _col0, _col32, _col33, _col34, _col35, _col36
+ Statistics: Num rows: 1 Data size: 413 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: int)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 1 Data size: 413 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
+ Reducer 3
+ Execution mode: llap
+ Reduce Operator Tree:
+ Merge Join Operator
+ condition map:
+ Left Outer Join 0 to 1
+ keys:
+ 0 _col0 (type: int)
+ 1 _col1 (type: int)
+ outputColumnNames: _col0, _col32, _col33, _col34, _col35, _col36, _col40
+ Statistics: Num rows: 1 Data size: 454 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: _col40 is null (type: boolean)
+ Statistics: Num rows: 1 Data size: 454 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: int), _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
+ outputColumnNames: _col0, _col32, _col33, _col34, _col35, _col36
+ Statistics: Num rows: 1 Data size: 454 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: int)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 1 Data size: 454 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
+ Reducer 4
+ Execution mode: llap
+ Reduce Operator Tree:
+ Merge Join Operator
+ condition map:
+ Anti Join 0 to 1
+ keys:
+ 0 _col0 (type: int)
+ 1 _col0 (type: int)
+ outputColumnNames: _col32, _col33, _col34, _col35, _col36
+ Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+ Top N Key Operator
+ sort order: +++++
+ keys: _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
+ null sort order: zzzzz
+ Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+ top n: 100
+ Group By Operator
+ aggregations: count()
+ keys: _col32 (type: char(1)), _col33 (type: char(1)), _col34 (type: char(20)), _col35 (type: int), _col36 (type: char(10))
+ minReductionHashAggr: 0.99
+ mode: hash
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+ Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: char(1)), _col1 (type: char(1)), _col2 (type: char(20)), _col3 (type: int), _col4 (type: char(10))
+ null sort order: zzzzz
+ sort order: +++++
+ Map-reduce partition columns: _col0 (type: char(1)), _col1 (type: char(1)), _col2 (type: char(20)), _col3 (type: int), _col4 (type: char(10))
+ Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col5 (type: bigint)
+ Reducer 5
+ Execution mode: vectorized, llap
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations: count(VALUE._col0)
+ keys: KEY._col0 (type: char(1)), KEY._col1 (type: char(1)), KEY._col2 (type: char(20)), KEY._col3 (type: int), KEY._col4 (type: char(10))
+ mode: mergepartial
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+ Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: char(1)), _col1 (type: char(1)), _col2 (type: char(20)), _col5 (type: bigint), _col3 (type: int), _col4 (type: char(10))
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col6
+ Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: char(1)), _col1 (type: char(1)), _col2 (type: char(20)), _col4 (type: int), _col6 (type: char(10))
+ null sort order: zzzzz
+ sort order: +++++
+ Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col3 (type: bigint)
+ Reducer 6
+ Execution mode: vectorized, llap
+ Reduce Operator Tree:
+ Select Operator
+ expressions: KEY.reducesinkkey0 (type: char(1)), KEY.reducesinkkey1 (type: char(1)), KEY.reducesinkkey2 (type: char(20)), VALUE._col0 (type: bigint), KEY.reducesinkkey3 (type: int), VALUE._col0 (type: bigint), KEY.reducesinkkey4 (type: char(10)), VALUE._col0 (type: bigint)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
+ Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+ Limit
+ Number of rows: 100
+ Statistics: Num rows: 1 Data size: 499 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 499 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: 100
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: SELECT cd_gender,
+ cd_marital_status,
+ cd_education_status,
+ Count(*) cnt1,
+ cd_purchase_estimate,
+ Count(*) cnt2,
+ cd_credit_rating,
+ Count(*) cnt3
+FROM customer c,
+ customer_address ca,
+ customer_demographics
+WHERE c.c_current_addr_sk = ca.ca_address_sk
+ AND ca_state IN ( 'CO', 'IL', 'MN' )
+ AND cd_demo_sk = c.c_current_cdemo_sk
+ AND EXISTS (SELECT *
+ FROM store_sales,
+ date_dim
+ WHERE c.c_customer_sk = ss_customer_sk
+ AND ss_sold_date_sk = d_date_sk
+ AND d_year = 1999
+ AND d_moy NOT BETWEEN 1 AND 1 + 2)
+ AND ( NOT EXISTS (SELECT *
+ FROM web_sales,
+ date_dim
+ WHERE c.c_customer_sk = ws_bill_customer_sk
+ AND ws_sold_date_sk = d_date_sk
+ AND d_year = 1999
+ AND d_moy NOT BETWEEN 1 AND 1 + 2)
+ AND NOT EXISTS (SELECT *
+ FROM catalog_sales,
+ date_dim
+ WHERE c.c_customer_sk = cs_ship_customer_sk
+ AND cs_sold_date_sk = d_date_sk
+ AND d_year = 1999
+ AND d_moy NOT BETWEEN 1 AND 1 + 2) )
+GROUP BY cd_gender,
+ cd_marital_status,
+ cd_education_status,
+ cd_purchase_estimate,
+ cd_credit_rating
+ORDER BY cd_gender,
+ cd_marital_status,
+ cd_education_status,
+ cd_purchase_estimate,
+ cd_credit_rating
+LIMIT 100
+PREHOOK: type: QUERY
+PREHOOK: Input: default@catalog_sales
+PREHOOK: Input: default@customer
+PREHOOK: Input: default@customer_address
+PREHOOK: Input: default@customer_demographics
+PREHOOK: Input: default@date_dim
+PREHOOK: Input: default@store_sales
+PREHOOK: Input: default@web_sales
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT cd_gender,
+ cd_marital_status,
+ cd_education_status,
+ Count(*) cnt1,
+ cd_purchase_estimate,
+ Count(*) cnt2,
+ cd_credit_rating,
+ Count(*) cnt3
+FROM customer c,
+ customer_address ca,
+ customer_demographics
+WHERE c.c_current_addr_sk = ca.ca_address_sk
+ AND ca_state IN ( 'CO', 'IL', 'MN' )
+ AND cd_demo_sk = c.c_current_cdemo_sk
+ AND EXISTS (SELECT *
+ FROM store_sales,
+ date_dim
+ WHERE c.c_customer_sk = ss_customer_sk
+ AND ss_sold_date_sk = d_date_sk
+ AND d_year = 1999
+ AND d_moy NOT BETWEEN 1 AND 1 + 2)
+ AND ( NOT EXISTS (SELECT *
+ FROM web_sales,
+ date_dim
+ WHERE c.c_customer_sk = ws_bill_customer_sk
+ AND ws_sold_date_sk = d_date_sk
+ AND d_year = 1999
+ AND d_moy NOT BETWEEN 1 AND 1 + 2)
+ AND NOT EXISTS (SELECT *
+ FROM catalog_sales,
+ date_dim
+ WHERE c.c_customer_sk = cs_ship_customer_sk
+ AND cs_sold_date_sk = d_date_sk
+ AND d_year = 1999
+ AND d_moy NOT BETWEEN 1 AND 1 + 2) )
+GROUP BY cd_gender,
+ cd_marital_status,
+ cd_education_status,
+ cd_purchase_estimate,
+ cd_credit_rating
+ORDER BY cd_gender,
+ cd_marital_status,
+ cd_education_status,
+ cd_purchase_estimate,
+ cd_credit_rating
+LIMIT 100
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@catalog_sales
+POSTHOOK: Input: default@customer
+POSTHOOK: Input: default@customer_address
+POSTHOOK: Input: default@customer_demographics
+POSTHOOK: Input: default@date_dim
+POSTHOOK: Input: default@store_sales
+POSTHOOK: Input: default@web_sales
+#### A masked pattern was here ####
PREHOOK: query: DROP TABLE catalog_sales
PREHOOK: type: DROPTABLE
PREHOOK: Input: default@catalog_sales
diff --git a/ql/src/test/results/clientpositive/llap/lineage3.q.out b/ql/src/test/results/clientpositive/llap/lineage3.q.out
index 964075f..83233df 100644
--- a/ql/src/test/results/clientpositive/llap/lineage3.q.out
+++ b/ql/src/test/results/clientpositive/llap/lineage3.q.out
@@ -202,7 +202,7 @@ PREHOOK: type: QUERY
PREHOOK: Input: default@alltypesorc
PREHOOK: Input: default@src1
#### A masked pattern was here ####
-{"version":"1.0","engine":"tez","database":"default","hash":"723e79692e1de404c4ffb702097586da","queryText":"select * from src1 a\nwhere not exists\n (select cint from alltypesorc b\n where a.key = b.ctinyint + 300)\nand key > 300","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"edgeType":"PROJECTION"},{"sources":[2],"targets":[0,1],"expression":"(UDFToDouble(a.key) > 300.0D)","edgeType":"PREDICATE"},{"sources":[2],"targets":[0,1],"expressio [...]
+{"version":"1.0","engine":"tez","database":"default","hash":"723e79692e1de404c4ffb702097586da","queryText":"select * from src1 a\nwhere not exists\n (select cint from alltypesorc b\n where a.key = b.ctinyint + 300)\nand key > 300","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"edgeType":"PROJECTION"},{"sources":[2],"targets":[0,1],"expression":"(UDFToDouble(a.key) > 300.0D)","edgeType":"PREDICATE"},{"sources":[2],"targets":[0,1],"expressio [...]
369
401 val_401
406 val_406
@@ -401,3 +401,16 @@ Result schema has 2 fields, but we don't get as many dependencies
Result schema has 2 fields, but we don't get as many dependencies
Result schema has 2 fields, but we don't get as many dependencies
{"version":"1.0","engine":"tez","database":"default","hash":"e540a88155ffa4bf6842a4fdf3bfe639","queryText":"from src_dp, src_dp1\ninsert into dest_dp1 partition (year) select first, word, year\ninsert into dest_dp2 partition (y, m) select first, word, year, month\ninsert into dest_dp3 partition (y=2, m, d) select first, word, month m, day d where year=2\ninsert into dest_dp2 partition (y=1, m) select f, w, m\ninsert into dest_dp1 partition (year=0) select f, w","edges":[{"sources":[11]," [...]
+PREHOOK: query: select * from src1 a
+where not exists
+ (select cint from alltypesorc b
+ where a.key = b.ctinyint + 300)
+and key > 300
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc
+PREHOOK: Input: default@src1
+#### A masked pattern was here ####
+{"version":"1.0","engine":"tez","database":"default","hash":"723e79692e1de404c4ffb702097586da","queryText":"select * from src1 a\nwhere not exists\n (select cint from alltypesorc b\n where a.key = b.ctinyint + 300)\nand key > 300","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"edgeType":"PROJECTION"},{"sources":[2],"targets":[0,1],"expression":"(UDFToDouble(a.key) > 300.0D)","edgeType":"PREDICATE"},{"sources":[2],"targets":[0,1],"expressio [...]
+369
+401 val_401
+406 val_406
diff --git a/ql/src/test/results/clientpositive/llap/subquery_in_having.q.out b/ql/src/test/results/clientpositive/llap/subquery_in_having.q.out
index 68c9924..4164d1b 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_in_having.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_in_having.q.out
@@ -1643,6 +1643,374 @@ POSTHOOK: Input: _dummy_database@_dummy_table
POSTHOOK: Output: default@src_null_n4
POSTHOOK: Lineage: src_null_n4.key SCRIPT []
POSTHOOK: Lineage: src_null_n4.value EXPRESSION []
+Warning: Map Join MAPJOIN[127][bigTable=?] in task 'Map 1' is a cross product
+Warning: Map Join MAPJOIN[128][bigTable=?] in task 'Map 4' is a cross product
+Warning: Map Join MAPJOIN[129][bigTable=?] in task 'Reducer 5' is a cross product
+PREHOOK: query: explain
+select key, value, count(*)
+from src_null_n4 b
+where NOT EXISTS (select key from src_null_n4 where src_null_n4.value <> b.value)
+group by key, value
+having count(*) not in (select count(*) from src_null_n4 s1 where s1.key > '9' and s1.value <> b.value group by s1.key )
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src_null_n4
+#### A masked pattern was here ####
+POSTHOOK: query: explain
+select key, value, count(*)
+from src_null_n4 b
+where NOT EXISTS (select key from src_null_n4 where src_null_n4.value <> b.value)
+group by key, value
+having count(*) not in (select count(*) from src_null_n4 s1 where s1.key > '9' and s1.value <> b.value group by s1.key )
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src_null_n4
+#### A masked pattern was here ####
+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:
+ Map 1 <- Map 3 (BROADCAST_EDGE)
+ Map 4 <- Map 3 (BROADCAST_EDGE)
+ Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 6 (BROADCAST_EDGE), Reducer 8 (BROADCAST_EDGE)
+ Reducer 5 <- Map 1 (BROADCAST_EDGE), Map 4 (SIMPLE_EDGE)
+ Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
+ Reducer 7 <- Reducer 5 (SIMPLE_EDGE)
+ Reducer 8 <- Reducer 7 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: b
+ Statistics: Num rows: 1 Data size: 368 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: key (type: string), value (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 1 Data size: 368 Basic stats: COMPLETE Column stats: NONE
+ Map Join Operator
+ condition map:
+ Anti Join 0 to 1
+ keys:
+ 0
+ 1
+ outputColumnNames: _col0, _col1, _col2
+ input vertices:
+ 1 Map 3
+ residual filter predicates: {(_col2 <> _col1)}
+ Statistics: Num rows: 1 Data size: 553 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string), _col1 (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 1 Data size: 553 Basic stats: COMPLETE Column stats: NONE
+ Group By Operator
+ aggregations: count()
+ keys: _col1 (type: string), _col0 (type: string)
+ minReductionHashAggr: 0.99
+ mode: hash
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 1 Data size: 553 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: string), _col1 (type: string)
+ null sort order: zz
+ sort order: ++
+ Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
+ Statistics: Num rows: 1 Data size: 553 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col2 (type: bigint)
+ Filter Operator
+ predicate: (key > '9') (type: boolean)
+ Statistics: Num rows: 1 Data size: 368 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: key (type: string), value (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 1 Data size: 368 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ null sort order:
+ sort order:
+ Statistics: Num rows: 1 Data size: 368 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: string), _col1 (type: string)
+ Execution mode: llap
+ LLAP IO: all inputs
+ Map 3
+ Map Operator Tree:
+ TableScan
+ alias: src_null_n4
+ filterExpr: value is not null (type: boolean)
+ Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: value is not null (type: boolean)
+ Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: value (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE
+ Group By Operator
+ keys: _col0 (type: string)
+ minReductionHashAggr: 0.99
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ null sort order:
+ sort order:
+ Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: string)
+ Reduce Output Operator
+ null sort order:
+ sort order:
+ Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: string)
+ Execution mode: vectorized, llap
+ LLAP IO: all inputs
+ Map 4
+ Map Operator Tree:
+ TableScan
+ alias: b
+ filterExpr: value is not null (type: boolean)
+ Statistics: Num rows: 1 Data size: 368 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: value is not null (type: boolean)
+ Statistics: Num rows: 1 Data size: 368 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: key (type: string), value (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 1 Data size: 368 Basic stats: COMPLETE Column stats: NONE
+ Map Join Operator
+ condition map:
+ Anti Join 0 to 1
+ keys:
+ 0
+ 1
+ outputColumnNames: _col0, _col1, _col2
+ input vertices:
+ 1 Map 3
+ residual filter predicates: {(_col2 <> _col1)}
+ Statistics: Num rows: 1 Data size: 553 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string), _col1 (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 1 Data size: 553 Basic stats: COMPLETE Column stats: NONE
+ Group By Operator
+ keys: _col1 (type: string), _col0 (type: string)
+ minReductionHashAggr: 0.99
+ mode: hash
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 1 Data size: 553 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: string), _col1 (type: string)
+ null sort order: zz
+ sort order: ++
+ Map-reduce partition columns: _col0 (type: string)
+ Statistics: Num rows: 1 Data size: 553 Basic stats: COMPLETE Column stats: NONE
+ Execution mode: llap
+ LLAP IO: all inputs
+ Reducer 2
+ Execution mode: vectorized, llap
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations: count(VALUE._col0)
+ keys: KEY._col0 (type: string), KEY._col1 (type: string)
+ mode: mergepartial
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 1 Data size: 553 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col1 (type: string), _col0 (type: string), _col2 (type: bigint)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 1 Data size: 553 Basic stats: COMPLETE Column stats: NONE
+ Map Join Operator
+ condition map:
+ Left Outer Join 0 to 1
+ keys:
+ 0 _col1 (type: string)
+ 1 _col0 (type: string)
+ outputColumnNames: _col0, _col1, _col2, _col4, _col5
+ input vertices:
+ 1 Reducer 6
+ Statistics: Num rows: 1 Data size: 608 Basic stats: COMPLETE Column stats: NONE
+ Map Join Operator
+ condition map:
+ Left Outer Join 0 to 1
+ keys:
+ 0 _col1 (type: string), _col2 (type: bigint)
+ 1 _col2 (type: string), _col0 (type: bigint)
+ outputColumnNames: _col0, _col1, _col2, _col4, _col5, _col7
+ input vertices:
+ 1 Reducer 8
+ Statistics: Num rows: 1 Data size: 668 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (_col4 is null or (_col4 = 0L) or (_col7 is not null or _col2 is null or (_col5 < _col4)) is not true) (type: boolean)
+ Statistics: Num rows: 1 Data size: 668 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string), _col1 (type: string), _col2 (type: bigint)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 1 Data size: 668 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 668 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
+ Reducer 5
+ Execution mode: llap
+ Reduce Operator Tree:
+ Group By Operator
+ keys: KEY._col0 (type: string), KEY._col1 (type: string)
+ mode: mergepartial
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 1 Data size: 553 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string)
+ outputColumnNames: _col1
+ Statistics: Num rows: 1 Data size: 553 Basic stats: COMPLETE Column stats: NONE
+ Group By Operator
+ keys: _col1 (type: string)
+ mode: complete
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 553 Basic stats: COMPLETE Column stats: NONE
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ keys:
+ 0
+ 1
+ outputColumnNames: _col0, _col1, _col2
+ input vertices:
+ 0 Map 1
+ residual filter predicates: {(_col1 <> _col2)}
+ Statistics: Num rows: 1 Data size: 922 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string), _col2 (type: string)
+ outputColumnNames: _col0, _col2
+ Statistics: Num rows: 1 Data size: 922 Basic stats: COMPLETE Column stats: NONE
+ Group By Operator
+ aggregations: count()
+ keys: _col2 (type: string), _col0 (type: string)
+ minReductionHashAggr: 0.99
+ mode: hash
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 1 Data size: 922 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: string), _col1 (type: string)
+ null sort order: zz
+ sort order: ++
+ Map-reduce partition columns: _col0 (type: string)
+ Statistics: Num rows: 1 Data size: 922 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col2 (type: bigint)
+ Reduce Output Operator
+ key expressions: _col0 (type: string), _col1 (type: string)
+ null sort order: zz
+ sort order: ++
+ Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
+ Statistics: Num rows: 1 Data size: 922 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col2 (type: bigint)
+ Reducer 6
+ Execution mode: vectorized, llap
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations: count(VALUE._col0)
+ keys: KEY._col0 (type: string), KEY._col1 (type: string)
+ mode: mergepartial
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 1 Data size: 922 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string), _col2 (type: bigint)
+ outputColumnNames: _col1, _col2
+ Statistics: Num rows: 1 Data size: 922 Basic stats: COMPLETE Column stats: NONE
+ Group By Operator
+ aggregations: count(), count(_col2)
+ keys: _col1 (type: string)
+ mode: complete
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 1 Data size: 922 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
+ Statistics: Num rows: 1 Data size: 922 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col1 (type: bigint), _col2 (type: bigint)
+ Reducer 7
+ Execution mode: vectorized, llap
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations: count(VALUE._col0)
+ keys: KEY._col0 (type: string), KEY._col1 (type: string)
+ mode: mergepartial
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 1 Data size: 922 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string), _col2 (type: bigint)
+ outputColumnNames: _col0, _col2
+ Statistics: Num rows: 1 Data size: 922 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: _col2 is not null (type: boolean)
+ Statistics: Num rows: 1 Data size: 922 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string), _col2 (type: bigint)
+ outputColumnNames: _col1, _col2
+ Statistics: Num rows: 1 Data size: 922 Basic stats: COMPLETE Column stats: NONE
+ Group By Operator
+ keys: _col1 (type: string), _col2 (type: bigint)
+ minReductionHashAggr: 0.99
+ mode: hash
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 1 Data size: 922 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: string), _col1 (type: bigint)
+ null sort order: zz
+ sort order: ++
+ Map-reduce partition columns: _col0 (type: string), _col1 (type: bigint)
+ Statistics: Num rows: 1 Data size: 922 Basic stats: COMPLETE Column stats: NONE
+ Reducer 8
+ Execution mode: vectorized, llap
+ Reduce Operator Tree:
+ Group By Operator
+ keys: KEY._col0 (type: string), KEY._col1 (type: bigint)
+ mode: mergepartial
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 1 Data size: 922 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col1 (type: bigint), true (type: boolean), _col0 (type: string)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 1 Data size: 922 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col2 (type: string), _col0 (type: bigint)
+ null sort order: zz
+ sort order: ++
+ Map-reduce partition columns: _col2 (type: string), _col0 (type: bigint)
+ Statistics: Num rows: 1 Data size: 922 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col1 (type: boolean)
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+Warning: Map Join MAPJOIN[133][bigTable=?] in task 'Map 1' is a cross product
+Warning: Map Join MAPJOIN[134][bigTable=?] in task 'Map 5' is a cross product
+Warning: Map Join MAPJOIN[135][bigTable=?] in task 'Reducer 6' is a cross product
+PREHOOK: query: select key, value, count(*)
+from src_null_n4 b
+where NOT EXISTS (select key from src_null_n4 where src_null_n4.value <> b.value)
+group by key, value
+having count(*) not in (select count(*) from src_null_n4 s1 where s1.key > '9' and s1.value <> b.value group by s1.key )
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src_null_n4
+#### A masked pattern was here ####
+POSTHOOK: query: select key, value, count(*)
+from src_null_n4 b
+where NOT EXISTS (select key from src_null_n4 where src_null_n4.value <> b.value)
+group by key, value
+having count(*) not in (select count(*) from src_null_n4 s1 where s1.key > '9' and s1.value <> b.value group by s1.key )
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src_null_n4
+#### A masked pattern was here ####
+5444 NULL 1
Warning: Map Join MAPJOIN[133][bigTable=?] in task 'Map 1' is a cross product
Warning: Map Join MAPJOIN[134][bigTable=?] in task 'Map 5' is a cross product
Warning: Map Join MAPJOIN[135][bigTable=?] in task 'Reducer 6' is a cross product
diff --git a/ql/src/test/results/clientpositive/llap/subquery_multi.q.out b/ql/src/test/results/clientpositive/llap/subquery_multi.q.out
index 95539e5..be03281 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_multi.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_multi.q.out
@@ -4457,6 +4457,1085 @@ STAGE PLANS:
#### A masked pattern was here ####
Edges:
Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
+ Reducer 3 <- Map 5 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: ws1
+ filterExpr: p_type is not null (type: boolean)
+ Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator
+ predicate: p_type is not null (type: boolean)
+ Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
+ Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col4 (type: string)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col4 (type: string)
+ Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
+ Execution mode: vectorized, llap
+ LLAP IO: all inputs
+ Map 4
+ Map Operator Tree:
+ TableScan
+ alias: ws2
+ filterExpr: (p_type is not null and p_retailprice is not null) (type: boolean)
+ Statistics: Num rows: 26 Data size: 2912 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator
+ predicate: (p_type is not null and p_retailprice is not null) (type: boolean)
+ Statistics: Num rows: 26 Data size: 2912 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: p_type (type: string), p_retailprice (type: double)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 26 Data size: 2912 Basic stats: COMPLETE Column stats: COMPLETE
+ Group By Operator
+ keys: _col0 (type: string), _col1 (type: double)
+ minReductionHashAggr: 0.0
+ mode: hash
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 13 Data size: 1456 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)
+ Statistics: Num rows: 13 Data size: 1456 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col1 (type: double)
+ Execution mode: vectorized, llap
+ LLAP IO: all inputs
+ Map 5
+ Map Operator Tree:
+ TableScan
+ alias: wr1
+ filterExpr: p_name is not null (type: boolean)
+ Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: p_name is not null (type: boolean)
+ Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: p_name (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE
+ Group By Operator
+ keys: _col0 (type: string)
+ minReductionHashAggr: 0.99
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
+ Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE
+ Execution mode: vectorized, llap
+ LLAP IO: all inputs
+ Reducer 2
+ Execution mode: llap
+ Reduce Operator Tree:
+ Merge Join Operator
+ condition map:
+ Left Semi Join 0 to 1
+ keys:
+ 0 _col4 (type: string)
+ 1 _col0 (type: string)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10
+ residual filter predicates: {(_col7 <> _col10)}
+ Statistics: Num rows: 14 Data size: 8778 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
+ Statistics: Num rows: 14 Data size: 8666 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col4 (type: string)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col4 (type: string)
+ Statistics: Num rows: 14 Data size: 8666 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
+ Reducer 3
+ Execution mode: llap
+ Reduce Operator Tree:
+ Merge Join Operator
+ condition map:
+ Anti Join 0 to 1
+ keys:
+ 0 _col4 (type: string)
+ 1 _col0 (type: string)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
+ Statistics: Num rows: 15 Data size: 9532 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 15 Data size: 9532 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 * from part ws1 where
+ exists (select * from part ws2 where ws1.p_type= ws2.p_type
+ and ws1.p_retailprice <> ws2.p_retailprice)
+ and not exists(select * from part_null wr1 where ws1.p_type = wr1.p_name)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part
+PREHOOK: Input: default@part_null
+#### A masked pattern was here ####
+POSTHOOK: query: select * from part ws1 where
+ exists (select * from part ws2 where ws1.p_type= ws2.p_type
+ and ws1.p_retailprice <> ws2.p_retailprice)
+ and not exists(select * from part_null wr1 where ws1.p_type = wr1.p_name)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part
+POSTHOOK: Input: default@part_null
+#### A masked pattern was here ####
+192697 almond antique blue firebrick mint Manufacturer#5 Brand#52 MEDIUM BURNISHED TIN 31 LG DRUM 1789.69 ickly ir
+90681 almond antique chartreuse khaki white Manufacturer#3 Brand#31 MEDIUM BURNISHED TIN 17 SM CASE 1671.68 are slyly after the sl
+Warning: Shuffle Join MERGEJOIN[41][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+PREHOOK: query: explain select * from part_null where p_size IN (select p_size from part_null) AND NOT EXISTS (select c from tempty)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part_null
+PREHOOK: Input: default@tempty
+#### A masked pattern was here ####
+POSTHOOK: query: explain select * from part_null where p_size IN (select p_size from part_null) AND NOT EXISTS (select c from tempty)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part_null
+POSTHOOK: Input: default@tempty
+#### A masked pattern was here ####
+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), Reducer 4 (SIMPLE_EDGE)
+ Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE), Reducer 6 (CUSTOM_SIMPLE_EDGE)
+ Reducer 4 <- Map 1 (SIMPLE_EDGE)
+ Reducer 6 <- Map 5 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: part_null
+ filterExpr: p_size is not null (type: boolean)
+ Statistics: Num rows: 1 Data size: 1120 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: p_size is not null (type: boolean)
+ Statistics: Num rows: 1 Data size: 1120 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
+ Statistics: Num rows: 1 Data size: 1120 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col5 (type: int)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col5 (type: int)
+ Statistics: Num rows: 1 Data size: 1120 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col6 (type: string), _col7 (type: double), _col8 (type: string)
+ Group By Operator
+ keys: p_size (type: int)
+ minReductionHashAggr: 0.99
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: int)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+ Execution mode: vectorized, llap
+ LLAP IO: all inputs
+ Map 5
+ Map Operator Tree:
+ TableScan
+ alias: tempty
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+ Select Operator
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+ Group By Operator
+ keys: true (type: boolean)
+ minReductionHashAggr: 0.99
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: boolean)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: boolean)
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+ Execution mode: vectorized, llap
+ LLAP IO: all inputs
+ Reducer 2
+ Execution mode: llap
+ Reduce Operator Tree:
+ Merge Join Operator
+ condition map:
+ Inner Join 0 to 1
+ keys:
+ 0 _col5 (type: int)
+ 1 _col0 (type: int)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
+ Statistics: Num rows: 1 Data size: 1232 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ null sort order:
+ sort order:
+ Statistics: Num rows: 1 Data size: 1232 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
+ Reducer 3
+ Execution mode: llap
+ Reduce Operator Tree:
+ Merge Join Operator
+ condition map:
+ Left Outer Join 0 to 1
+ keys:
+ 0
+ 1
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10
+ Statistics: Num rows: 1 Data size: 1233 Basic stats: PARTIAL Column stats: NONE
+ Filter Operator
+ predicate: _col10 is null (type: boolean)
+ Statistics: Num rows: 1 Data size: 1233 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
+ Statistics: Num rows: 1 Data size: 1233 Basic stats: PARTIAL Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 1233 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
+ Reducer 4
+ Execution mode: vectorized, llap
+ Reduce Operator Tree:
+ Group By Operator
+ keys: KEY._col0 (type: int)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: int)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
+ Reducer 6
+ Execution mode: vectorized, llap
+ Reduce Operator Tree:
+ Group By Operator
+ keys: KEY._col0 (type: boolean)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+ Reduce Output Operator
+ null sort order:
+ sort order:
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+ value expressions: _col0 (type: boolean)
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+Warning: Shuffle Join MERGEJOIN[41][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+PREHOOK: query: select * from part_null where p_size IN (select p_size from part_null) AND NOT EXISTS (select c from tempty)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part_null
+PREHOOK: Input: default@tempty
+#### A masked pattern was here ####
+POSTHOOK: query: select * from part_null where p_size IN (select p_size from part_null) AND NOT EXISTS (select c from tempty)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part_null
+POSTHOOK: Input: default@tempty
+#### A masked pattern was here ####
+105685 almond antique violet chocolate turquoise Manufacturer#2 Brand#22 MEDIUM ANODIZED COPPER 14 MED CAN 1690.68 ly pending requ
+110592 almond antique salmon chartreuse burlywood Manufacturer#1 Brand#15 PROMO BURNISHED NICKEL 6 JUMBO PKG 1602.59 to the furiously
+112398 almond antique metallic orange dim Manufacturer#3 Brand#32 MEDIUM BURNISHED BRASS 19 JUMBO JAR 1410.39 ole car
+121152 almond antique burnished rose metallic Manufacturer#1 Brand#14 PROMO PLATED TIN 2 JUMBO BOX 1173.15 e pinto beans h
+121152 almond antique burnished rose metallic Manufacturer#1 Brand#14 PROMO PLATED TIN 2 JUMBO BOX 1173.15 e pinto beans h
+132666 almond aquamarine rose maroon antique Manufacturer#2 Brand#24 SMALL POLISHED NICKEL 25 MED BOX 1698.66 even
+144293 almond antique olive coral navajo Manufacturer#3 Brand#34 STANDARD POLISHED STEEL 45 JUMBO CAN 1337.29 ag furiously about
+146985 almond aquamarine midnight light salmon Manufacturer#2 Brand#23 MEDIUM BURNISHED COPPER 2 SM CASE 2031.98 s cajole caref
+15103 almond aquamarine dodger light gainsboro Manufacturer#5 Brand#53 ECONOMY BURNISHED STEEL 46 LG PACK 1018.1 packages hinder carefu
+155733 almond antique sky peru orange Manufacturer#5 Brand#53 SMALL PLATED BRASS 2 WRAP DRUM 1788.73 furiously. bra
+17273 almond antique forest lavender goldenrod Manufacturer#3 Brand#35 PROMO ANODIZED TIN 14 JUMBO CASE 1190.27 along the
+17927 almond aquamarine yellow dodger mint Manufacturer#4 Brand#41 ECONOMY BRUSHED COPPER 7 SM PKG 1844.92 ites. eve
+191709 almond antique violet turquoise frosted Manufacturer#2 Brand#22 ECONOMY POLISHED STEEL 40 MED BOX 1800.7 haggle
+192697 almond antique blue firebrick mint Manufacturer#5 Brand#52 MEDIUM BURNISHED TIN 31 LG DRUM 1789.69 ickly ir
+195606 almond aquamarine sandy cyan gainsboro Manufacturer#2 Brand#25 STANDARD PLATED TIN 18 SM PKG 1701.6 ic de
+33357 almond azure aquamarine papaya violet Manufacturer#4 Brand#41 STANDARD ANODIZED TIN 12 WRAP CASE 1290.35 reful
+40982 almond antique misty red olive Manufacturer#3 Brand#32 ECONOMY PLATED COPPER 1 LG PKG 1922.98 c foxes can s
+42669 almond antique medium spring khaki Manufacturer#5 Brand#51 STANDARD BURNISHED TIN 6 MED CAN 1611.66 sits haggl
+45261 almond aquamarine floral ivory bisque Manufacturer#4 Brand#42 SMALL PLATED STEEL 27 WRAP CASE 1206.26 careful
+48427 almond antique violet mint lemon Manufacturer#4 Brand#42 PROMO POLISHED STEEL 39 SM CASE 1375.42 hely ironic i
+49671 almond antique gainsboro frosted violet Manufacturer#4 Brand#41 SMALL BRUSHED BRASS 10 SM BOX 1620.67 ccounts run quick
+65667 almond aquamarine pink moccasin thistle Manufacturer#1 Brand#12 LARGE BURNISHED STEEL 42 JUMBO CASE 1632.66 e across the expr
+78486 almond azure blanched chiffon midnight Manufacturer#5 Brand#52 LARGE BRUSHED BRASS 23 MED BAG 1464.48 hely blith
+78487 NULL Manufacturer#6 Brand#52 LARGE BRUSHED BRASS 23 MED BAG 1464.48 hely blith
+85768 almond antique chartreuse lavender yellow Manufacturer#1 Brand#12 LARGE BRUSHED STEEL 34 SM BAG 1753.76 refull
+86428 almond aquamarine burnished black steel Manufacturer#1 Brand#12 STANDARD ANODIZED STEEL 28 WRAP BAG 1414.42 arefully
+90681 almond antique chartreuse khaki white Manufacturer#3 Brand#31 MEDIUM BURNISHED TIN 17 SM CASE 1671.68 are slyly after the sl
+Warning: Shuffle Join MERGEJOIN[41][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+PREHOOK: query: explain select * from part_null where p_name IN (select p_name from part_null) AND NOT EXISTS (select c from tempty)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part_null
+PREHOOK: Input: default@tempty
+#### A masked pattern was here ####
+POSTHOOK: query: explain select * from part_null where p_name IN (select p_name from part_null) AND NOT EXISTS (select c from tempty)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part_null
+POSTHOOK: Input: default@tempty
+#### A masked pattern was here ####
+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), Reducer 4 (SIMPLE_EDGE)
+ Reducer 3 <- Reducer 2 (CUSTOM_SIMPLE_EDGE), Reducer 6 (CUSTOM_SIMPLE_EDGE)
+ Reducer 4 <- Map 1 (SIMPLE_EDGE)
+ Reducer 6 <- Map 5 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: part_null
+ filterExpr: p_name is not null (type: boolean)
+ Statistics: Num rows: 1 Data size: 1120 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: p_name is not null (type: boolean)
+ Statistics: Num rows: 1 Data size: 1120 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
+ Statistics: Num rows: 1 Data size: 1120 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col1 (type: string)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col1 (type: string)
+ Statistics: Num rows: 1 Data size: 1120 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: int), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
+ Group By Operator
+ keys: p_name (type: string)
+ minReductionHashAggr: 0.99
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
+ Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE
+ Execution mode: vectorized, llap
+ LLAP IO: all inputs
+ Map 5
+ Map Operator Tree:
+ TableScan
+ alias: tempty
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+ Select Operator
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+ Group By Operator
+ keys: true (type: boolean)
+ minReductionHashAggr: 0.99
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: boolean)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: boolean)
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+ Execution mode: vectorized, llap
+ LLAP IO: all inputs
+ Reducer 2
+ Execution mode: llap
+ Reduce Operator Tree:
+ Merge Join Operator
+ condition map:
+ Inner Join 0 to 1
+ keys:
+ 0 _col1 (type: string)
+ 1 _col0 (type: string)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
+ Statistics: Num rows: 1 Data size: 1232 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ null sort order:
+ sort order:
+ Statistics: Num rows: 1 Data size: 1232 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
+ Reducer 3
+ Execution mode: llap
+ Reduce Operator Tree:
+ Merge Join Operator
+ condition map:
+ Left Outer Join 0 to 1
+ keys:
+ 0
+ 1
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10
+ Statistics: Num rows: 1 Data size: 1233 Basic stats: PARTIAL Column stats: NONE
+ Filter Operator
+ predicate: _col10 is null (type: boolean)
+ Statistics: Num rows: 1 Data size: 1233 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
+ Statistics: Num rows: 1 Data size: 1233 Basic stats: PARTIAL Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 1233 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
+ Reducer 4
+ Execution mode: vectorized, llap
+ Reduce Operator Tree:
+ Group By Operator
+ keys: KEY._col0 (type: string)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
+ Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE
+ Reducer 6
+ Execution mode: vectorized, llap
+ Reduce Operator Tree:
+ Group By Operator
+ keys: KEY._col0 (type: boolean)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+ Reduce Output Operator
+ null sort order:
+ sort order:
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+ value expressions: _col0 (type: boolean)
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+Warning: Shuffle Join MERGEJOIN[41][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Reducer 3' is a cross product
+PREHOOK: query: select * from part_null where p_name IN (select p_name from part_null) AND NOT EXISTS (select c from tempty)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part_null
+PREHOOK: Input: default@tempty
+#### A masked pattern was here ####
+POSTHOOK: query: select * from part_null where p_name IN (select p_name from part_null) AND NOT EXISTS (select c from tempty)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part_null
+POSTHOOK: Input: default@tempty
+#### A masked pattern was here ####
+105685 almond antique violet chocolate turquoise Manufacturer#2 Brand#22 MEDIUM ANODIZED COPPER 14 MED CAN 1690.68 ly pending requ
+110592 almond antique salmon chartreuse burlywood Manufacturer#1 Brand#15 PROMO BURNISHED NICKEL 6 JUMBO PKG 1602.59 to the furiously
+112398 almond antique metallic orange dim Manufacturer#3 Brand#32 MEDIUM BURNISHED BRASS 19 JUMBO JAR 1410.39 ole car
+121152 almond antique burnished rose metallic Manufacturer#1 Brand#14 PROMO PLATED TIN 2 JUMBO BOX 1173.15 e pinto beans h
+121152 almond antique burnished rose metallic Manufacturer#1 Brand#14 PROMO PLATED TIN 2 JUMBO BOX 1173.15 e pinto beans h
+132666 almond aquamarine rose maroon antique Manufacturer#2 Brand#24 SMALL POLISHED NICKEL 25 MED BOX 1698.66 even
+144293 almond antique olive coral navajo Manufacturer#3 Brand#34 STANDARD POLISHED STEEL 45 JUMBO CAN 1337.29 ag furiously about
+146985 almond aquamarine midnight light salmon Manufacturer#2 Brand#23 MEDIUM BURNISHED COPPER 2 SM CASE 2031.98 s cajole caref
+15103 almond aquamarine dodger light gainsboro Manufacturer#5 Brand#53 ECONOMY BURNISHED STEEL 46 LG PACK 1018.1 packages hinder carefu
+155733 almond antique sky peru orange Manufacturer#5 Brand#53 SMALL PLATED BRASS 2 WRAP DRUM 1788.73 furiously. bra
+17273 almond antique forest lavender goldenrod Manufacturer#3 Brand#35 PROMO ANODIZED TIN 14 JUMBO CASE 1190.27 along the
+17927 almond aquamarine yellow dodger mint Manufacturer#4 Brand#41 ECONOMY BRUSHED COPPER 7 SM PKG 1844.92 ites. eve
+191709 almond antique violet turquoise frosted Manufacturer#2 Brand#22 ECONOMY POLISHED STEEL 40 MED BOX 1800.7 haggle
+192697 almond antique blue firebrick mint Manufacturer#5 Brand#52 MEDIUM BURNISHED TIN 31 LG DRUM 1789.69 ickly ir
+195606 almond aquamarine sandy cyan gainsboro Manufacturer#2 Brand#25 STANDARD PLATED TIN 18 SM PKG 1701.6 ic de
+33357 almond azure aquamarine papaya violet Manufacturer#4 Brand#41 STANDARD ANODIZED TIN 12 WRAP CASE 1290.35 reful
+40982 almond antique misty red olive Manufacturer#3 Brand#32 ECONOMY PLATED COPPER 1 LG PKG 1922.98 c foxes can s
+42669 almond antique medium spring khaki Manufacturer#5 Brand#51 STANDARD BURNISHED TIN 6 MED CAN 1611.66 sits haggl
+45261 almond aquamarine floral ivory bisque Manufacturer#4 Brand#42 SMALL PLATED STEEL 27 WRAP CASE 1206.26 careful
+48427 almond antique violet mint lemon Manufacturer#4 Brand#42 PROMO POLISHED STEEL 39 SM CASE 1375.42 hely ironic i
+49671 almond antique gainsboro frosted violet Manufacturer#4 Brand#41 SMALL BRUSHED BRASS 10 SM BOX 1620.67 ccounts run quick
+65667 almond aquamarine pink moccasin thistle Manufacturer#1 Brand#12 LARGE BURNISHED STEEL 42 JUMBO CASE 1632.66 e across the expr
+78486 almond azure blanched chiffon midnight Manufacturer#5 Brand#52 LARGE BRUSHED BRASS 23 MED BAG 1464.48 hely blith
+85768 almond antique chartreuse lavender yellow Manufacturer#1 Brand#12 LARGE BRUSHED STEEL 34 SM BAG 1753.76 refull
+86428 almond aquamarine burnished black steel Manufacturer#1 Brand#12 STANDARD ANODIZED STEEL 28 WRAP BAG 1414.42 arefully
+90681 almond antique chartreuse khaki white Manufacturer#3 Brand#31 MEDIUM BURNISHED TIN 17 SM CASE 1671.68 are slyly after the sl
+PREHOOK: query: explain
+select key, value, count(*)
+from src b
+where b.key in (select key from src where src.value = b.value)
+group by key, value
+having count(*) in (select count(*) from src s1 where s1.key > '9' and not exists (select * from src s2 where s1.value = s2.value) group by s1.key )
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: explain
+select key, value, count(*)
+from src b
+where b.key in (select key from src where src.value = b.value)
+group by key, value
+having count(*) in (select count(*) from src s1 where s1.key > '9' and not exists (select * from src s2 where s1.value = s2.value) group by s1.key )
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+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), Map 8 (SIMPLE_EDGE)
+ Reducer 3 <- Reducer 2 (SIMPLE_EDGE)
+ Reducer 4 <- Reducer 3 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
+ Reducer 5 <- Map 1 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE)
+ Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
+ Reducer 7 <- Map 1 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: b
+ filterExpr: ((value is not null and key is not null) or (key > '9') or value is not null) (type: boolean)
+ Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator
+ predicate: (value is not null and key is not null) (type: boolean)
+ Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+ 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: zz
+ sort order: ++
+ Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
+ Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator
+ predicate: (key > '9') (type: boolean)
+ Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: key (type: string), value (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col1 (type: string)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col1 (type: string)
+ Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col0 (type: string)
+ Filter Operator
+ predicate: value is not null (type: boolean)
+ Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE
+ Group By Operator
+ keys: value (type: string)
+ minReductionHashAggr: 0.5
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 250 Data size: 22750 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)
+ Statistics: Num rows: 250 Data size: 22750 Basic stats: COMPLETE Column stats: COMPLETE
+ Execution mode: vectorized, llap
+ LLAP IO: all inputs
+ Map 8
+ Map Operator Tree:
+ TableScan
+ alias: src
+ filterExpr: (value is not null and key is not null) (type: boolean)
+ Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator
+ predicate: (value is not null and key is not null) (type: boolean)
+ Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+ 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
+ Group By Operator
+ keys: _col0 (type: string), _col1 (type: string)
+ minReductionHashAggr: 0.0
+ mode: hash
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 250 Data size: 44500 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: string), _col1 (type: string)
+ null sort order: zz
+ sort order: ++
+ Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
+ Statistics: Num rows: 250 Data size: 44500 Basic stats: COMPLETE Column stats: COMPLETE
+ Execution mode: vectorized, llap
+ LLAP IO: all inputs
+ Reducer 2
+ Execution mode: llap
+ Reduce Operator Tree:
+ Merge Join Operator
+ condition map:
+ Left Semi Join 0 to 1
+ keys:
+ 0 _col0 (type: string), _col1 (type: string)
+ 1 _col0 (type: string), _col1 (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 395 Data size: 70310 Basic stats: COMPLETE Column stats: COMPLETE
+ Group By Operator
+ aggregations: count()
+ keys: _col0 (type: string), _col1 (type: string)
+ minReductionHashAggr: 0.0
+ mode: hash
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 197 Data size: 36642 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: string), _col1 (type: string)
+ null sort order: zz
+ sort order: ++
+ Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
+ Statistics: Num rows: 197 Data size: 36642 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col2 (type: bigint)
+ Reducer 3
+ Execution mode: vectorized, llap
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations: count(VALUE._col0)
+ keys: KEY._col0 (type: string), KEY._col1 (type: string)
+ mode: mergepartial
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 197 Data size: 36642 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator
+ predicate: _col2 is not null (type: boolean)
+ Statistics: Num rows: 197 Data size: 36642 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col2 (type: bigint)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col2 (type: bigint)
+ Statistics: Num rows: 197 Data size: 36642 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col0 (type: string), _col1 (type: string)
+ Reducer 4
+ Execution mode: llap
+ Reduce Operator Tree:
+ Merge Join Operator
+ condition map:
+ Left Semi Join 0 to 1
+ keys:
+ 0 _col2 (type: bigint)
+ 1 _col0 (type: bigint)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 21 Data size: 3906 Basic stats: COMPLETE Column stats: COMPLETE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 21 Data size: 3906 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
+ Reducer 5
+ Execution mode: llap
+ Reduce Operator Tree:
+ Merge Join Operator
+ condition map:
+ Left Outer Join 0 to 1
+ keys:
+ 0 _col1 (type: string)
+ 1 _col1 (type: string)
+ outputColumnNames: _col0, _col2
+ Statistics: Num rows: 250 Data size: 22418 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator
+ predicate: _col2 is null (type: boolean)
+ Statistics: Num rows: 84 Data size: 7536 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: _col0 (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 84 Data size: 7536 Basic stats: COMPLETE Column stats: COMPLETE
+ Group By Operator
+ aggregations: count()
+ keys: _col0 (type: string)
+ minReductionHashAggr: 0.5
+ mode: hash
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 42 Data size: 3990 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)
+ Statistics: Num rows: 42 Data size: 3990 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col1 (type: bigint)
+ Reducer 6
+ Execution mode: vectorized, llap
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations: count(VALUE._col0)
+ keys: KEY._col0 (type: string)
+ mode: mergepartial
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 42 Data size: 3990 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: _col1 (type: bigint)
+ outputColumnNames: _col1
+ Statistics: Num rows: 42 Data size: 336 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator
+ predicate: _col1 is not null (type: boolean)
+ Statistics: Num rows: 42 Data size: 336 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: _col1 (type: bigint)
+ outputColumnNames: _col0
+ Statistics: Num rows: 42 Data size: 336 Basic stats: COMPLETE Column stats: COMPLETE
+ Group By Operator
+ keys: _col0 (type: bigint)
+ minReductionHashAggr: 0.5
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 21 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: bigint)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: bigint)
+ Statistics: Num rows: 21 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE
+ Reducer 7
+ Execution mode: vectorized, llap
+ Reduce Operator Tree:
+ Group By Operator
+ keys: KEY._col0 (type: string)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 250 Data size: 22750 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: true (type: boolean), _col0 (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 250 Data size: 23750 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col1 (type: string)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col1 (type: string)
+ Statistics: Num rows: 250 Data size: 23750 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col0 (type: boolean)
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: select key, value, count(*)
+from src b
+where b.key in (select key from src where src.value = b.value)
+group by key, value
+having count(*) in (select count(*) from src s1 where s1.key > '9' and not exists (select * from src s2 where s1.value = s2.value) group by s1.key )
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: select key, value, count(*)
+from src b
+where b.key in (select key from src where src.value = b.value)
+group by key, value
+having count(*) in (select count(*) from src s1 where s1.key > '9' and not exists (select * from src s2 where s1.value = s2.value) group by s1.key )
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+Warning: Shuffle Join MERGEJOIN[47][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+Warning: Shuffle Join MERGEJOIN[49][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product
+PREHOOK: query: explain select count(*) from src
+ where src.key in (select key from src s1 where s1.key > '9')
+ or src.value is not null
+ or not exists(select key from src)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: explain select count(*) from src
+ where src.key in (select key from src s1 where s1.key > '9')
+ or src.value is not null
+ or not exists(select key from src)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+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 (XPROD_EDGE), Reducer 6 (XPROD_EDGE)
+ Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE)
+ Reducer 4 <- Reducer 3 (CUSTOM_SIMPLE_EDGE), Reducer 8 (CUSTOM_SIMPLE_EDGE)
+ Reducer 5 <- Reducer 4 (CUSTOM_SIMPLE_EDGE)
+ Reducer 6 <- Map 1 (CUSTOM_SIMPLE_EDGE)
+ Reducer 7 <- Map 1 (SIMPLE_EDGE)
+ Reducer 8 <- Map 1 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: src
+ Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+ 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
+ null sort order:
+ sort order:
+ Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col0 (type: string), _col1 (type: string)
+ Filter Operator
+ predicate: (key > '9') (type: boolean)
+ Statistics: Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ Statistics: Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE
+ Group By Operator
+ aggregations: count()
+ minReductionHashAggr: 0.99
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ null sort order:
+ sort order:
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col0 (type: bigint)
+ Filter Operator
+ predicate: (key > '9') (type: boolean)
+ Statistics: Num rows: 166 Data size: 14442 Basic stats: COMPLETE Column stats: COMPLETE
+ Group By Operator
+ keys: key (type: string)
+ minReductionHashAggr: 0.5
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 83 Data size: 7221 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)
+ Statistics: Num rows: 83 Data size: 7221 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
+ Group By Operator
+ keys: true (type: boolean)
+ minReductionHashAggr: 0.99
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: boolean)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: boolean)
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ Execution mode: vectorized, llap
+ LLAP IO: all inputs
+ Reducer 2
+ Execution mode: llap
+ Reduce Operator Tree:
+ Merge Join Operator
+ condition map:
+ Inner Join 0 to 1
+ keys:
+ 0
+ 1
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 500 Data size: 93000 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)
+ Statistics: Num rows: 500 Data size: 93000 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col1 (type: string), _col2 (type: bigint)
+ Reducer 3
+ Execution mode: llap
+ Reduce Operator Tree:
+ Merge Join Operator
+ condition map:
+ Left Outer Join 0 to 1
+ keys:
+ 0 _col0 (type: string)
+ 1 _col0 (type: string)
+ outputColumnNames: _col1, _col2, _col4
+ Statistics: Num rows: 631 Data size: 62997 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ null sort order:
+ sort order:
+ Statistics: Num rows: 631 Data size: 62997 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col1 (type: string), _col2 (type: bigint), _col4 (type: boolean)
+ Reducer 4
+ Execution mode: llap
+ Reduce Operator Tree:
+ Merge Join Operator
+ condition map:
+ Left Outer Join 0 to 1
+ keys:
+ 0
+ 1
+ outputColumnNames: _col1, _col2, _col4, _col5
+ Statistics: Num rows: 631 Data size: 65521 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator
+ predicate: (_col5 is null or ((_col2 <> 0L) and _col4 is not null) or _col1 is not null) (type: boolean)
+ Statistics: Num rows: 631 Data size: 65521 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ Statistics: Num rows: 631 Data size: 65521 Basic stats: COMPLETE Column stats: COMPLETE
+ Group By Operator
+ aggregations: count()
+ minReductionHashAggr: 0.99
+ mode: hash
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ null sort order:
+ sort order:
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col0 (type: bigint)
+ Reducer 5
+ Execution mode: vectorized, llap
+ 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
+ Reducer 6
+ Execution mode: vectorized, llap
+ 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
+ Reduce Output Operator
+ null sort order:
+ sort order:
+ Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col0 (type: bigint)
+ Reducer 7
+ Execution mode: vectorized, llap
+ Reduce Operator Tree:
+ Group By Operator
+ keys: KEY._col0 (type: string)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 83 Data size: 7221 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: _col0 (type: string), true (type: boolean)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 83 Data size: 7553 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)
+ Statistics: Num rows: 83 Data size: 7553 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col1 (type: boolean)
+ Reducer 8
+ Execution mode: vectorized, llap
+ Reduce Operator Tree:
+ Group By Operator
+ keys: KEY._col0 (type: boolean)
+ mode: mergepartial
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ null sort order:
+ sort order:
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col0 (type: boolean)
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+Warning: Shuffle Join MERGEJOIN[47][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+Warning: Shuffle Join MERGEJOIN[49][tables = [$hdt$_0, $hdt$_1, $hdt$_2, $hdt$_3]] in Stage 'Reducer 4' is a cross product
+PREHOOK: query: select count(*) from src
+ where src.key in (select key from src s1 where s1.key > '9')
+ or src.value is not null
+ or not exists(select key from src)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+POSTHOOK: query: select count(*) from src
+ where src.key in (select key from src s1 where s1.key > '9')
+ or src.value is not null
+ or not exists(select key from src)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+#### A masked pattern was here ####
+500
+PREHOOK: query: explain select * from part ws1 where
+ exists (select * from part ws2 where ws1.p_type= ws2.p_type
+ and ws1.p_retailprice <> ws2.p_retailprice)
+ and not exists(select * from part_null wr1 where ws1.p_type = wr1.p_name)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part
+PREHOOK: Input: default@part_null
+#### A masked pattern was here ####
+POSTHOOK: query: explain select * from part ws1 where
+ exists (select * from part ws2 where ws1.p_type= ws2.p_type
+ and ws1.p_retailprice <> ws2.p_retailprice)
+ and not exists(select * from part_null wr1 where ws1.p_type = wr1.p_name)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part
+POSTHOOK: Input: default@part_null
+#### A masked pattern was here ####
+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), Map 4 (SIMPLE_EDGE)
Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
Reducer 6 <- Map 5 (SIMPLE_EDGE)
#### A masked pattern was here ####
diff --git a/ql/src/test/results/clientpositive/llap/subquery_notexists.q.out b/ql/src/test/results/clientpositive/llap/subquery_notexists.q.out
index 226c00b..faea4f4 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_notexists.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_notexists.q.out
@@ -29,8 +29,7 @@ STAGE PLANS:
Tez
#### A masked pattern was here ####
Edges:
- Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
- Reducer 3 <- Map 1 (SIMPLE_EDGE)
+ Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 3 (SIMPLE_EDGE)
#### A masked pattern was here ####
Vertices:
Map 1
@@ -48,21 +47,33 @@ STAGE PLANS:
sort order: ++
Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
+ Execution mode: vectorized, llap
+ LLAP IO: all inputs
+ Map 3
+ Map Operator Tree:
+ TableScan
+ alias: a
+ filterExpr: ((value > 'val_2') and key is not null) (type: boolean)
+ Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
Filter Operator
predicate: ((value > 'val_2') and key is not null) (type: boolean)
Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
- Group By Operator
- keys: key (type: string), value (type: string)
- minReductionHashAggr: 0.0
- mode: hash
+ Select Operator
+ expressions: key (type: string), value (type: string)
outputColumnNames: _col0, _col1
- Statistics: Num rows: 83 Data size: 14774 Basic stats: COMPLETE Column stats: COMPLETE
- Reduce Output Operator
- key expressions: _col0 (type: string), _col1 (type: string)
- null sort order: zz
- sort order: ++
- Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
+ Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+ Group By Operator
+ keys: _col0 (type: string), _col1 (type: string)
+ minReductionHashAggr: 0.0
+ mode: hash
+ outputColumnNames: _col0, _col1
Statistics: Num rows: 83 Data size: 14774 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: string), _col1 (type: string)
+ null sort order: zz
+ sort order: ++
+ Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
+ Statistics: Num rows: 83 Data size: 14774 Basic stats: COMPLETE Column stats: COMPLETE
Execution mode: vectorized, llap
LLAP IO: all inputs
Reducer 2
@@ -70,45 +81,19 @@ STAGE PLANS:
Reduce Operator Tree:
Merge Join Operator
condition map:
- Left Outer Join 0 to 1
+ Anti Join 0 to 1
keys:
0 _col0 (type: string), _col1 (type: string)
- 1 _col1 (type: string), _col2 (type: string)
- outputColumnNames: _col0, _col1, _col2
- Statistics: Num rows: 631 Data size: 112846 Basic stats: COMPLETE Column stats: COMPLETE
- Filter Operator
- predicate: _col2 is null (type: boolean)
- Statistics: Num rows: 500 Data size: 89420 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: _col0 (type: string), _col1 (type: string)
- outputColumnNames: _col0, _col1
- Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
- File Output Operator
- compressed: false
- Statistics: Num rows: 500 Data size: 89000 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
- Reducer 3
- Execution mode: vectorized, llap
- Reduce Operator Tree:
- Group By Operator
- keys: KEY._col0 (type: string), KEY._col1 (type: string)
- mode: mergepartial
+ 1 _col0 (type: string), _col1 (type: string)
outputColumnNames: _col0, _col1
- Statistics: Num rows: 83 Data size: 14774 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: true (type: boolean), _col0 (type: string), _col1 (type: string)
- outputColumnNames: _col0, _col1, _col2
- Statistics: Num rows: 83 Data size: 15106 Basic stats: COMPLETE Column stats: COMPLETE
- Reduce Output Operator
- key expressions: _col1 (type: string), _col2 (type: string)
- null sort order: zz
- sort order: ++
- Map-reduce partition columns: _col1 (type: string), _col2 (type: string)
- Statistics: Num rows: 83 Data size: 15106 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: _col0 (type: boolean)
+ Statistics: Num rows: 369 Data size: 65682 Basic stats: COMPLETE Column stats: COMPLETE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 369 Data size: 65682 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
@@ -319,7 +304,7 @@ STAGE PLANS:
key expressions: _col0 (type: string), _col1 (type: string)
null sort order: zz
sort order: ++
- Map-reduce partition columns: _col0 (type: string)
+ Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
Statistics: Num rows: 83 Data size: 14774 Basic stats: COMPLETE Column stats: COMPLETE
Execution mode: vectorized, llap
LLAP IO: all inputs
@@ -328,26 +313,19 @@ STAGE PLANS:
Reduce Operator Tree:
Merge Join Operator
condition map:
- Left Outer Join 0 to 1
+ Anti Join 0 to 1
keys:
0 _col1 (type: string)
- 1 _col1 (type: string)
- outputColumnNames: _col0, _col1, _col2
- Statistics: Num rows: 566 Data size: 101016 Basic stats: COMPLETE Column stats: COMPLETE
- Filter Operator
- predicate: _col2 is null (type: boolean)
- Statistics: Num rows: 500 Data size: 89236 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: _col0 (type: string), _col1 (type: string)
- outputColumnNames: _col0, _col1
- Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
- File Output Operator
- compressed: false
- Statistics: Num rows: 500 Data size: 89000 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
+ 1 _col0 (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 434 Data size: 77252 Basic stats: COMPLETE Column stats: COMPLETE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 434 Data size: 77252 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
Reducer 3
Execution mode: vectorized, llap
Reduce Operator Tree:
@@ -358,24 +336,20 @@ STAGE PLANS:
Statistics: Num rows: 83 Data size: 14774 Basic stats: COMPLETE Column stats: COMPLETE
Select Operator
expressions: _col0 (type: string)
- outputColumnNames: _col1
- Statistics: Num rows: 83 Data size: 14774 Basic stats: COMPLETE Column stats: COMPLETE
+ outputColumnNames: _col0
+ Statistics: Num rows: 83 Data size: 7553 Basic stats: COMPLETE Column stats: COMPLETE
Group By Operator
- keys: _col1 (type: string)
- mode: complete
+ keys: _col0 (type: string)
+ minReductionHashAggr: 0.5060241
+ mode: hash
outputColumnNames: _col0
Statistics: Num rows: 41 Data size: 3731 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: true (type: boolean), _col0 (type: string)
- outputColumnNames: _col0, _col1
- Statistics: Num rows: 41 Data size: 3895 Basic stats: COMPLETE Column stats: COMPLETE
- Reduce Output Operator
- key expressions: _col1 (type: string)
- null sort order: z
- sort order: +
- Map-reduce partition columns: _col1 (type: string)
- Statistics: Num rows: 41 Data size: 3895 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: _col0 (type: boolean)
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
+ Statistics: Num rows: 41 Data size: 3731 Basic stats: COMPLETE Column stats: COMPLETE
Stage: Stage-0
Fetch Operator
@@ -522,7 +496,7 @@ POSTHOOK: Input: default@src
199 val_199
199 val_199
2 val_2
-Warning: Shuffle Join MERGEJOIN[17][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+Warning: Shuffle Join MERGEJOIN[15][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
PREHOOK: query: explain
select *
from src b
@@ -554,8 +528,7 @@ STAGE PLANS:
Tez
#### A masked pattern was here ####
Edges:
- Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Reducer 3 (CUSTOM_SIMPLE_EDGE)
- Reducer 3 <- Map 1 (SIMPLE_EDGE)
+ Reducer 2 <- Map 1 (XPROD_EDGE), Map 3 (XPROD_EDGE)
#### A masked pattern was here ####
Vertices:
Map 1
@@ -572,21 +545,32 @@ STAGE PLANS:
sort order:
Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
value expressions: _col0 (type: string), _col1 (type: string)
+ Execution mode: vectorized, llap
+ LLAP IO: all inputs
+ Map 3
+ Map Operator Tree:
+ TableScan
+ alias: a
+ filterExpr: ((value > 'val_2') and key is not null) (type: boolean)
+ Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
Filter Operator
predicate: ((value > 'val_2') and key is not null) (type: boolean)
Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
- Group By Operator
- keys: key (type: string), value (type: string)
- minReductionHashAggr: 0.0
- mode: hash
+ Select Operator
+ expressions: key (type: string), value (type: string)
outputColumnNames: _col0, _col1
- Statistics: Num rows: 83 Data size: 14774 Basic stats: COMPLETE Column stats: COMPLETE
- Reduce Output Operator
- key expressions: _col0 (type: string), _col1 (type: string)
- null sort order: zz
- sort order: ++
- Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
+ Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+ Group By Operator
+ keys: _col0 (type: string), _col1 (type: string)
+ minReductionHashAggr: 0.0
+ mode: hash
+ outputColumnNames: _col0, _col1
Statistics: Num rows: 83 Data size: 14774 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ null sort order:
+ sort order:
+ Statistics: Num rows: 83 Data size: 14774 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col0 (type: string), _col1 (type: string)
Execution mode: vectorized, llap
LLAP IO: all inputs
Reducer 2
@@ -594,44 +578,24 @@ STAGE PLANS:
Reduce Operator Tree:
Merge Join Operator
condition map:
- Left Outer Join 0 to 1
+ Anti Join 0 to 1
keys:
0
1
- outputColumnNames: _col0, _col1, _col2, _col3, _col4
- residual filter predicates: {(_col3 > _col0)} {(_col1 <> _col4)}
- Statistics: Num rows: 41500 Data size: 14940000 Basic stats: COMPLETE Column stats: COMPLETE
- Filter Operator
- predicate: _col2 is null (type: boolean)
- Statistics: Num rows: 1 Data size: 360 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: _col0 (type: string), _col1 (type: string)
- outputColumnNames: _col0, _col1
- Statistics: Num rows: 1 Data size: 178 Basic stats: COMPLETE Column stats: COMPLETE
- File Output Operator
- compressed: false
- Statistics: Num rows: 1 Data size: 178 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
- Reducer 3
- Execution mode: vectorized, llap
- Reduce Operator Tree:
- Group By Operator
- keys: KEY._col0 (type: string), KEY._col1 (type: string)
- mode: mergepartial
- outputColumnNames: _col0, _col1
- Statistics: Num rows: 83 Data size: 14774 Basic stats: COMPLETE Column stats: COMPLETE
+ outputColumnNames: _col0, _col1, _col2, _col3
+ residual filter predicates: {(_col2 > _col0)} {(_col1 <> _col3)}
+ Statistics: Num rows: 13833 Data size: 4924548 Basic stats: COMPLETE Column stats: COMPLETE
Select Operator
- expressions: true (type: boolean), _col0 (type: string), _col1 (type: string)
- outputColumnNames: _col0, _col1, _col2
- Statistics: Num rows: 83 Data size: 15106 Basic stats: COMPLETE Column stats: COMPLETE
- Reduce Output Operator
- null sort order:
- sort order:
- Statistics: Num rows: 83 Data size: 15106 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: _col0 (type: boolean), _col1 (type: string), _col2 (type: string)
+ expressions: _col0 (type: string), _col1 (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 13833 Data size: 2462274 Basic stats: COMPLETE Column stats: COMPLETE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 13833 Data size: 2462274 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
@@ -639,7 +603,7 @@ STAGE PLANS:
Processor Tree:
ListSink
-Warning: Shuffle Join MERGEJOIN[17][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+Warning: Shuffle Join MERGEJOIN[15][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
PREHOOK: query: select *
from src b
where not exists
@@ -662,8 +626,8 @@ POSTHOOK: Input: default@src
#### A masked pattern was here ####
98 val_98
98 val_98
-Warning: Shuffle Join MERGEJOIN[60][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
-Warning: Shuffle Join MERGEJOIN[61][tables = [$hdt$_3, $hdt$_4]] in Stage 'Reducer 6' is a cross product
+Warning: Shuffle Join MERGEJOIN[57][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+Warning: Shuffle Join MERGEJOIN[58][tables = [$hdt$_3, $hdt$_4]] in Stage 'Reducer 5' is a cross product
PREHOOK: query: explain SELECT p1.p_name FROM part p1 LEFT JOIN (select p_type as p_col from part ) p2 WHERE NOT EXISTS
(select pp1.p_type as p_col from part pp1 where pp1.p_partkey = p2.p_col)
PREHOOK: type: QUERY
@@ -683,12 +647,11 @@ STAGE PLANS:
Tez
#### A masked pattern was here ####
Edges:
- Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Map 8 (CUSTOM_SIMPLE_EDGE)
- Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 5 (SIMPLE_EDGE)
- Reducer 4 <- Map 1 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE)
- Reducer 5 <- Reducer 4 (SIMPLE_EDGE)
- Reducer 6 <- Map 1 (XPROD_EDGE), Map 8 (XPROD_EDGE)
- Reducer 7 <- Reducer 6 (SIMPLE_EDGE)
+ Reducer 2 <- Map 1 (CUSTOM_SIMPLE_EDGE), Map 7 (CUSTOM_SIMPLE_EDGE)
+ Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
+ Reducer 4 <- Map 1 (SIMPLE_EDGE), Reducer 6 (SIMPLE_EDGE)
+ Reducer 5 <- Map 1 (XPROD_EDGE), Map 7 (XPROD_EDGE)
+ Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
#### A masked pattern was here ####
Vertices:
Map 1
@@ -726,7 +689,7 @@ STAGE PLANS:
Statistics: Num rows: 26 Data size: 104 Basic stats: COMPLETE Column stats: COMPLETE
Execution mode: vectorized, llap
LLAP IO: all inputs
- Map 8
+ Map 7
Map Operator Tree:
TableScan
alias: part
@@ -777,26 +740,19 @@ STAGE PLANS:
Reduce Operator Tree:
Merge Join Operator
condition map:
- Left Outer Join 0 to 1
+ Anti Join 0 to 1
keys:
0 _col1 (type: string)
- 1 _col1 (type: string)
- outputColumnNames: _col0, _col2
- Statistics: Num rows: 1014 Data size: 124050 Basic stats: COMPLETE Column stats: COMPLETE
- Filter Operator
- predicate: _col2 is null (type: boolean)
- Statistics: Num rows: 676 Data size: 82700 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: _col0 (type: string)
- outputColumnNames: _col0
- Statistics: Num rows: 676 Data size: 81796 Basic stats: COMPLETE Column stats: COMPLETE
- File Output Operator
- compressed: false
- Statistics: Num rows: 676 Data size: 81796 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
+ 1 _col0 (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 338 Data size: 40898 Basic stats: COMPLETE Column stats: COMPLETE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 338 Data size: 40898 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
Reducer 4
Execution mode: llap
Reduce Operator Tree:
@@ -808,38 +764,23 @@ STAGE PLANS:
1 _col1 (type: double)
outputColumnNames: _col1
Statistics: Num rows: 24 Data size: 2496 Basic stats: COMPLETE Column stats: COMPLETE
- Group By Operator
- keys: _col1 (type: string)
- minReductionHashAggr: 0.5
- mode: hash
+ Select Operator
+ expressions: _col1 (type: string)
outputColumnNames: _col0
- Statistics: Num rows: 12 Data size: 1248 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)
+ Statistics: Num rows: 24 Data size: 2496 Basic stats: COMPLETE Column stats: COMPLETE
+ Group By Operator
+ keys: _col0 (type: string)
+ minReductionHashAggr: 0.5
+ mode: hash
+ outputColumnNames: _col0
Statistics: Num rows: 12 Data size: 1248 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)
+ Statistics: Num rows: 12 Data size: 1248 Basic stats: COMPLETE Column stats: COMPLETE
Reducer 5
- Execution mode: vectorized, llap
- Reduce Operator Tree:
- Group By Operator
- keys: KEY._col0 (type: string)
- mode: mergepartial
- outputColumnNames: _col0
- Statistics: Num rows: 12 Data size: 1248 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: true (type: boolean), _col0 (type: string)
- outputColumnNames: _col0, _col1
- Statistics: Num rows: 12 Data size: 1296 Basic stats: COMPLETE Column stats: COMPLETE
- Reduce Output Operator
- key expressions: _col1 (type: string)
- null sort order: z
- sort order: +
- Map-reduce partition columns: _col1 (type: string)
- Statistics: Num rows: 12 Data size: 1296 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: _col0 (type: boolean)
- Reducer 6
Execution mode: llap
Reduce Operator Tree:
Merge Join Operator
@@ -862,7 +803,7 @@ STAGE PLANS:
sort order: +
Map-reduce partition columns: _col0 (type: string)
Statistics: Num rows: 24 Data size: 2496 Basic stats: COMPLETE Column stats: COMPLETE
- Reducer 7
+ Reducer 6
Execution mode: vectorized, llap
Reduce Operator Tree:
Group By Operator
@@ -888,8 +829,8 @@ STAGE PLANS:
Processor Tree:
ListSink
-Warning: Shuffle Join MERGEJOIN[60][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
-Warning: Shuffle Join MERGEJOIN[61][tables = [$hdt$_3, $hdt$_4]] in Stage 'Reducer 6' is a cross product
+Warning: Shuffle Join MERGEJOIN[57][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+Warning: Shuffle Join MERGEJOIN[58][tables = [$hdt$_3, $hdt$_4]] in Stage 'Reducer 5' is a cross product
PREHOOK: query: SELECT p1.p_name FROM part p1 LEFT JOIN (select p_type as p_col from part ) p2 WHERE NOT EXISTS
(select pp1.p_type as p_col from part pp1 where pp1.p_partkey = p2.p_col)
PREHOOK: type: QUERY
@@ -1608,14 +1549,12 @@ POSTHOOK: Input: default@tschema
CBO PLAN:
HiveProject(eid=[_UTF-16LE'empno':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"], id=[$0])
HiveAggregate(group=[{0}])
- HiveFilter(condition=[IS NULL($1)])
- HiveJoin(condition=[=($0, $2)], joinType=[left], algorithm=[none], cost=[not available])
- HiveProject(id=[$0])
- HiveTableScan(table=[[default, tschema]], table:alias=[a])
- HiveProject(literalTrue=[true], id0=[$0])
- HiveAggregate(group=[{0}])
- HiveFilter(condition=[IS NOT NULL($0)])
- HiveTableScan(table=[[default, tschema]], table:alias=[c])
+ HiveAntiJoin(condition=[=($0, $2)], joinType=[anti])
+ HiveProject(id=[$0])
+ HiveTableScan(table=[[default, tschema]], table:alias=[a])
+ HiveProject(literalTrue=[true], id0=[$0])
+ HiveFilter(condition=[IS NOT NULL($0)])
+ HiveTableScan(table=[[default, tschema]], table:alias=[c])
PREHOOK: query: select distinct 'empno' as eid, a.id from tschema a
where NOT EXISTS (select c.id from tschema c where a.id=c.id)
diff --git a/ql/src/test/results/clientpositive/llap/subquery_notexists_having.q.out b/ql/src/test/results/clientpositive/llap/subquery_notexists_having.q.out
index f24ad46..4a2d993 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_notexists_having.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_notexists_having.q.out
@@ -31,7 +31,8 @@ STAGE PLANS:
Tez
#### A masked pattern was here ####
Edges:
- Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 3 (SIMPLE_EDGE)
+ Reducer 2 <- Map 1 (SIMPLE_EDGE)
+ Reducer 3 <- Map 1 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
#### A masked pattern was here ####
Vertices:
Map 1
@@ -55,72 +56,59 @@ STAGE PLANS:
sort order: ++
Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
Statistics: Num rows: 250 Data size: 44500 Basic stats: COMPLETE Column stats: COMPLETE
- Execution mode: vectorized, llap
- LLAP IO: all inputs
- Map 3
- Map Operator Tree:
- TableScan
- alias: a
- filterExpr: ((value > 'val_12') and key is not null) (type: boolean)
- Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
Filter Operator
predicate: ((value > 'val_12') and key is not null) (type: boolean)
Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
- Group By Operator
- keys: key (type: string), value (type: string)
- minReductionHashAggr: 0.0
- mode: hash
+ Select Operator
+ expressions: key (type: string), value (type: string)
outputColumnNames: _col0, _col1
- Statistics: Num rows: 83 Data size: 14774 Basic stats: COMPLETE Column stats: COMPLETE
- Reduce Output Operator
- key expressions: _col0 (type: string), _col1 (type: string)
- null sort order: zz
- sort order: ++
- Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
+ Statistics: Num rows: 166 Data size: 29548 Basic stats: COMPLETE Column stats: COMPLETE
+ Group By Operator
+ keys: _col0 (type: string), _col1 (type: string)
+ minReductionHashAggr: 0.0
+ mode: hash
+ outputColumnNames: _col0, _col1
Statistics: Num rows: 83 Data size: 14774 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: string), _col1 (type: string)
+ null sort order: zz
+ sort order: ++
+ Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
+ Statistics: Num rows: 83 Data size: 14774 Basic stats: COMPLETE Column stats: COMPLETE
Execution mode: vectorized, llap
LLAP IO: all inputs
Reducer 2
+ Execution mode: vectorized, llap
Reduce Operator Tree:
Group By Operator
keys: KEY._col0 (type: string), KEY._col1 (type: string)
mode: mergepartial
outputColumnNames: _col0, _col1
- Statistics: Num rows: 83 Data size: 14774 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: true (type: boolean), _col0 (type: string), _col1 (type: string)
- outputColumnNames: _col0, _col1, _col2
- Statistics: Num rows: 83 Data size: 15106 Basic stats: COMPLETE Column stats: COMPLETE
- Dummy Store
+ Statistics: Num rows: 250 Data size: 44500 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ key expressions: _col0 (type: string), _col1 (type: string)
+ null sort order: zz
+ sort order: ++
+ Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
+ Statistics: Num rows: 250 Data size: 44500 Basic stats: COMPLETE Column stats: COMPLETE
+ Reducer 3
Execution mode: llap
Reduce Operator Tree:
- Group By Operator
- keys: KEY._col0 (type: string), KEY._col1 (type: string)
- mode: mergepartial
+ Merge Join Operator
+ condition map:
+ Anti Join 0 to 1
+ keys:
+ 0 _col0 (type: string), _col1 (type: string)
+ 1 _col0 (type: string), _col1 (type: string)
outputColumnNames: _col0, _col1
- Statistics: Num rows: 250 Data size: 44500 Basic stats: COMPLETE Column stats: COMPLETE
- Merge Join Operator
- condition map:
- Left Outer Join 0 to 1
- keys:
- 0 _col0 (type: string), _col1 (type: string)
- 1 _col1 (type: string), _col2 (type: string)
- outputColumnNames: _col0, _col1, _col2
- Statistics: Num rows: 333 Data size: 59610 Basic stats: COMPLETE Column stats: COMPLETE
- Filter Operator
- predicate: _col2 is null (type: boolean)
- Statistics: Num rows: 250 Data size: 44752 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: _col0 (type: string), _col1 (type: string)
- outputColumnNames: _col0, _col1
- Statistics: Num rows: 250 Data size: 44500 Basic stats: COMPLETE Column stats: COMPLETE
- File Output Operator
- compressed: false
- Statistics: Num rows: 250 Data size: 44500 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
+ Statistics: Num rows: 167 Data size: 29726 Basic stats: COMPLETE Column stats: COMPLETE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 167 Data size: 29726 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
@@ -236,7 +224,7 @@ STAGE PLANS:
key expressions: _col0 (type: string), _col1 (type: string)
null sort order: zz
sort order: ++
- Map-reduce partition columns: _col0 (type: string)
+ Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
Statistics: Num rows: 83 Data size: 14774 Basic stats: COMPLETE Column stats: COMPLETE
Execution mode: vectorized, llap
LLAP IO: all inputs
@@ -264,26 +252,19 @@ STAGE PLANS:
Reduce Operator Tree:
Merge Join Operator
condition map:
- Left Outer Join 0 to 1
+ Anti Join 0 to 1
keys:
0 _col1 (type: string)
- 1 _col1 (type: string)
- outputColumnNames: _col0, _col1, _col2
- Statistics: Num rows: 291 Data size: 51966 Basic stats: COMPLETE Column stats: COMPLETE
- Filter Operator
- predicate: _col2 is null (type: boolean)
- Statistics: Num rows: 250 Data size: 44644 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: _col0 (type: string), _col1 (type: string)
- outputColumnNames: _col0, _col1
- Statistics: Num rows: 250 Data size: 44500 Basic stats: COMPLETE Column stats: COMPLETE
- File Output Operator
- compressed: false
- Statistics: Num rows: 250 Data size: 44500 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
+ 1 _col0 (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 209 Data size: 37202 Basic stats: COMPLETE Column stats: COMPLETE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 209 Data size: 37202 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
Reducer 4
Execution mode: vectorized, llap
Reduce Operator Tree:
@@ -294,24 +275,20 @@ STAGE PLANS:
Statistics: Num rows: 83 Data size: 14774 Basic stats: COMPLETE Column stats: COMPLETE
Select Operator
expressions: _col0 (type: string)
- outputColumnNames: _col1
- Statistics: Num rows: 83 Data size: 14774 Basic stats: COMPLETE Column stats: COMPLETE
+ outputColumnNames: _col0
+ Statistics: Num rows: 83 Data size: 7553 Basic stats: COMPLETE Column stats: COMPLETE
Group By Operator
- keys: _col1 (type: string)
- mode: complete
+ keys: _col0 (type: string)
+ minReductionHashAggr: 0.5060241
+ mode: hash
outputColumnNames: _col0
Statistics: Num rows: 41 Data size: 3731 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: true (type: boolean), _col0 (type: string)
- outputColumnNames: _col0, _col1
- Statistics: Num rows: 41 Data size: 3895 Basic stats: COMPLETE Column stats: COMPLETE
- Reduce Output Operator
- key expressions: _col1 (type: string)
- null sort order: z
- sort order: +
- Map-reduce partition columns: _col1 (type: string)
- Statistics: Num rows: 41 Data size: 3895 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions: _col0 (type: boolean)
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ null sort order: z
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
+ Statistics: Num rows: 41 Data size: 3731 Basic stats: COMPLETE Column stats: COMPLETE
Stage: Stage-0
Fetch Operator
diff --git a/ql/src/test/results/clientpositive/llap/subquery_scalar.q.out b/ql/src/test/results/clientpositive/llap/subquery_scalar.q.out
index 0f166ce..4ef2b56 100644
--- a/ql/src/test/results/clientpositive/llap/subquery_scalar.q.out
+++ b/ql/src/test/results/clientpositive/llap/subquery_scalar.q.out
@@ -4461,6 +4461,150 @@ POSTHOOK: Input: default@part
POSTHOOK: Input: default@part_null_n0
#### A masked pattern was here ####
192697 almond antique blue firebrick mint Manufacturer#5 Brand#52 MEDIUM BURNISHED TIN 31 LG DRUM 1789.69 ickly ir
+Warning: Shuffle Join MERGEJOIN[15][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+PREHOOK: query: explain select * from part where (select i from tnull_n0 limit 1) is null
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part
+PREHOOK: Input: default@tnull_n0
+#### A masked pattern was here ####
+POSTHOOK: query: explain select * from part where (select i from tnull_n0 limit 1) is null
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part
+POSTHOOK: Input: default@tnull_n0
+#### A masked pattern was here ####
+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 (CUSTOM_SIMPLE_EDGE), Reducer 4 (CUSTOM_SIMPLE_EDGE)
+ Reducer 4 <- Map 3 (CUSTOM_SIMPLE_EDGE)
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: part
+ Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
+ Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ null sort order:
+ sort order:
+ Statistics: Num rows: 26 Data size: 16094 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
+ Execution mode: vectorized, llap
+ LLAP IO: all inputs
+ Map 3
+ Map Operator Tree:
+ TableScan
+ alias: tnull_n0
+ Statistics: Num rows: 2 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: i (type: int)
+ outputColumnNames: _col0
+ Statistics: Num rows: 2 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ Limit
+ Number of rows: 1
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ null sort order:
+ sort order:
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ TopN Hash Memory Usage: 0.1
+ value expressions: _col0 (type: int)
+ Execution mode: vectorized, llap
+ LLAP IO: all inputs
+ Reducer 2
+ Execution mode: llap
+ Reduce Operator Tree:
+ Merge Join Operator
+ condition map:
+ Left Outer Join 0 to 1
+ keys:
+ 0
+ 1
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
+ Statistics: Num rows: 26 Data size: 16198 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator
+ predicate: _col9 is null (type: boolean)
+ Statistics: Num rows: 1 Data size: 623 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator
+ expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
+ Statistics: Num rows: 1 Data size: 619 Basic stats: COMPLETE Column stats: COMPLETE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 619 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
+ Reducer 4
+ Execution mode: vectorized, llap
+ Reduce Operator Tree:
+ Select Operator
+ expressions: VALUE._col0 (type: int)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ Limit
+ Number of rows: 1
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ Reduce Output Operator
+ null sort order:
+ sort order:
+ Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
+ value expressions: _col0 (type: int)
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+Warning: Shuffle Join MERGEJOIN[15][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+PREHOOK: query: select * from part where (select i from tnull_n0 limit 1) is null
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part
+PREHOOK: Input: default@tnull_n0
+#### A masked pattern was here ####
+POSTHOOK: query: select * from part where (select i from tnull_n0 limit 1) is null
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part
+POSTHOOK: Input: default@tnull_n0
+#### A masked pattern was here ####
+105685 almond antique violet chocolate turquoise Manufacturer#2 Brand#22 MEDIUM ANODIZED COPPER 14 MED CAN 1690.68 ly pending requ
+110592 almond antique salmon chartreuse burlywood Manufacturer#1 Brand#15 PROMO BURNISHED NICKEL 6 JUMBO PKG 1602.59 to the furiously
+112398 almond antique metallic orange dim Manufacturer#3 Brand#32 MEDIUM BURNISHED BRASS 19 JUMBO JAR 1410.39 ole car
+121152 almond antique burnished rose metallic Manufacturer#1 Brand#14 PROMO PLATED TIN 2 JUMBO BOX 1173.15 e pinto beans h
+121152 almond antique burnished rose metallic Manufacturer#1 Brand#14 PROMO PLATED TIN 2 JUMBO BOX 1173.15 e pinto beans h
+132666 almond aquamarine rose maroon antique Manufacturer#2 Brand#24 SMALL POLISHED NICKEL 25 MED BOX 1698.66 even
+144293 almond antique olive coral navajo Manufacturer#3 Brand#34 STANDARD POLISHED STEEL 45 JUMBO CAN 1337.29 ag furiously about
+146985 almond aquamarine midnight light salmon Manufacturer#2 Brand#23 MEDIUM BURNISHED COPPER 2 SM CASE 2031.98 s cajole caref
+15103 almond aquamarine dodger light gainsboro Manufacturer#5 Brand#53 ECONOMY BURNISHED STEEL 46 LG PACK 1018.1 packages hinder carefu
+155733 almond antique sky peru orange Manufacturer#5 Brand#53 SMALL PLATED BRASS 2 WRAP DRUM 1788.73 furiously. bra
+17273 almond antique forest lavender goldenrod Manufacturer#3 Brand#35 PROMO ANODIZED TIN 14 JUMBO CASE 1190.27 along the
+17927 almond aquamarine yellow dodger mint Manufacturer#4 Brand#41 ECONOMY BRUSHED COPPER 7 SM PKG 1844.92 ites. eve
+191709 almond antique violet turquoise frosted Manufacturer#2 Brand#22 ECONOMY POLISHED STEEL 40 MED BOX 1800.7 haggle
+192697 almond antique blue firebrick mint Manufacturer#5 Brand#52 MEDIUM BURNISHED TIN 31 LG DRUM 1789.69 ickly ir
+195606 almond aquamarine sandy cyan gainsboro Manufacturer#2 Brand#25 STANDARD PLATED TIN 18 SM PKG 1701.6 ic de
+33357 almond azure aquamarine papaya violet Manufacturer#4 Brand#41 STANDARD ANODIZED TIN 12 WRAP CASE 1290.35 reful
+40982 almond antique misty red olive Manufacturer#3 Brand#32 ECONOMY PLATED COPPER 1 LG PKG 1922.98 c foxes can s
+42669 almond antique medium spring khaki Manufacturer#5 Brand#51 STANDARD BURNISHED TIN 6 MED CAN 1611.66 sits haggl
+45261 almond aquamarine floral ivory bisque Manufacturer#4 Brand#42 SMALL PLATED STEEL 27 WRAP CASE 1206.26 careful
+48427 almond antique violet mint lemon Manufacturer#4 Brand#42 PROMO POLISHED STEEL 39 SM CASE 1375.42 hely ironic i
+49671 almond antique gainsboro frosted violet Manufacturer#4 Brand#41 SMALL BRUSHED BRASS 10 SM BOX 1620.67 ccounts run quick
+65667 almond aquamarine pink moccasin thistle Manufacturer#1 Brand#12 LARGE BURNISHED STEEL 42 JUMBO CASE 1632.66 e across the expr
+78486 almond azure blanched chiffon midnight Manufacturer#5 Brand#52 LARGE BRUSHED BRASS 23 MED BAG 1464.48 hely blith
+85768 almond antique chartreuse lavender yellow Manufacturer#1 Brand#12 LARGE BRUSHED STEEL 34 SM BAG 1753.76 refull
+86428 almond aquamarine burnished black steel Manufacturer#1 Brand#12 STANDARD ANODIZED STEEL 28 WRAP BAG 1414.42 arefully
+90681 almond antique chartreuse khaki white Manufacturer#3 Brand#31 MEDIUM BURNISHED TIN 17 SM CASE 1671.68 are slyly after the sl
PREHOOK: query: drop table tnull_n0
PREHOOK: type: DROPTABLE
PREHOOK: Input: default@tnull_n0
diff --git a/ql/src/test/results/clientpositive/perf/tez/cbo_query16.q.out b/ql/src/test/results/clientpositive/perf/tez/cbo_query16.q.out
index 2bc16ff..5fb0f8e 100644
--- a/ql/src/test/results/clientpositive/perf/tez/cbo_query16.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/cbo_query16.q.out
@@ -72,30 +72,28 @@ POSTHOOK: Input: default@date_dim
POSTHOOK: Output: hdfs://### HDFS PATH ###
CBO PLAN:
HiveAggregate(group=[{}], agg#0=[count(DISTINCT $4)], agg#1=[sum($5)], agg#2=[sum($6)])
- HiveFilter(condition=[IS NULL($13)])
- HiveJoin(condition=[=($4, $14)], joinType=[left], algorithm=[none], cost=[not available])
- HiveSemiJoin(condition=[AND(<>($3, $13), =($4, $14))], joinType=[semi])
- HiveProject(cs_ship_date_sk=[$2], cs_ship_addr_sk=[$3], cs_call_center_sk=[$4], cs_warehouse_sk=[$5], cs_order_number=[$6], cs_ext_ship_cost=[$7], cs_net_profit=[$8], d_date_sk=[$9], d_date=[$10], ca_address_sk=[$0], ca_state=[$1], cc_call_center_sk=[$11], cc_county=[$12])
- HiveJoin(condition=[=($4, $11)], joinType=[inner], algorithm=[none], cost=[not available])
- HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
- HiveProject(ca_address_sk=[$0], ca_state=[CAST(_UTF-16LE'NY':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"):VARCHAR(2147483647) CHARACTER SET "UTF-16LE"])
- HiveFilter(condition=[AND(=($8, _UTF-16LE'NY'), IS NOT NULL($0))])
- HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
- HiveJoin(condition=[=($0, $7)], joinType=[inner], algorithm=[none], cost=[not available])
- HiveProject(cs_ship_date_sk=[$2], cs_ship_addr_sk=[$10], cs_call_center_sk=[$11], cs_warehouse_sk=[$14], cs_order_number=[$17], cs_ext_ship_cost=[$28], cs_net_profit=[$33])
- HiveFilter(condition=[AND(IS NOT NULL($2), IS NOT NULL($11), IS NOT NULL($10), IS NOT NULL($17))])
- HiveTableScan(table=[[default, catalog_sales]], table:alias=[cs1])
- HiveProject(d_date_sk=[$0], d_date=[$2])
- HiveFilter(condition=[AND(BETWEEN(false, CAST($2):TIMESTAMP(9), 2001-04-01 00:00:00:TIMESTAMP(9), 2001-05-31 00:00:00:TIMESTAMP(9)), IS NOT NULL($0))])
- HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
- HiveProject(cc_call_center_sk=[$0], cc_county=[$25])
- HiveFilter(condition=[AND(IN($25, _UTF-16LE'Ziebach County':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'Levy County':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'Huron County':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'Franklin Parish':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'Daviess County':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"), IS NOT NULL($0))])
- HiveTableScan(table=[[default, call_center]], table:alias=[call_center])
- HiveProject(cs_warehouse_sk=[$14], cs_order_number=[$17])
- HiveFilter(condition=[AND(IS NOT NULL($14), IS NOT NULL($17))])
- HiveTableScan(table=[[default, catalog_sales]], table:alias=[cs2])
- HiveProject(literalTrue=[true], cr_order_number0=[$0])
- HiveAggregate(group=[{16}])
- HiveFilter(condition=[IS NOT NULL($16)])
- HiveTableScan(table=[[default, catalog_returns]], table:alias=[cr1])
+ HiveAntiJoin(condition=[=($4, $14)], joinType=[anti])
+ HiveSemiJoin(condition=[AND(<>($3, $13), =($4, $14))], joinType=[semi])
+ HiveProject(cs_ship_date_sk=[$2], cs_ship_addr_sk=[$3], cs_call_center_sk=[$4], cs_warehouse_sk=[$5], cs_order_number=[$6], cs_ext_ship_cost=[$7], cs_net_profit=[$8], d_date_sk=[$9], d_date=[$10], ca_address_sk=[$0], ca_state=[$1], cc_call_center_sk=[$11], cc_county=[$12])
+ HiveJoin(condition=[=($4, $11)], joinType=[inner], algorithm=[none], cost=[not available])
+ HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+ HiveProject(ca_address_sk=[$0], ca_state=[CAST(_UTF-16LE'NY':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"):VARCHAR(2147483647) CHARACTER SET "UTF-16LE"])
+ HiveFilter(condition=[AND(=($8, _UTF-16LE'NY'), IS NOT NULL($0))])
+ HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
+ HiveJoin(condition=[=($0, $7)], joinType=[inner], algorithm=[none], cost=[not available])
+ HiveProject(cs_ship_date_sk=[$2], cs_ship_addr_sk=[$10], cs_call_center_sk=[$11], cs_warehouse_sk=[$14], cs_order_number=[$17], cs_ext_ship_cost=[$28], cs_net_profit=[$33])
+ HiveFilter(condition=[AND(IS NOT NULL($2), IS NOT NULL($11), IS NOT NULL($10), IS NOT NULL($17))])
+ HiveTableScan(table=[[default, catalog_sales]], table:alias=[cs1])
+ HiveProject(d_date_sk=[$0], d_date=[$2])
+ HiveFilter(condition=[AND(BETWEEN(false, CAST($2):TIMESTAMP(9), 2001-04-01 00:00:00:TIMESTAMP(9), 2001-05-31 00:00:00:TIMESTAMP(9)), IS NOT NULL($0))])
+ HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+ HiveProject(cc_call_center_sk=[$0], cc_county=[$25])
+ HiveFilter(condition=[AND(IN($25, _UTF-16LE'Ziebach County':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'Levy County':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'Huron County':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'Franklin Parish':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'Daviess County':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"), IS NOT NULL($0))])
+ HiveTableScan(table=[[default, call_center]], table:alias=[call_center])
+ HiveProject(cs_warehouse_sk=[$14], cs_order_number=[$17])
+ HiveFilter(condition=[AND(IS NOT NULL($14), IS NOT NULL($17))])
+ HiveTableScan(table=[[default, catalog_sales]], table:alias=[cs2])
+ HiveProject(literalTrue=[true], cr_order_number0=[$16])
+ HiveFilter(condition=[IS NOT NULL($16)])
+ HiveTableScan(table=[[default, catalog_returns]], table:alias=[cr1])
diff --git a/ql/src/test/results/clientpositive/perf/tez/cbo_query69.q.out b/ql/src/test/results/clientpositive/perf/tez/cbo_query69.q.out
index e114e34..e75ff38 100644
--- a/ql/src/test/results/clientpositive/perf/tez/cbo_query69.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/cbo_query69.q.out
@@ -110,47 +110,44 @@ CBO PLAN:
HiveSortLimit(sort0=[$0], sort1=[$1], sort2=[$2], sort3=[$4], sort4=[$6], dir0=[ASC], dir1=[ASC], dir2=[ASC], dir3=[ASC], dir4=[ASC], fetch=[100])
HiveProject(cd_gender=[$0], cd_marital_status=[$1], cd_education_status=[$2], cnt1=[$5], cd_purchase_estimate=[$3], cnt2=[$5], cd_credit_rating=[$4], cnt3=[$5])
HiveAggregate(group=[{6, 7, 8, 9, 10}], agg#0=[count()])
- HiveFilter(condition=[IS NULL($13)])
- HiveJoin(condition=[=($0, $14)], joinType=[left], algorithm=[none], cost=[not available])
- HiveProject(c_customer_sk=[$0], c_current_cdemo_sk=[$1], c_current_addr_sk=[$2], ca_address_sk=[$3], ca_state=[$4], cd_demo_sk=[$5], cd_gender=[$6], cd_marital_status=[$7], cd_education_status=[$8], cd_purchase_estimate=[$9], cd_credit_rating=[$10], literalTrue=[$11], ws_bill_customer_sk0=[$12])
- HiveFilter(condition=[IS NULL($11)])
- HiveJoin(condition=[=($0, $12)], joinType=[left], algorithm=[none], cost=[not available])
- HiveSemiJoin(condition=[=($0, $11)], joinType=[semi])
- HiveJoin(condition=[=($5, $1)], joinType=[inner], algorithm=[none], cost=[not available])
- HiveJoin(condition=[=($2, $3)], joinType=[inner], algorithm=[none], cost=[not available])
- HiveProject(c_customer_sk=[$0], c_current_cdemo_sk=[$2], c_current_addr_sk=[$4])
- HiveFilter(condition=[AND(IS NOT NULL($2), IS NOT NULL($4), IS NOT NULL($0))])
- HiveTableScan(table=[[default, customer]], table:alias=[c])
- HiveProject(ca_address_sk=[$0], ca_state=[$8])
- HiveFilter(condition=[AND(IN($8, _UTF-16LE'CO':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'IL':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'MN':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"), IS NOT NULL($0))])
- HiveTableScan(table=[[default, customer_address]], table:alias=[ca])
- HiveProject(cd_demo_sk=[$0], cd_gender=[$1], cd_marital_status=[$2], cd_education_status=[$3], cd_purchase_estimate=[$4], cd_credit_rating=[$5])
- HiveFilter(condition=[IS NOT NULL($0)])
- HiveTableScan(table=[[default, customer_demographics]], table:alias=[customer_demographics])
- HiveProject(ss_customer_sk0=[$1])
- HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
- HiveProject(ss_sold_date_sk=[$0], ss_customer_sk=[$3])
- HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($3))])
- HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
- HiveProject(d_date_sk=[$0])
- HiveFilter(condition=[AND(=($6, 1999), BETWEEN(false, $8, 1, 3), IS NOT NULL($0))])
- HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
- HiveProject(literalTrue=[true], ws_bill_customer_sk0=[$0])
- HiveAggregate(group=[{1}])
- HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
- HiveProject(ws_sold_date_sk=[$0], ws_bill_customer_sk=[$4])
- HiveFilter(condition=[AND(IS NOT NULL($4), IS NOT NULL($0))])
- HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
- HiveProject(d_date_sk=[$0])
- HiveFilter(condition=[AND(=($6, 1999), BETWEEN(false, $8, 1, 3), IS NOT NULL($0))])
- HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
- HiveProject(literalTrue=[true], cs_ship_customer_sk0=[$0])
- HiveAggregate(group=[{1}])
- HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
- HiveProject(cs_sold_date_sk=[$0], cs_ship_customer_sk=[$7])
- HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($0))])
- HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
- HiveProject(d_date_sk=[$0])
- HiveFilter(condition=[AND(=($6, 1999), BETWEEN(false, $8, 1, 3), IS NOT NULL($0))])
- HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+ HiveAntiJoin(condition=[=($0, $14)], joinType=[anti])
+ HiveProject(c_customer_sk=[$0], c_current_cdemo_sk=[$1], c_current_addr_sk=[$2], ca_address_sk=[$3], ca_state=[$4], cd_demo_sk=[$5], cd_gender=[$6], cd_marital_status=[$7], cd_education_status=[$8], cd_purchase_estimate=[$9], cd_credit_rating=[$10], literalTrue=[$11], ws_bill_customer_sk0=[$12])
+ HiveFilter(condition=[IS NULL($11)])
+ HiveJoin(condition=[=($0, $12)], joinType=[left], algorithm=[none], cost=[not available])
+ HiveSemiJoin(condition=[=($0, $11)], joinType=[semi])
+ HiveJoin(condition=[=($5, $1)], joinType=[inner], algorithm=[none], cost=[not available])
+ HiveJoin(condition=[=($2, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+ HiveProject(c_customer_sk=[$0], c_current_cdemo_sk=[$2], c_current_addr_sk=[$4])
+ HiveFilter(condition=[AND(IS NOT NULL($2), IS NOT NULL($4), IS NOT NULL($0))])
+ HiveTableScan(table=[[default, customer]], table:alias=[c])
+ HiveProject(ca_address_sk=[$0], ca_state=[$8])
+ HiveFilter(condition=[AND(IN($8, _UTF-16LE'CO':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'IL':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'MN':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"), IS NOT NULL($0))])
+ HiveTableScan(table=[[default, customer_address]], table:alias=[ca])
+ HiveProject(cd_demo_sk=[$0], cd_gender=[$1], cd_marital_status=[$2], cd_education_status=[$3], cd_purchase_estimate=[$4], cd_credit_rating=[$5])
+ HiveFilter(condition=[IS NOT NULL($0)])
+ HiveTableScan(table=[[default, customer_demographics]], table:alias=[customer_demographics])
+ HiveProject(ss_customer_sk0=[$1])
+ HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+ HiveProject(ss_sold_date_sk=[$0], ss_customer_sk=[$3])
+ HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($3))])
+ HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+ HiveProject(d_date_sk=[$0])
+ HiveFilter(condition=[AND(=($6, 1999), BETWEEN(false, $8, 1, 3), IS NOT NULL($0))])
+ HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+ HiveProject(literalTrue=[true], ws_bill_customer_sk0=[$1])
+ HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+ HiveProject(ws_sold_date_sk=[$0], ws_bill_customer_sk=[$4])
+ HiveFilter(condition=[AND(IS NOT NULL($4), IS NOT NULL($0))])
+ HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
+ HiveProject(d_date_sk=[$0])
+ HiveFilter(condition=[AND(=($6, 1999), BETWEEN(false, $8, 1, 3), IS NOT NULL($0))])
+ HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+ HiveProject(literalTrue=[true], cs_ship_customer_sk0=[$1])
+ HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+ HiveProject(cs_sold_date_sk=[$0], cs_ship_customer_sk=[$7])
+ HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($0))])
+ HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
+ HiveProject(d_date_sk=[$0])
+ HiveFilter(condition=[AND(=($6, 1999), BETWEEN(false, $8, 1, 3), IS NOT NULL($0))])
+ HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
diff --git a/ql/src/test/results/clientpositive/perf/tez/cbo_query78.q.out b/ql/src/test/results/clientpositive/perf/tez/cbo_query78.q.out
index 180204c..df6fb01 100644
--- a/ql/src/test/results/clientpositive/perf/tez/cbo_query78.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/cbo_query78.q.out
@@ -142,14 +142,13 @@ HiveSortLimit(fetch=[100])
HiveFilter(condition=[AND(=($6, 2000), IS NOT NULL($0))])
HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$1], ss_customer_sk=[$2], ss_quantity=[$4], ss_wholesale_cost=[$5], ss_sales_price=[$6])
- HiveFilter(condition=[IS NULL($8)])
- HiveJoin(condition=[AND(=($8, $3), =($1, $7))], joinType=[left], algorithm=[none], cost=[not available])
- HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2], ss_customer_sk=[$3], ss_ticket_number=[$9], ss_quantity=[$10], ss_wholesale_cost=[$11], ss_sales_price=[$13])
- HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($3), IS NOT NULL($2))])
- HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
- HiveProject(sr_item_sk=[$2], sr_ticket_number=[$9])
- HiveFilter(condition=[AND(IS NOT NULL($9), IS NOT NULL($2))])
- HiveTableScan(table=[[default, store_returns]], table:alias=[store_returns])
+ HiveAntiJoin(condition=[AND(=($8, $3), =($1, $7))], joinType=[anti])
+ HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2], ss_customer_sk=[$3], ss_ticket_number=[$9], ss_quantity=[$10], ss_wholesale_cost=[$11], ss_sales_price=[$13])
+ HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($3), IS NOT NULL($2))])
+ HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+ HiveProject(sr_item_sk=[$2], sr_ticket_number=[$9])
+ HiveFilter(condition=[AND(IS NOT NULL($9), IS NOT NULL($2))])
+ HiveTableScan(table=[[default, store_returns]], table:alias=[store_returns])
HiveProject($f1=[$0], $f2=[$1], $f2_0=[$2], $f3=[$3], $f4=[$4])
HiveFilter(condition=[>($2, 0)])
HiveAggregate(group=[{2, 3}], agg#0=[sum($4)], agg#1=[sum($5)], agg#2=[sum($6)])
@@ -158,14 +157,13 @@ HiveSortLimit(fetch=[100])
HiveFilter(condition=[AND(=($6, 2000), IS NOT NULL($0))])
HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
HiveProject(ws_sold_date_sk=[$0], ws_item_sk=[$1], ws_bill_customer_sk=[$2], ws_quantity=[$4], ws_wholesale_cost=[$5], ws_sales_price=[$6])
- HiveFilter(condition=[IS NULL($8)])
- HiveJoin(condition=[AND(=($8, $3), =($1, $7))], joinType=[left], algorithm=[none], cost=[not available])
- HiveProject(ws_sold_date_sk=[$0], ws_item_sk=[$3], ws_bill_customer_sk=[$4], ws_order_number=[$17], ws_quantity=[$18], ws_wholesale_cost=[$19], ws_sales_price=[$21])
- HiveFilter(condition=[AND(IS NOT NULL($4), IS NOT NULL($0), IS NOT NULL($3))])
- HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
- HiveProject(wr_item_sk=[$2], wr_order_number=[$13])
- HiveFilter(condition=[AND(IS NOT NULL($13), IS NOT NULL($2))])
- HiveTableScan(table=[[default, web_returns]], table:alias=[web_returns])
+ HiveAntiJoin(condition=[AND(=($8, $3), =($1, $7))], joinType=[anti])
+ HiveProject(ws_sold_date_sk=[$0], ws_item_sk=[$3], ws_bill_customer_sk=[$4], ws_order_number=[$17], ws_quantity=[$18], ws_wholesale_cost=[$19], ws_sales_price=[$21])
+ HiveFilter(condition=[AND(IS NOT NULL($4), IS NOT NULL($0), IS NOT NULL($3))])
+ HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
+ HiveProject(wr_item_sk=[$2], wr_order_number=[$13])
+ HiveFilter(condition=[AND(IS NOT NULL($13), IS NOT NULL($2))])
+ HiveTableScan(table=[[default, web_returns]], table:alias=[web_returns])
HiveProject($f2=[$1], $f3=[$2], IS NOT NULL=[IS NOT NULL($2)], CASE=[CASE(IS NOT NULL($2), $2, 0:BIGINT)], CASE7=[CASE(IS NOT NULL($3), $3, 0:DECIMAL(17, 2))], CASE8=[CASE(IS NOT NULL($4), $4, 0:DECIMAL(17, 2))])
HiveFilter(condition=[>($2, 0)])
HiveProject(cs_item_sk=[$1], cs_bill_customer_sk=[$0], $f2=[$2], $f3=[$3], $f4=[$4])
@@ -175,12 +173,11 @@ HiveSortLimit(fetch=[100])
HiveFilter(condition=[AND(=($6, 2000), IS NOT NULL($0))])
HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
HiveProject(cs_sold_date_sk=[$0], cs_bill_customer_sk=[$1], cs_item_sk=[$2], cs_quantity=[$4], cs_wholesale_cost=[$5], cs_sales_price=[$6])
- HiveFilter(condition=[IS NULL($8)])
- HiveJoin(condition=[AND(=($8, $3), =($2, $7))], joinType=[left], algorithm=[none], cost=[not available])
- HiveProject(cs_sold_date_sk=[$0], cs_bill_customer_sk=[$3], cs_item_sk=[$15], cs_order_number=[$17], cs_quantity=[$18], cs_wholesale_cost=[$19], cs_sales_price=[$21])
- HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($3), IS NOT NULL($15))])
- HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
- HiveProject(cr_item_sk=[$2], cr_order_number=[$16])
- HiveFilter(condition=[AND(IS NOT NULL($16), IS NOT NULL($2))])
- HiveTableScan(table=[[default, catalog_returns]], table:alias=[catalog_returns])
+ HiveAntiJoin(condition=[AND(=($8, $3), =($2, $7))], joinType=[anti])
+ HiveProject(cs_sold_date_sk=[$0], cs_bill_customer_sk=[$3], cs_item_sk=[$15], cs_order_number=[$17], cs_quantity=[$18], cs_wholesale_cost=[$19], cs_sales_price=[$21])
+ HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($3), IS NOT NULL($15))])
+ HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
+ HiveProject(cr_item_sk=[$2], cr_order_number=[$16])
+ HiveFilter(condition=[AND(IS NOT NULL($16), IS NOT NULL($2))])
+ HiveTableScan(table=[[default, catalog_returns]], table:alias=[catalog_returns])
diff --git a/ql/src/test/results/clientpositive/perf/tez/cbo_query94.q.out b/ql/src/test/results/clientpositive/perf/tez/cbo_query94.q.out
index b8521f4..722de27 100644
--- a/ql/src/test/results/clientpositive/perf/tez/cbo_query94.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/cbo_query94.q.out
@@ -68,30 +68,28 @@ POSTHOOK: Input: default@web_site
POSTHOOK: Output: hdfs://### HDFS PATH ###
CBO PLAN:
HiveAggregate(group=[{}], agg#0=[count(DISTINCT $4)], agg#1=[sum($5)], agg#2=[sum($6)])
- HiveFilter(condition=[IS NULL($13)])
- HiveJoin(condition=[=($4, $14)], joinType=[left], algorithm=[none], cost=[not available])
- HiveSemiJoin(condition=[AND(<>($3, $13), =($4, $14))], joinType=[semi])
- HiveProject(ws_ship_date_sk=[$2], ws_ship_addr_sk=[$3], ws_web_site_sk=[$4], ws_warehouse_sk=[$5], ws_order_number=[$6], ws_ext_ship_cost=[$7], ws_net_profit=[$8], d_date_sk=[$9], d_date=[$10], ca_address_sk=[$0], ca_state=[$1], web_site_sk=[$11], web_company_name=[$12])
- HiveJoin(condition=[=($4, $11)], joinType=[inner], algorithm=[none], cost=[not available])
- HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
- HiveProject(ca_address_sk=[$0], ca_state=[CAST(_UTF-16LE'TX':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"):VARCHAR(2147483647) CHARACTER SET "UTF-16LE"])
- HiveFilter(condition=[AND(=($8, _UTF-16LE'TX'), IS NOT NULL($0))])
- HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
- HiveJoin(condition=[=($0, $7)], joinType=[inner], algorithm=[none], cost=[not available])
- HiveProject(ws_ship_date_sk=[$2], ws_ship_addr_sk=[$11], ws_web_site_sk=[$13], ws_warehouse_sk=[$15], ws_order_number=[$17], ws_ext_ship_cost=[$28], ws_net_profit=[$33])
- HiveFilter(condition=[AND(IS NOT NULL($13), IS NOT NULL($11), IS NOT NULL($2), IS NOT NULL($17))])
- HiveTableScan(table=[[default, web_sales]], table:alias=[ws1])
- HiveProject(d_date_sk=[$0], d_date=[$2])
- HiveFilter(condition=[AND(BETWEEN(false, CAST($2):TIMESTAMP(9), 1999-05-01 00:00:00:TIMESTAMP(9), 1999-06-30 00:00:00:TIMESTAMP(9)), IS NOT NULL($0))])
- HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
- HiveProject(web_site_sk=[$0], web_company_name=[CAST(_UTF-16LE'pri':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"):VARCHAR(2147483647) CHARACTER SET "UTF-16LE"])
- HiveFilter(condition=[AND(=($14, _UTF-16LE'pri'), IS NOT NULL($0))])
- HiveTableScan(table=[[default, web_site]], table:alias=[web_site])
- HiveProject(ws_warehouse_sk=[$15], ws_order_number=[$17])
- HiveFilter(condition=[AND(IS NOT NULL($15), IS NOT NULL($17))])
- HiveTableScan(table=[[default, web_sales]], table:alias=[ws2])
- HiveProject(literalTrue=[true], wr_order_number0=[$0])
- HiveAggregate(group=[{13}])
- HiveFilter(condition=[IS NOT NULL($13)])
- HiveTableScan(table=[[default, web_returns]], table:alias=[wr1])
+ HiveAntiJoin(condition=[=($4, $14)], joinType=[anti])
+ HiveSemiJoin(condition=[AND(<>($3, $13), =($4, $14))], joinType=[semi])
+ HiveProject(ws_ship_date_sk=[$2], ws_ship_addr_sk=[$3], ws_web_site_sk=[$4], ws_warehouse_sk=[$5], ws_order_number=[$6], ws_ext_ship_cost=[$7], ws_net_profit=[$8], d_date_sk=[$9], d_date=[$10], ca_address_sk=[$0], ca_state=[$1], web_site_sk=[$11], web_company_name=[$12])
+ HiveJoin(condition=[=($4, $11)], joinType=[inner], algorithm=[none], cost=[not available])
+ HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+ HiveProject(ca_address_sk=[$0], ca_state=[CAST(_UTF-16LE'TX':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"):VARCHAR(2147483647) CHARACTER SET "UTF-16LE"])
+ HiveFilter(condition=[AND(=($8, _UTF-16LE'TX'), IS NOT NULL($0))])
+ HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
+ HiveJoin(condition=[=($0, $7)], joinType=[inner], algorithm=[none], cost=[not available])
+ HiveProject(ws_ship_date_sk=[$2], ws_ship_addr_sk=[$11], ws_web_site_sk=[$13], ws_warehouse_sk=[$15], ws_order_number=[$17], ws_ext_ship_cost=[$28], ws_net_profit=[$33])
+ HiveFilter(condition=[AND(IS NOT NULL($13), IS NOT NULL($11), IS NOT NULL($2), IS NOT NULL($17))])
+ HiveTableScan(table=[[default, web_sales]], table:alias=[ws1])
+ HiveProject(d_date_sk=[$0], d_date=[$2])
+ HiveFilter(condition=[AND(BETWEEN(false, CAST($2):TIMESTAMP(9), 1999-05-01 00:00:00:TIMESTAMP(9), 1999-06-30 00:00:00:TIMESTAMP(9)), IS NOT NULL($0))])
+ HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+ HiveProject(web_site_sk=[$0], web_company_name=[CAST(_UTF-16LE'pri':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"):VARCHAR(2147483647) CHARACTER SET "UTF-16LE"])
+ HiveFilter(condition=[AND(=($14, _UTF-16LE'pri'), IS NOT NULL($0))])
+ HiveTableScan(table=[[default, web_site]], table:alias=[web_site])
+ HiveProject(ws_warehouse_sk=[$15], ws_order_number=[$17])
+ HiveFilter(condition=[AND(IS NOT NULL($15), IS NOT NULL($17))])
+ HiveTableScan(table=[[default, web_sales]], table:alias=[ws2])
+ HiveProject(literalTrue=[true], wr_order_number0=[$13])
+ HiveFilter(condition=[IS NOT NULL($13)])
+ HiveTableScan(table=[[default, web_returns]], table:alias=[wr1])
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query16.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query16.q.out
index 376b60a..0a3ef38 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query16.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query16.q.out
@@ -72,29 +72,27 @@ POSTHOOK: Input: default@date_dim
POSTHOOK: Output: hdfs://### HDFS PATH ###
CBO PLAN:
HiveAggregate(group=[{}], agg#0=[count(DISTINCT $4)], agg#1=[sum($5)], agg#2=[sum($6)])
- HiveFilter(condition=[IS NULL($13)])
- HiveJoin(condition=[=($4, $14)], joinType=[left], algorithm=[none], cost=[not available])
- HiveSemiJoin(condition=[AND(<>($3, $13), =($4, $14))], joinType=[semi])
- HiveProject(cs_ship_date_sk=[$2], cs_ship_addr_sk=[$3], cs_call_center_sk=[$4], cs_warehouse_sk=[$5], cs_order_number=[$6], cs_ext_ship_cost=[$7], cs_net_profit=[$8], d_date_sk=[$9], d_date=[$10], ca_address_sk=[$0], ca_state=[$1], cc_call_center_sk=[$11], cc_county=[$12])
- HiveJoin(condition=[=($4, $11)], joinType=[inner], algorithm=[none], cost=[not available])
- HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
- HiveProject(ca_address_sk=[$0], ca_state=[CAST(_UTF-16LE'NY':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"):VARCHAR(2147483647) CHARACTER SET "UTF-16LE"])
- HiveFilter(condition=[=($8, _UTF-16LE'NY')])
- HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
- HiveJoin(condition=[=($0, $7)], joinType=[inner], algorithm=[none], cost=[not available])
- HiveProject(cs_ship_date_sk=[$2], cs_ship_addr_sk=[$10], cs_call_center_sk=[$11], cs_warehouse_sk=[$14], cs_order_number=[$17], cs_ext_ship_cost=[$28], cs_net_profit=[$33])
- HiveFilter(condition=[AND(IS NOT NULL($2), IS NOT NULL($11), IS NOT NULL($10))])
- HiveTableScan(table=[[default, catalog_sales]], table:alias=[cs1])
- HiveProject(d_date_sk=[$0], d_date=[$2])
- HiveFilter(condition=[BETWEEN(false, CAST($2):TIMESTAMP(9), 2001-04-01 00:00:00:TIMESTAMP(9), 2001-05-31 00:00:00:TIMESTAMP(9))])
- HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
- HiveProject(cc_call_center_sk=[$0], cc_county=[$25])
- HiveFilter(condition=[IN($25, _UTF-16LE'Ziebach County':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'Levy County':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'Huron County':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'Franklin Parish':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'Daviess County':VARCHAR(2147483647) CHARACTER SET "UTF-16LE")])
- HiveTableScan(table=[[default, call_center]], table:alias=[call_center])
- HiveProject(cs_warehouse_sk=[$14], cs_order_number=[$17])
- HiveFilter(condition=[IS NOT NULL($14)])
- HiveTableScan(table=[[default, catalog_sales]], table:alias=[cs2])
- HiveProject(literalTrue=[true], cr_order_number0=[$0])
- HiveAggregate(group=[{16}])
- HiveTableScan(table=[[default, catalog_returns]], table:alias=[cr1])
+ HiveAntiJoin(condition=[=($4, $14)], joinType=[anti])
+ HiveSemiJoin(condition=[AND(<>($3, $13), =($4, $14))], joinType=[semi])
+ HiveProject(cs_ship_date_sk=[$2], cs_ship_addr_sk=[$3], cs_call_center_sk=[$4], cs_warehouse_sk=[$5], cs_order_number=[$6], cs_ext_ship_cost=[$7], cs_net_profit=[$8], d_date_sk=[$9], d_date=[$10], ca_address_sk=[$0], ca_state=[$1], cc_call_center_sk=[$11], cc_county=[$12])
+ HiveJoin(condition=[=($4, $11)], joinType=[inner], algorithm=[none], cost=[not available])
+ HiveJoin(condition=[=($3, $0)], joinType=[inner], algorithm=[none], cost=[not available])
+ HiveProject(ca_address_sk=[$0], ca_state=[CAST(_UTF-16LE'NY':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"):VARCHAR(2147483647) CHARACTER SET "UTF-16LE"])
+ HiveFilter(condition=[=($8, _UTF-16LE'NY')])
+ HiveTableScan(table=[[default, customer_address]], table:alias=[customer_address])
+ HiveJoin(condition=[=($0, $7)], joinType=[inner], algorithm=[none], cost=[not available])
+ HiveProject(cs_ship_date_sk=[$2], cs_ship_addr_sk=[$10], cs_call_center_sk=[$11], cs_warehouse_sk=[$14], cs_order_number=[$17], cs_ext_ship_cost=[$28], cs_net_profit=[$33])
+ HiveFilter(condition=[AND(IS NOT NULL($2), IS NOT NULL($11), IS NOT NULL($10))])
+ HiveTableScan(table=[[default, catalog_sales]], table:alias=[cs1])
+ HiveProject(d_date_sk=[$0], d_date=[$2])
+ HiveFilter(condition=[BETWEEN(false, CAST($2):TIMESTAMP(9), 2001-04-01 00:00:00:TIMESTAMP(9), 2001-05-31 00:00:00:TIMESTAMP(9))])
+ HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+ HiveProject(cc_call_center_sk=[$0], cc_county=[$25])
+ HiveFilter(condition=[IN($25, _UTF-16LE'Ziebach County':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'Levy County':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'Huron County':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'Franklin Parish':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'Daviess County':VARCHAR(2147483647) CHARACTER SET "UTF-16LE")])
+ HiveTableScan(table=[[default, call_center]], table:alias=[call_center])
+ HiveProject(cs_warehouse_sk=[$14], cs_order_number=[$17])
+ HiveFilter(condition=[IS NOT NULL($14)])
+ HiveTableScan(table=[[default, catalog_sales]], table:alias=[cs2])
+ HiveProject(literalTrue=[true], cr_order_number0=[$16])
+ HiveTableScan(table=[[default, catalog_returns]], table:alias=[cr1])
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query69.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query69.q.out
index 761532c..c39199e 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query69.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query69.q.out
@@ -109,48 +109,45 @@ POSTHOOK: Output: hdfs://### HDFS PATH ###
CBO PLAN:
HiveSortLimit(sort0=[$0], sort1=[$1], sort2=[$2], sort3=[$4], sort4=[$6], dir0=[ASC], dir1=[ASC], dir2=[ASC], dir3=[ASC], dir4=[ASC], fetch=[100])
HiveProject(cd_gender=[$0], cd_marital_status=[$1], cd_education_status=[$2], cnt1=[$5], cd_purchase_estimate=[$3], cnt2=[$5], cd_credit_rating=[$4], cnt3=[$5])
- HiveAggregate(group=[{10, 11, 12, 13, 14}], agg#0=[count()])
- HiveJoin(condition=[=($9, $1)], joinType=[inner], algorithm=[none], cost=[not available])
- HiveProject(c_customer_sk=[$0], c_current_cdemo_sk=[$1], c_current_addr_sk=[$2], ca_address_sk=[$3], ca_state=[$4], literalTrue=[$5], ws_bill_customer_sk0=[$6], literalTrue0=[$7], cs_ship_customer_sk0=[$8])
- HiveFilter(condition=[IS NULL($7)])
- HiveJoin(condition=[=($0, $8)], joinType=[left], algorithm=[none], cost=[not available])
- HiveProject(c_customer_sk=[$0], c_current_cdemo_sk=[$1], c_current_addr_sk=[$2], ca_address_sk=[$3], ca_state=[$4], literalTrue=[$5], ws_bill_customer_sk0=[$6])
- HiveFilter(condition=[IS NULL($5)])
- HiveJoin(condition=[=($0, $6)], joinType=[left], algorithm=[none], cost=[not available])
- HiveSemiJoin(condition=[=($0, $5)], joinType=[semi])
- HiveJoin(condition=[=($2, $3)], joinType=[inner], algorithm=[none], cost=[not available])
- HiveProject(c_customer_sk=[$0], c_current_cdemo_sk=[$2], c_current_addr_sk=[$4])
- HiveFilter(condition=[AND(IS NOT NULL($2), IS NOT NULL($4))])
- HiveTableScan(table=[[default, customer]], table:alias=[c])
- HiveProject(ca_address_sk=[$0], ca_state=[$8])
- HiveFilter(condition=[IN($8, _UTF-16LE'CO':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'IL':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'MN':VARCHAR(2147483647) CHARACTER SET "UTF-16LE")])
- HiveTableScan(table=[[default, customer_address]], table:alias=[ca])
- HiveProject(ss_customer_sk0=[$1])
- HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
- HiveProject(ss_sold_date_sk=[$0], ss_customer_sk=[$3])
- HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($3))])
- HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
- HiveProject(d_date_sk=[$0])
- HiveFilter(condition=[AND(=($6, 1999), BETWEEN(false, $8, 1, 3))])
- HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
- HiveProject(literalTrue=[true], ws_bill_customer_sk0=[$0])
- HiveAggregate(group=[{1}])
- HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
- HiveProject(ws_sold_date_sk=[$0], ws_bill_customer_sk=[$4])
- HiveFilter(condition=[AND(IS NOT NULL($4), IS NOT NULL($0))])
- HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
- HiveProject(d_date_sk=[$0])
- HiveFilter(condition=[AND(=($6, 1999), BETWEEN(false, $8, 1, 3))])
- HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
- HiveProject(literalTrue=[true], cs_ship_customer_sk0=[$0])
- HiveAggregate(group=[{1}])
- HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
- HiveProject(cs_sold_date_sk=[$0], cs_ship_customer_sk=[$7])
- HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($0))])
- HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
- HiveProject(d_date_sk=[$0])
- HiveFilter(condition=[AND(=($6, 1999), BETWEEN(false, $8, 1, 3))])
- HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
- HiveProject(cd_demo_sk=[$0], cd_gender=[$1], cd_marital_status=[$2], cd_education_status=[$3], cd_purchase_estimate=[$4], cd_credit_rating=[$5])
- HiveTableScan(table=[[default, customer_demographics]], table:alias=[customer_demographics])
+ HiveAggregate(group=[{6, 7, 8, 9, 10}], agg#0=[count()])
+ HiveAntiJoin(condition=[=($0, $14)], joinType=[anti])
+ HiveProject(c_customer_sk=[$0], c_current_cdemo_sk=[$1], c_current_addr_sk=[$2], ca_address_sk=[$3], ca_state=[$4], cd_demo_sk=[$7], cd_gender=[$8], cd_marital_status=[$9], cd_education_status=[$10], cd_purchase_estimate=[$11], cd_credit_rating=[$12], literalTrue=[$5], ws_bill_customer_sk0=[$6])
+ HiveJoin(condition=[=($7, $1)], joinType=[inner], algorithm=[none], cost=[not available])
+ HiveProject(c_customer_sk=[$0], c_current_cdemo_sk=[$1], c_current_addr_sk=[$2], ca_address_sk=[$3], ca_state=[$4], literalTrue=[$5], ws_bill_customer_sk0=[$6])
+ HiveFilter(condition=[IS NULL($5)])
+ HiveJoin(condition=[=($0, $6)], joinType=[left], algorithm=[none], cost=[not available])
+ HiveSemiJoin(condition=[=($0, $5)], joinType=[semi])
+ HiveJoin(condition=[=($2, $3)], joinType=[inner], algorithm=[none], cost=[not available])
+ HiveProject(c_customer_sk=[$0], c_current_cdemo_sk=[$2], c_current_addr_sk=[$4])
+ HiveFilter(condition=[AND(IS NOT NULL($2), IS NOT NULL($4))])
+ HiveTableScan(table=[[default, customer]], table:alias=[c])
+ HiveProject(ca_address_sk=[$0], ca_state=[$8])
+ HiveFilter(condition=[IN($8, _UTF-16LE'CO':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'IL':VARCHAR(2147483647) CHARACTER SET "UTF-16LE", _UTF-16LE'MN':VARCHAR(2147483647) CHARACTER SET "UTF-16LE")])
+ HiveTableScan(table=[[default, customer_address]], table:alias=[ca])
+ HiveProject(ss_customer_sk0=[$1])
+ HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+ HiveProject(ss_sold_date_sk=[$0], ss_customer_sk=[$3])
+ HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($3))])
+ HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+ HiveProject(d_date_sk=[$0])
+ HiveFilter(condition=[AND(=($6, 1999), BETWEEN(false, $8, 1, 3))])
+ HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+ HiveProject(literalTrue=[true], ws_bill_customer_sk0=[$1])
+ HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+ HiveProject(ws_sold_date_sk=[$0], ws_bill_customer_sk=[$4])
+ HiveFilter(condition=[AND(IS NOT NULL($4), IS NOT NULL($0))])
+ HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
+ HiveProject(d_date_sk=[$0])
+ HiveFilter(condition=[AND(=($6, 1999), BETWEEN(false, $8, 1, 3))])
+ HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
+ HiveProject(cd_demo_sk=[$0], cd_gender=[$1], cd_marital_status=[$2], cd_education_status=[$3], cd_purchase_estimate=[$4], cd_credit_rating=[$5])
+ HiveTableScan(table=[[default, customer_demographics]], table:alias=[customer_demographics])
+ HiveProject(literalTrue=[true], cs_ship_customer_sk0=[$1])
+ HiveJoin(condition=[=($0, $2)], joinType=[inner], algorithm=[none], cost=[not available])
+ HiveProject(cs_sold_date_sk=[$0], cs_ship_customer_sk=[$7])
+ HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($0))])
+ HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
+ HiveProject(d_date_sk=[$0])
+ HiveFilter(condition=[AND(=($6, 1999), BETWEEN(false, $8, 1, 3))])
+ HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query78.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query78.q.out
index 14028ff..b893ae7 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query78.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query78.q.out
@@ -142,13 +142,12 @@ HiveSortLimit(fetch=[100])
HiveFilter(condition=[=($6, 2000)])
HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$1], ss_customer_sk=[$2], ss_quantity=[$4], ss_wholesale_cost=[$5], ss_sales_price=[$6])
- HiveFilter(condition=[IS NULL($8)])
- HiveJoin(condition=[AND(=($8, $3), =($1, $7))], joinType=[left], algorithm=[none], cost=[not available])
- HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2], ss_customer_sk=[$3], ss_ticket_number=[$9], ss_quantity=[$10], ss_wholesale_cost=[$11], ss_sales_price=[$13])
- HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($3))])
- HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
- HiveProject(sr_item_sk=[$2], sr_ticket_number=[$9])
- HiveTableScan(table=[[default, store_returns]], table:alias=[store_returns])
+ HiveAntiJoin(condition=[AND(=($8, $3), =($1, $7))], joinType=[anti])
+ HiveProject(ss_sold_date_sk=[$0], ss_item_sk=[$2], ss_customer_sk=[$3], ss_ticket_number=[$9], ss_quantity=[$10], ss_wholesale_cost=[$11], ss_sales_price=[$13])
+ HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($3))])
+ HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales])
+ HiveProject(sr_item_sk=[$2], sr_ticket_number=[$9])
+ HiveTableScan(table=[[default, store_returns]], table:alias=[store_returns])
HiveProject($f2=[$1], $f3=[$2], $f4=[$3], $f5=[$4])
HiveFilter(condition=[>($2, 0)])
HiveProject(cs_item_sk=[$1], cs_bill_customer_sk=[$0], $f2=[$2], $f3=[$3], $f4=[$4])
@@ -158,13 +157,12 @@ HiveSortLimit(fetch=[100])
HiveFilter(condition=[=($6, 2000)])
HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
HiveProject(cs_sold_date_sk=[$0], cs_bill_customer_sk=[$1], cs_item_sk=[$2], cs_quantity=[$4], cs_wholesale_cost=[$5], cs_sales_price=[$6])
- HiveFilter(condition=[IS NULL($8)])
- HiveJoin(condition=[AND(=($8, $3), =($2, $7))], joinType=[left], algorithm=[none], cost=[not available])
- HiveProject(cs_sold_date_sk=[$0], cs_bill_customer_sk=[$3], cs_item_sk=[$15], cs_order_number=[$17], cs_quantity=[$18], cs_wholesale_cost=[$19], cs_sales_price=[$21])
- HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($3))])
- HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
- HiveProject(cr_item_sk=[$2], cr_order_number=[$16])
- HiveTableScan(table=[[default, catalog_returns]], table:alias=[catalog_returns])
+ HiveAntiJoin(condition=[AND(=($8, $3), =($2, $7))], joinType=[anti])
+ HiveProject(cs_sold_date_sk=[$0], cs_bill_customer_sk=[$3], cs_item_sk=[$15], cs_order_number=[$17], cs_quantity=[$18], cs_wholesale_cost=[$19], cs_sales_price=[$21])
+ HiveFilter(condition=[AND(IS NOT NULL($0), IS NOT NULL($3))])
+ HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales])
+ HiveProject(cr_item_sk=[$2], cr_order_number=[$16])
+ HiveTableScan(table=[[default, catalog_returns]], table:alias=[catalog_returns])
HiveProject($f1=[$0], $f2=[$1], $f2_0=[$2], $f3=[$3], $f4=[$4])
HiveFilter(condition=[>($2, 0)])
HiveAggregate(group=[{2, 3}], agg#0=[sum($4)], agg#1=[sum($5)], agg#2=[sum($6)])
@@ -173,11 +171,10 @@ HiveSortLimit(fetch=[100])
HiveFilter(condition=[=($6, 2000)])
HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim])
HiveProject(ws_sold_date_sk=[$0], ws_item_sk=[$1], ws_bill_customer_sk=[$2], ws_quantity=[$4], ws_wholesale_cost=[$5], ws_sales_price=[$6])
- HiveFilter(condition=[IS NULL($8)])
- HiveJoin(condition=[AND(=($8, $3), =($1, $7))], joinType=[left], algorithm=[none], cost=[not available])
- HiveProject(ws_sold_date_sk=[$0], ws_item_sk=[$3], ws_bill_customer_sk=[$4], ws_order_number=[$17], ws_quantity=[$18], ws_wholesale_cost=[$19], ws_sales_price=[$21])
- HiveFilter(condition=[AND(IS NOT NULL($4), IS NOT NULL($0))])
- HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
- HiveProject(wr_item_sk=[$2], wr_order_number=[$13])
- HiveTableScan(table=[[default, web_returns]], table:alias=[web_returns])
+ HiveAntiJoin(condition=[AND(=($8, $3), =($1, $7))], joinType=[anti])
+ HiveProject(ws_sold_date_sk=[$0], ws_item_sk=[$3], ws_bill_customer_sk=[$4], ws_order_number=[$17], ws_quantity=[$18], ws_wholesale_cost=[$19], ws_sales_price=[$21])
+ HiveFilter(condition=[AND(IS NOT NULL($4), IS NOT NULL($0))])
+ HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales])
+ HiveProject(wr_item_sk=[$2], wr_order_number=[$13])
+ HiveTableScan(table=[[default, web_returns]], table:alias=[web_returns])
diff --git a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query94.q.out b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query94.q.out
index ef108a6..8210cec 100644
--- a/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query94.q.out
+++ b/ql/src/test/results/clientpositive/perf/tez/constraints/cbo_query94.q.out
@@ -68,29 +68,27 @@ POSTHOOK: Input: default@web_site
... 2593 lines suppressed ...