mirror of
https://github.com/apache/impala.git
synced 2026-01-09 06:05:09 -05:00
IMPALA-1270: add distinct aggregation to semi joins
When generating plans with left semi/anti joins (typically resulting from subquery rewrites), the planner now considers inserting a distinct aggregation on the inner side of the join. The decision is based on whether that aggregation would reduce the number of rows by more than 75%. This is fairly conservative and the optimization might be beneficial for smaller reductions, but the conservative threshold is chosen to reduce the number of potential plan regressions. The aggregation can both reduce the # of rows and the width of the rows, by projecting out unneeded slots. ENABLE_DISTINCT_SEMI_JOIN_OPTIMIZATION query option is added to allow toggling the optimization. Tests: * Add positive and negative planner tests for various cases - including semi/anti joins, missing stats, broadcast/shuffle, different numbers of join predicates. * Add some end-to-end tests to verify plans execute correctly. Change-Id: Icbb955e805d9e764edf11c57b98f341b88a37fcc Reviewed-on: http://gerrit.cloudera.org:8080/16180 Reviewed-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com> Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
This commit is contained in:
committed by
Impala Public Jenkins
parent
0b5a9889ee
commit
63f5e8ec00
@@ -910,6 +910,10 @@ Status impala::SetQueryOption(const string& key, const string& value,
|
||||
query_options->__set_async_codegen(IsTrue(value));
|
||||
break;
|
||||
}
|
||||
case TImpalaQueryOptions::ENABLE_DISTINCT_SEMI_JOIN_OPTIMIZATION: {
|
||||
query_options->__set_enable_distinct_semi_join_optimization(IsTrue(value));
|
||||
break;
|
||||
}
|
||||
default:
|
||||
if (IsRemovedQueryOption(key)) {
|
||||
LOG(WARNING) << "Ignoring attempt to set removed query option '" << key << "'";
|
||||
|
||||
@@ -47,7 +47,7 @@ typedef std::unordered_map<string, beeswax::TQueryOptionLevel::type>
|
||||
// time we add or remove a query option to/from the enum TImpalaQueryOptions.
|
||||
#define QUERY_OPTS_TABLE\
|
||||
DCHECK_EQ(_TImpalaQueryOptions_VALUES_TO_NAMES.size(),\
|
||||
TImpalaQueryOptions::ASYNC_CODEGEN + 1);\
|
||||
TImpalaQueryOptions::ENABLE_DISTINCT_SEMI_JOIN_OPTIMIZATION + 1);\
|
||||
REMOVED_QUERY_OPT_FN(abort_on_default_limit_exceeded, ABORT_ON_DEFAULT_LIMIT_EXCEEDED)\
|
||||
QUERY_OPT_FN(abort_on_error, ABORT_ON_ERROR, TQueryOptionLevel::REGULAR)\
|
||||
REMOVED_QUERY_OPT_FN(allow_unsupported_formats, ALLOW_UNSUPPORTED_FORMATS)\
|
||||
@@ -203,6 +203,8 @@ typedef std::unordered_map<string, beeswax::TQueryOptionLevel::type>
|
||||
QUERY_OPT_FN(enabled_runtime_filter_types, ENABLED_RUNTIME_FILTER_TYPES,\
|
||||
TQueryOptionLevel::ADVANCED)\
|
||||
QUERY_OPT_FN(async_codegen, ASYNC_CODEGEN, TQueryOptionLevel::DEVELOPMENT)\
|
||||
QUERY_OPT_FN(enable_distinct_semi_join_optimization,\
|
||||
ENABLE_DISTINCT_SEMI_JOIN_OPTIMIZATION, TQueryOptionLevel::ADVANCED)\
|
||||
;
|
||||
|
||||
/// Enforce practical limits on some query options to avoid undesired query state.
|
||||
|
||||
@@ -429,6 +429,9 @@ struct TQueryOptions {
|
||||
|
||||
// See comment in ImpalaService.thrift
|
||||
105: optional bool async_codegen = false;
|
||||
|
||||
// See comment in ImpalaService.thrift
|
||||
106: optional bool enable_distinct_semi_join_optimization = true;
|
||||
}
|
||||
|
||||
// Impala currently has two types of sessions: Beeswax and HiveServer2
|
||||
|
||||
@@ -538,6 +538,11 @@ enum TImpalaQueryOptions {
|
||||
|
||||
// Enable asynchronous codegen.
|
||||
ASYNC_CODEGEN = 104
|
||||
|
||||
// If true, the planner will consider adding a distinct aggregation to SEMI JOIN
|
||||
// operations. If false, disables the optimization (i.e. falls back to pre-Impala-4.0
|
||||
// behaviour).
|
||||
ENABLE_DISTINCT_SEMI_JOIN_OPTIMIZATION = 105
|
||||
}
|
||||
|
||||
// The summary of a DML statement.
|
||||
|
||||
@@ -58,13 +58,13 @@ import org.apache.impala.catalog.IcebergTable;
|
||||
import org.apache.impala.catalog.KuduTable;
|
||||
import org.apache.impala.catalog.TableLoadingException;
|
||||
import org.apache.impala.catalog.Type;
|
||||
import org.apache.impala.compat.MetastoreShim;
|
||||
import org.apache.impala.common.AnalysisException;
|
||||
import org.apache.impala.common.IdGenerator;
|
||||
import org.apache.impala.common.ImpalaException;
|
||||
import org.apache.impala.common.InternalException;
|
||||
import org.apache.impala.common.Pair;
|
||||
import org.apache.impala.common.RuntimeEnv;
|
||||
import org.apache.impala.compat.MetastoreShim;
|
||||
import org.apache.impala.planner.JoinNode;
|
||||
import org.apache.impala.planner.PlanNode;
|
||||
import org.apache.impala.rewrite.BetweenToCompoundRule;
|
||||
@@ -73,6 +73,7 @@ import org.apache.impala.rewrite.EqualityDisjunctsToInRule;
|
||||
import org.apache.impala.rewrite.ExprRewriteRule;
|
||||
import org.apache.impala.rewrite.ExprRewriter;
|
||||
import org.apache.impala.rewrite.ExtractCommonConjunctRule;
|
||||
import org.apache.impala.rewrite.ExtractCompoundVerticalBarExprRule;
|
||||
import org.apache.impala.rewrite.FoldConstantsRule;
|
||||
import org.apache.impala.rewrite.NormalizeBinaryPredicatesRule;
|
||||
import org.apache.impala.rewrite.NormalizeCountStarRule;
|
||||
@@ -80,7 +81,6 @@ import org.apache.impala.rewrite.NormalizeExprsRule;
|
||||
import org.apache.impala.rewrite.SimplifyCastStringToTimestamp;
|
||||
import org.apache.impala.rewrite.SimplifyConditionalsRule;
|
||||
import org.apache.impala.rewrite.SimplifyDistinctFromRule;
|
||||
import org.apache.impala.rewrite.ExtractCompoundVerticalBarExprRule;
|
||||
import org.apache.impala.service.FeSupport;
|
||||
import org.apache.impala.thrift.TAccessEvent;
|
||||
import org.apache.impala.thrift.TCatalogObjectType;
|
||||
@@ -875,6 +875,17 @@ public class Analyzer {
|
||||
return globalState_.descTbl.getSlotDesc(id);
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper to get all slot descriptors in list.
|
||||
*/
|
||||
public List<SlotDescriptor> getSlotDescs(List<SlotId> ids) {
|
||||
List<SlotDescriptor> result = new ArrayList<>(ids.size());
|
||||
for (SlotId id : ids) {
|
||||
result.add(getSlotDesc(id));
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
public int getNumTableRefs() { return tableRefMap_.size(); }
|
||||
public TableRef getTableRef(TupleId tid) { return tableRefMap_.get(tid); }
|
||||
public ExprRewriter getConstantFolder() { return globalState_.constantFolder_; }
|
||||
|
||||
@@ -1316,6 +1316,11 @@ abstract public class Expr extends TreeNode<Expr> implements ParseNode, Cloneabl
|
||||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Find all unique slot and/or tuple ids referenced by this expr tree.
|
||||
* @param tupleIds unique tuple IDs from this expr tree are appended here.
|
||||
* @param slotIds unique slot IDs from this expr tree are appended here.
|
||||
*/
|
||||
public void getIds(List<TupleId> tupleIds, List<SlotId> slotIds) {
|
||||
Set<TupleId> tupleIdSet = new HashSet<>();
|
||||
Set<SlotId> slotIdSet = new HashSet<>();
|
||||
|
||||
@@ -247,18 +247,28 @@ public class AggregationNode extends PlanNode {
|
||||
// limit the potential overestimation. We could, in future, improve this further
|
||||
// by recognizing functional dependencies.
|
||||
List<Expr> groupingExprs = aggInfo.getGroupingExprs();
|
||||
long aggInputCardinality = getAggInputCardinality();
|
||||
long numGroups = estimateNumGroups(groupingExprs, aggInputCardinality);
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("Node " + id_ + " numGroups= " + numGroups + " aggInputCardinality=" +
|
||||
aggInputCardinality + " for agg class " + aggInfo.debugString());
|
||||
}
|
||||
return numGroups;
|
||||
}
|
||||
|
||||
/**
|
||||
* Estimate the number of groups that will be present for the provided grouping
|
||||
* expressions and input cardinality.
|
||||
* Returns -1 if a reasonable cardinality estimate cannot be produced.
|
||||
*/
|
||||
public static long estimateNumGroups(
|
||||
List<Expr> groupingExprs, long aggInputCardinality) {
|
||||
if (groupingExprs.isEmpty()) {
|
||||
// Non-grouping aggregation class - always results in one group even if there are
|
||||
// zero input rows.
|
||||
return 1;
|
||||
}
|
||||
long numGroups = Expr.getNumDistinctValues(groupingExprs);
|
||||
// Sanity check the cardinality_ based on the input cardinality_.
|
||||
long aggInputCardinality = getAggInputCardinality();
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("Node " + id_ + " numGroups= " + numGroups + " aggInputCardinality=" +
|
||||
aggInputCardinality + " for agg class " + aggInfo.debugString());
|
||||
}
|
||||
if (numGroups == -1) {
|
||||
// A worst-case cardinality_ is better than an unknown cardinality_.
|
||||
// Note that this will still be -1 if the child's cardinality is unknown.
|
||||
|
||||
@@ -96,10 +96,20 @@ import com.google.common.collect.Sets;
|
||||
* The single-node plan needs to be wrapped in a plan fragment for it to be executable.
|
||||
*/
|
||||
public class SingleNodePlanner {
|
||||
// Controls whether a distinct aggregation should be inserted before a join input.
|
||||
// If the size of the distinct values after aggregation is less than or equal to
|
||||
// the original input size multiplied by this threshold, the distinct agg should be
|
||||
// inserted.
|
||||
private static final double JOIN_DISTINCT_THRESHOLD = 0.25;
|
||||
|
||||
private final static Logger LOG = LoggerFactory.getLogger(SingleNodePlanner.class);
|
||||
|
||||
private final PlannerContext ctx_;
|
||||
|
||||
// Set to true if single node planning added new value transfers to the
|
||||
// value transfer graph in 'analyzer'.
|
||||
private boolean valueTransferGraphNeedsUpdate_ = false;
|
||||
|
||||
public SingleNodePlanner(PlannerContext ctx) {
|
||||
ctx_ = ctx;
|
||||
}
|
||||
@@ -161,6 +171,14 @@ public class SingleNodePlanner {
|
||||
PlanNode singleNodePlan = createQueryPlan(queryStmt, analyzer,
|
||||
ctx_.getQueryOptions().isDisable_outermost_topn());
|
||||
Preconditions.checkNotNull(singleNodePlan);
|
||||
// Recompute the graph since we may have added new equivalences.
|
||||
if (valueTransferGraphNeedsUpdate_) {
|
||||
ctx_.getTimeline().markEvent("Recomputing value transfer graph");
|
||||
analyzer.computeValueTransferGraph();
|
||||
ctx_.getTimeline().markEvent("Value transfer graph computed");
|
||||
valueTransferGraphNeedsUpdate_ = false;
|
||||
}
|
||||
|
||||
return singleNodePlan;
|
||||
}
|
||||
|
||||
@@ -1808,6 +1826,12 @@ public class SingleNodePlanner {
|
||||
}
|
||||
analyzer.markConjunctsAssigned(otherJoinConjuncts);
|
||||
|
||||
if (analyzer.getQueryOptions().isEnable_distinct_semi_join_optimization() &&
|
||||
innerRef.getJoinOp().isLeftSemiJoin()) {
|
||||
inner =
|
||||
addDistinctToJoinInput(inner, analyzer, eqJoinConjuncts, otherJoinConjuncts);
|
||||
}
|
||||
|
||||
// Use a nested-loop join if there are no equi-join conjuncts, or if the inner
|
||||
// (build side) is a singular row src. A singular row src has a cardinality of 1, so
|
||||
// a nested-loop join is certainly cheaper than a hash join.
|
||||
@@ -1827,6 +1851,99 @@ public class SingleNodePlanner {
|
||||
return result;
|
||||
}
|
||||
|
||||
/**
|
||||
* Optionally add a aggregation node on top of 'joinInput' if it is cheaper to project
|
||||
* and aggregate the slots needed to evaluate the provided join conjuncts. This
|
||||
* is only safe to do if the join's results do not depend on the number of duplicate
|
||||
* values and if the join does not need to return any slots from 'joinInput'. E.g.
|
||||
* the inner of a left semi join satisfies both of those conditions.
|
||||
* @return the original 'joinInput' or its new AggregationNode parent.
|
||||
*/
|
||||
private PlanNode addDistinctToJoinInput(PlanNode joinInput, Analyzer analyzer,
|
||||
List<BinaryPredicate> eqJoinConjuncts, List<Expr> otherJoinConjuncts)
|
||||
throws InternalException, AnalysisException {
|
||||
List<Expr> allJoinConjuncts = new ArrayList<>();
|
||||
allJoinConjuncts.addAll(eqJoinConjuncts);
|
||||
allJoinConjuncts.addAll(otherJoinConjuncts);
|
||||
allJoinConjuncts = Expr.substituteList(
|
||||
allJoinConjuncts, joinInput.getOutputSmap(), analyzer, true);
|
||||
|
||||
// Identify the unique slots from the inner required by the join conjuncts. Since this
|
||||
// is a semi-join, the inner tuple is not returned from the join and we do not need
|
||||
// any other slots from the inner.
|
||||
List<SlotId> allSlotIds = new ArrayList<>();
|
||||
Expr.getIds(allJoinConjuncts, null, allSlotIds);
|
||||
List<TupleId> joinInputTupleIds = joinInput.getTupleIds();
|
||||
List<Expr> distinctExprs = new ArrayList<>();
|
||||
double estDistinctTupleSize = 0;
|
||||
for (SlotDescriptor slot : analyzer.getSlotDescs(allSlotIds)) {
|
||||
if (joinInputTupleIds.contains(slot.getParent().getId())) {
|
||||
distinctExprs.add(new SlotRef(slot));
|
||||
}
|
||||
}
|
||||
|
||||
// If there are no join predicates, this can be more efficiently handled by
|
||||
// inserting a limit in the plan (since the first row returned from 'joinInput'
|
||||
// will satisfy the join predicates).
|
||||
if (distinctExprs.isEmpty()) {
|
||||
joinInput.setLimit(1);
|
||||
return joinInput;
|
||||
}
|
||||
long numDistinct = AggregationNode.estimateNumGroups(distinctExprs,
|
||||
joinInput.getCardinality());
|
||||
if (numDistinct < 0 || joinInput.getCardinality() < 0) {
|
||||
// Default to not adding the aggregation if stats are missing.
|
||||
LOG.trace("addDistinctToJoinInput():: missing stats, will not add agg");
|
||||
return joinInput;
|
||||
}
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("addDistinctToJoinInput(): " + "numDistinct=" + numDistinct +
|
||||
" inputCardinality=" + joinInput.getCardinality());
|
||||
}
|
||||
|
||||
// Check to see if an aggregation would reduce input by enough to justify inserting
|
||||
// it. We factor in the average row size to account for the aggregate projecting
|
||||
// out slots. The agg would be ineffective if the input already have 0 or 1 rows.
|
||||
if (joinInput.getCardinality() <= 1 ||
|
||||
numDistinct > JOIN_DISTINCT_THRESHOLD * joinInput.getCardinality()) {
|
||||
return joinInput;
|
||||
}
|
||||
|
||||
// Set up an aggregation node to return only distinct slots.
|
||||
MultiAggregateInfo distinctAggInfo =
|
||||
new MultiAggregateInfo(distinctExprs, Collections.emptyList(), null);
|
||||
distinctAggInfo.analyze(analyzer);
|
||||
distinctAggInfo.materializeRequiredSlots(analyzer, new ExprSubstitutionMap());
|
||||
AggregationNode agg = new AggregationNode(
|
||||
ctx_.getNextNodeId(), joinInput, distinctAggInfo, AggPhase.FIRST);
|
||||
agg.init(analyzer);
|
||||
// Mark the agg as materializing the same table ref. This is required so that other
|
||||
// parts of planning, e.g. subplan generation, know that this plan tree materialized
|
||||
// the table ref.
|
||||
agg.setTblRefIds(joinInput.getTblRefIds());
|
||||
// All references to the input slots in join conjuncts must be replaced with
|
||||
// references to aggregate slots. The output smap from the aggregate info contains
|
||||
// these mappings, so we can add it to the output smap of the agg to ensure that
|
||||
// join conjuncts get replaced correctly.
|
||||
agg.setOutputSmap(ExprSubstitutionMap.compose(
|
||||
agg.getOutputSmap(), distinctAggInfo.getOutputSmap(), analyzer));
|
||||
|
||||
// Add value transfers between original slots and aggregate tuple so that runtime
|
||||
// filters can be pushed through the aggregation. We can defer updating the
|
||||
// value transfer graph until after the single node plan is constructed because
|
||||
// a precondition of calling this function is that the join does not return any
|
||||
// of the slots from this plan tree.
|
||||
for (int i = 0; i < distinctExprs.size(); ++i) {
|
||||
Expr distinctExpr = distinctExprs.get(i);
|
||||
SlotDescriptor outputSlot =
|
||||
distinctAggInfo.getAggClass(0).getResultTupleDesc().getSlots().get(i);
|
||||
analyzer.registerValueTransfer(
|
||||
((SlotRef)distinctExpr).getSlotId(), outputSlot.getId());
|
||||
valueTransferGraphNeedsUpdate_ = true;
|
||||
}
|
||||
return agg;
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a tree of PlanNodes for the given tblRef, which can be a BaseTableRef,
|
||||
* CollectionTableRef or an InlineViewRef.
|
||||
|
||||
@@ -432,6 +432,15 @@ public class PlannerTest extends PlannerTestBase {
|
||||
runPlannerTestFile("subquery-rewrite", options);
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests for the IMPALA-1270 optimization of automatically adding a distinct
|
||||
* agg to semi joins.
|
||||
*/
|
||||
@Test
|
||||
public void testSemiJoinDistinct() {
|
||||
runPlannerTestFile("semi-join-distinct");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUnion() {
|
||||
runPlannerTestFile("union");
|
||||
|
||||
@@ -40,7 +40,7 @@ PLAN-ROOT SINK
|
||||
| row-size=117B cardinality=17.56K
|
||||
|
|
||||
|--00:SCAN HDFS [tpch.customer c]
|
||||
| partitions=1/1 files=1 size=23.08MB
|
||||
| HDFS partitions=1/1 files=1 size=23.08MB
|
||||
| predicates: c.c_mktsegment = 'BUILDING'
|
||||
| row-size=29B cardinality=30.00K
|
||||
|
|
||||
@@ -50,13 +50,13 @@ PLAN-ROOT SINK
|
||||
| row-size=88B cardinality=57.58K
|
||||
|
|
||||
|--01:SCAN HDFS [tpch.orders o]
|
||||
| partitions=1/1 files=1 size=162.56MB
|
||||
| HDFS partitions=1/1 files=1 size=162.56MB
|
||||
| predicates: o_orderdate < '1995-03-15'
|
||||
| runtime filters: RF000 -> o.o_custkey
|
||||
| row-size=42B cardinality=150.00K
|
||||
|
|
||||
02:SCAN HDFS [tpch.lineitem l]
|
||||
partitions=1/1 files=1 size=718.94MB
|
||||
HDFS partitions=1/1 files=1 size=718.94MB
|
||||
predicates: l_shipdate > '1995-03-15'
|
||||
runtime filters: RF002 -> l.l_orderkey
|
||||
row-size=46B cardinality=600.12K
|
||||
@@ -91,7 +91,7 @@ PLAN-ROOT SINK
|
||||
|--08:EXCHANGE [BROADCAST]
|
||||
| |
|
||||
| 00:SCAN HDFS [tpch.customer c]
|
||||
| partitions=1/1 files=1 size=23.08MB
|
||||
| HDFS partitions=1/1 files=1 size=23.08MB
|
||||
| predicates: c.c_mktsegment = 'BUILDING'
|
||||
| row-size=29B cardinality=30.00K
|
||||
|
|
||||
@@ -103,13 +103,13 @@ PLAN-ROOT SINK
|
||||
|--07:EXCHANGE [BROADCAST]
|
||||
| |
|
||||
| 01:SCAN HDFS [tpch.orders o]
|
||||
| partitions=1/1 files=1 size=162.56MB
|
||||
| HDFS partitions=1/1 files=1 size=162.56MB
|
||||
| predicates: o_orderdate < '1995-03-15'
|
||||
| runtime filters: RF000 -> o.o_custkey
|
||||
| row-size=42B cardinality=150.00K
|
||||
|
|
||||
02:SCAN HDFS [tpch.lineitem l]
|
||||
partitions=1/1 files=1 size=718.94MB
|
||||
HDFS partitions=1/1 files=1 size=718.94MB
|
||||
predicates: l_shipdate > '1995-03-15'
|
||||
runtime filters: RF002 -> l.l_orderkey
|
||||
row-size=46B cardinality=600.12K
|
||||
@@ -156,7 +156,7 @@ PLAN-ROOT SINK
|
||||
| row-size=117B cardinality=575.77K
|
||||
|
|
||||
|--02:SCAN HDFS [tpch.lineitem l]
|
||||
| partitions=1/1 files=1 size=718.94MB
|
||||
| HDFS partitions=1/1 files=1 size=718.94MB
|
||||
| predicates: l_shipdate > '1995-03-15'
|
||||
| row-size=46B cardinality=600.12K
|
||||
|
|
||||
@@ -166,13 +166,13 @@ PLAN-ROOT SINK
|
||||
| row-size=71B cardinality=150.00K
|
||||
|
|
||||
|--01:SCAN HDFS [tpch.orders o]
|
||||
| partitions=1/1 files=1 size=162.56MB
|
||||
| HDFS partitions=1/1 files=1 size=162.56MB
|
||||
| predicates: o_orderdate < '1995-03-15'
|
||||
| runtime filters: RF000 -> o.o_orderkey
|
||||
| row-size=42B cardinality=150.00K
|
||||
|
|
||||
00:SCAN HDFS [tpch.customer c]
|
||||
partitions=1/1 files=1 size=23.08MB
|
||||
HDFS partitions=1/1 files=1 size=23.08MB
|
||||
predicates: c.c_mktsegment = 'BUILDING'
|
||||
runtime filters: RF002 -> c.c_custkey
|
||||
row-size=29B cardinality=30.00K
|
||||
@@ -207,7 +207,7 @@ PLAN-ROOT SINK
|
||||
|--08:EXCHANGE [BROADCAST]
|
||||
| |
|
||||
| 02:SCAN HDFS [tpch.lineitem l]
|
||||
| partitions=1/1 files=1 size=718.94MB
|
||||
| HDFS partitions=1/1 files=1 size=718.94MB
|
||||
| predicates: l_shipdate > '1995-03-15'
|
||||
| row-size=46B cardinality=600.12K
|
||||
|
|
||||
@@ -219,13 +219,13 @@ PLAN-ROOT SINK
|
||||
|--07:EXCHANGE [BROADCAST]
|
||||
| |
|
||||
| 01:SCAN HDFS [tpch.orders o]
|
||||
| partitions=1/1 files=1 size=162.56MB
|
||||
| HDFS partitions=1/1 files=1 size=162.56MB
|
||||
| predicates: o_orderdate < '1995-03-15'
|
||||
| runtime filters: RF000 -> o.o_orderkey
|
||||
| row-size=42B cardinality=150.00K
|
||||
|
|
||||
00:SCAN HDFS [tpch.customer c]
|
||||
partitions=1/1 files=1 size=23.08MB
|
||||
HDFS partitions=1/1 files=1 size=23.08MB
|
||||
predicates: c.c_mktsegment = 'BUILDING'
|
||||
runtime filters: RF002 -> c.c_custkey
|
||||
row-size=29B cardinality=30.00K
|
||||
@@ -274,7 +274,7 @@ PLAN-ROOT SINK
|
||||
| row-size=134B cardinality=115.16K
|
||||
|
|
||||
|--05:SCAN HDFS [tpch.region]
|
||||
| partitions=1/1 files=1 size=384B
|
||||
| HDFS partitions=1/1 files=1 size=384B
|
||||
| predicates: r_name = 'ASIA'
|
||||
| row-size=21B cardinality=1
|
||||
|
|
||||
@@ -284,7 +284,7 @@ PLAN-ROOT SINK
|
||||
| row-size=113B cardinality=575.77K
|
||||
|
|
||||
|--04:SCAN HDFS [tpch.nation]
|
||||
| partitions=1/1 files=1 size=2.15KB
|
||||
| HDFS partitions=1/1 files=1 size=2.15KB
|
||||
| runtime filters: RF000 -> n_regionkey
|
||||
| row-size=23B cardinality=25
|
||||
|
|
||||
@@ -294,7 +294,7 @@ PLAN-ROOT SINK
|
||||
| row-size=90B cardinality=575.77K
|
||||
|
|
||||
|--03:SCAN HDFS [tpch.supplier s]
|
||||
| partitions=1/1 files=1 size=1.33MB
|
||||
| HDFS partitions=1/1 files=1 size=1.33MB
|
||||
| runtime filters: RF002 -> s_nationkey
|
||||
| row-size=10B cardinality=10.00K
|
||||
|
|
||||
@@ -304,7 +304,7 @@ PLAN-ROOT SINK
|
||||
| row-size=80B cardinality=575.77K
|
||||
|
|
||||
|--00:SCAN HDFS [tpch.customer]
|
||||
| partitions=1/1 files=1 size=23.08MB
|
||||
| HDFS partitions=1/1 files=1 size=23.08MB
|
||||
| runtime filters: RF002 -> tpch.customer.c_nationkey, RF004 -> c_nationkey
|
||||
| row-size=10B cardinality=150.00K
|
||||
|
|
||||
@@ -314,13 +314,13 @@ PLAN-ROOT SINK
|
||||
| row-size=70B cardinality=575.77K
|
||||
|
|
||||
|--01:SCAN HDFS [tpch.orders o]
|
||||
| partitions=1/1 files=1 size=162.56MB
|
||||
| HDFS partitions=1/1 files=1 size=162.56MB
|
||||
| predicates: o_orderdate < '1995-01-01', o_orderdate >= '1994-01-01'
|
||||
| runtime filters: RF008 -> o_custkey
|
||||
| row-size=38B cardinality=150.00K
|
||||
|
|
||||
02:SCAN HDFS [tpch.lineitem l]
|
||||
partitions=1/1 files=1 size=718.94MB
|
||||
HDFS partitions=1/1 files=1 size=718.94MB
|
||||
runtime filters: RF005 -> l_suppkey, RF010 -> l_orderkey
|
||||
row-size=32B cardinality=6.00M
|
||||
---- DISTRIBUTEDPLAN
|
||||
@@ -354,7 +354,7 @@ PLAN-ROOT SINK
|
||||
|--17:EXCHANGE [BROADCAST]
|
||||
| |
|
||||
| 05:SCAN HDFS [tpch.region]
|
||||
| partitions=1/1 files=1 size=384B
|
||||
| HDFS partitions=1/1 files=1 size=384B
|
||||
| predicates: r_name = 'ASIA'
|
||||
| row-size=21B cardinality=1
|
||||
|
|
||||
@@ -366,7 +366,7 @@ PLAN-ROOT SINK
|
||||
|--16:EXCHANGE [BROADCAST]
|
||||
| |
|
||||
| 04:SCAN HDFS [tpch.nation]
|
||||
| partitions=1/1 files=1 size=2.15KB
|
||||
| HDFS partitions=1/1 files=1 size=2.15KB
|
||||
| runtime filters: RF000 -> n_regionkey
|
||||
| row-size=23B cardinality=25
|
||||
|
|
||||
@@ -378,7 +378,7 @@ PLAN-ROOT SINK
|
||||
|--15:EXCHANGE [BROADCAST]
|
||||
| |
|
||||
| 03:SCAN HDFS [tpch.supplier s]
|
||||
| partitions=1/1 files=1 size=1.33MB
|
||||
| HDFS partitions=1/1 files=1 size=1.33MB
|
||||
| runtime filters: RF002 -> s_nationkey
|
||||
| row-size=10B cardinality=10.00K
|
||||
|
|
||||
@@ -390,7 +390,7 @@ PLAN-ROOT SINK
|
||||
|--14:EXCHANGE [BROADCAST]
|
||||
| |
|
||||
| 00:SCAN HDFS [tpch.customer]
|
||||
| partitions=1/1 files=1 size=23.08MB
|
||||
| HDFS partitions=1/1 files=1 size=23.08MB
|
||||
| runtime filters: RF002 -> tpch.customer.c_nationkey, RF004 -> c_nationkey
|
||||
| row-size=10B cardinality=150.00K
|
||||
|
|
||||
@@ -402,13 +402,13 @@ PLAN-ROOT SINK
|
||||
|--13:EXCHANGE [BROADCAST]
|
||||
| |
|
||||
| 01:SCAN HDFS [tpch.orders o]
|
||||
| partitions=1/1 files=1 size=162.56MB
|
||||
| HDFS partitions=1/1 files=1 size=162.56MB
|
||||
| predicates: o_orderdate < '1995-01-01', o_orderdate >= '1994-01-01'
|
||||
| runtime filters: RF008 -> o_custkey
|
||||
| row-size=38B cardinality=150.00K
|
||||
|
|
||||
02:SCAN HDFS [tpch.lineitem l]
|
||||
partitions=1/1 files=1 size=718.94MB
|
||||
HDFS partitions=1/1 files=1 size=718.94MB
|
||||
runtime filters: RF005 -> l_suppkey, RF010 -> l_orderkey
|
||||
row-size=32B cardinality=6.00M
|
||||
====
|
||||
@@ -446,7 +446,7 @@ PLAN-ROOT SINK
|
||||
| row-size=325B cardinality=1.01K
|
||||
|
|
||||
|--04:SCAN HDFS [tpch.region r]
|
||||
| partitions=1/1 files=1 size=384B
|
||||
| HDFS partitions=1/1 files=1 size=384B
|
||||
| predicates: r.r_name = 'EUROPE'
|
||||
| row-size=21B cardinality=1
|
||||
|
|
||||
@@ -456,7 +456,7 @@ PLAN-ROOT SINK
|
||||
| row-size=304B cardinality=5.05K
|
||||
|
|
||||
|--03:SCAN HDFS [tpch.nation n]
|
||||
| partitions=1/1 files=1 size=2.15KB
|
||||
| HDFS partitions=1/1 files=1 size=2.15KB
|
||||
| runtime filters: RF000 -> n.n_regionkey
|
||||
| row-size=23B cardinality=25
|
||||
|
|
||||
@@ -471,17 +471,17 @@ PLAN-ROOT SINK
|
||||
| | row-size=95B cardinality=5.05K
|
||||
| |
|
||||
| |--00:SCAN HDFS [tpch.part p]
|
||||
| | partitions=1/1 files=1 size=22.83MB
|
||||
| | HDFS partitions=1/1 files=1 size=22.83MB
|
||||
| | predicates: p.p_size = 15, p.p_type LIKE '%BRASS'
|
||||
| | row-size=71B cardinality=1.26K
|
||||
| |
|
||||
| 02:SCAN HDFS [tpch.partsupp ps]
|
||||
| partitions=1/1 files=1 size=112.71MB
|
||||
| HDFS partitions=1/1 files=1 size=112.71MB
|
||||
| runtime filters: RF006 -> ps.ps_partkey
|
||||
| row-size=24B cardinality=800.00K
|
||||
|
|
||||
01:SCAN HDFS [tpch.supplier s]
|
||||
partitions=1/1 files=1 size=1.33MB
|
||||
HDFS partitions=1/1 files=1 size=1.33MB
|
||||
runtime filters: RF002 -> s.s_nationkey, RF004 -> s.s_suppkey
|
||||
row-size=187B cardinality=10.00K
|
||||
---- DISTRIBUTEDPLAN
|
||||
@@ -497,7 +497,7 @@ PLAN-ROOT SINK
|
||||
|--12:EXCHANGE [BROADCAST]
|
||||
| |
|
||||
| 04:SCAN HDFS [tpch.region r]
|
||||
| partitions=1/1 files=1 size=384B
|
||||
| HDFS partitions=1/1 files=1 size=384B
|
||||
| predicates: r.r_name = 'EUROPE'
|
||||
| row-size=21B cardinality=1
|
||||
|
|
||||
@@ -509,7 +509,7 @@ PLAN-ROOT SINK
|
||||
|--11:EXCHANGE [BROADCAST]
|
||||
| |
|
||||
| 03:SCAN HDFS [tpch.nation n]
|
||||
| partitions=1/1 files=1 size=2.15KB
|
||||
| HDFS partitions=1/1 files=1 size=2.15KB
|
||||
| runtime filters: RF000 -> n.n_regionkey
|
||||
| row-size=23B cardinality=25
|
||||
|
|
||||
@@ -528,17 +528,17 @@ PLAN-ROOT SINK
|
||||
| |--09:EXCHANGE [BROADCAST]
|
||||
| | |
|
||||
| | 00:SCAN HDFS [tpch.part p]
|
||||
| | partitions=1/1 files=1 size=22.83MB
|
||||
| | HDFS partitions=1/1 files=1 size=22.83MB
|
||||
| | predicates: p.p_size = 15, p.p_type LIKE '%BRASS'
|
||||
| | row-size=71B cardinality=1.26K
|
||||
| |
|
||||
| 02:SCAN HDFS [tpch.partsupp ps]
|
||||
| partitions=1/1 files=1 size=112.71MB
|
||||
| HDFS partitions=1/1 files=1 size=112.71MB
|
||||
| runtime filters: RF006 -> ps.ps_partkey
|
||||
| row-size=24B cardinality=800.00K
|
||||
|
|
||||
01:SCAN HDFS [tpch.supplier s]
|
||||
partitions=1/1 files=1 size=1.33MB
|
||||
HDFS partitions=1/1 files=1 size=1.33MB
|
||||
runtime filters: RF002 -> s.s_nationkey, RF004 -> s.s_suppkey
|
||||
row-size=187B cardinality=10.00K
|
||||
====
|
||||
@@ -577,12 +577,12 @@ PLAN-ROOT SINK
|
||||
| row-size=50B cardinality=150.00K
|
||||
|
|
||||
|--00:SCAN HDFS [tpch.orders]
|
||||
| partitions=1/1 files=1 size=162.56MB
|
||||
| HDFS partitions=1/1 files=1 size=162.56MB
|
||||
| predicates: o_orderdate < '1993-10-01', o_orderdate >= '1993-07-01'
|
||||
| row-size=50B cardinality=150.00K
|
||||
|
|
||||
01:SCAN HDFS [tpch.lineitem]
|
||||
partitions=1/1 files=1 size=718.94MB
|
||||
HDFS partitions=1/1 files=1 size=718.94MB
|
||||
predicates: l_commitdate < l_receiptdate
|
||||
runtime filters: RF000 -> l_orderkey
|
||||
row-size=52B cardinality=600.12K
|
||||
@@ -617,14 +617,14 @@ PLAN-ROOT SINK
|
||||
|--06:EXCHANGE [HASH(o_orderkey)]
|
||||
| |
|
||||
| 00:SCAN HDFS [tpch.orders]
|
||||
| partitions=1/1 files=1 size=162.56MB
|
||||
| HDFS partitions=1/1 files=1 size=162.56MB
|
||||
| predicates: o_orderdate < '1993-10-01', o_orderdate >= '1993-07-01'
|
||||
| row-size=50B cardinality=150.00K
|
||||
|
|
||||
05:EXCHANGE [HASH(l_orderkey)]
|
||||
|
|
||||
01:SCAN HDFS [tpch.lineitem]
|
||||
partitions=1/1 files=1 size=718.94MB
|
||||
HDFS partitions=1/1 files=1 size=718.94MB
|
||||
predicates: l_commitdate < l_receiptdate
|
||||
runtime filters: RF000 -> l_orderkey
|
||||
row-size=52B cardinality=600.12K
|
||||
@@ -652,11 +652,11 @@ PLAN-ROOT SINK
|
||||
| row-size=36B cardinality=7.50M
|
||||
|
|
||||
|--00:SCAN HDFS [tpch.orders]
|
||||
| partitions=1/1 files=1 size=162.56MB
|
||||
| HDFS partitions=1/1 files=1 size=162.56MB
|
||||
| row-size=28B cardinality=1.50M
|
||||
|
|
||||
01:SCAN HDFS [tpch.lineitem]
|
||||
partitions=1/1 files=1 size=718.94MB
|
||||
HDFS partitions=1/1 files=1 size=718.94MB
|
||||
row-size=8B cardinality=6.00M
|
||||
---- DISTRIBUTEDPLAN
|
||||
PLAN-ROOT SINK
|
||||
@@ -688,13 +688,13 @@ PLAN-ROOT SINK
|
||||
|--06:EXCHANGE [HASH(o_orderkey)]
|
||||
| |
|
||||
| 00:SCAN HDFS [tpch.orders]
|
||||
| partitions=1/1 files=1 size=162.56MB
|
||||
| HDFS partitions=1/1 files=1 size=162.56MB
|
||||
| row-size=28B cardinality=1.50M
|
||||
|
|
||||
05:EXCHANGE [HASH(l_orderkey)]
|
||||
|
|
||||
01:SCAN HDFS [tpch.lineitem]
|
||||
partitions=1/1 files=1 size=718.94MB
|
||||
HDFS partitions=1/1 files=1 size=718.94MB
|
||||
row-size=8B cardinality=6.00M
|
||||
====
|
||||
select o_orderpriority, count(*) as order_count
|
||||
@@ -720,11 +720,11 @@ PLAN-ROOT SINK
|
||||
| row-size=36B cardinality=6.00M
|
||||
|
|
||||
|--00:SCAN HDFS [tpch.orders]
|
||||
| partitions=1/1 files=1 size=162.56MB
|
||||
| HDFS partitions=1/1 files=1 size=162.56MB
|
||||
| row-size=28B cardinality=1.50M
|
||||
|
|
||||
01:SCAN HDFS [tpch.lineitem]
|
||||
partitions=1/1 files=1 size=718.94MB
|
||||
HDFS partitions=1/1 files=1 size=718.94MB
|
||||
row-size=8B cardinality=6.00M
|
||||
---- DISTRIBUTEDPLAN
|
||||
PLAN-ROOT SINK
|
||||
@@ -756,13 +756,13 @@ PLAN-ROOT SINK
|
||||
|--06:EXCHANGE [HASH(o_orderkey)]
|
||||
| |
|
||||
| 00:SCAN HDFS [tpch.orders]
|
||||
| partitions=1/1 files=1 size=162.56MB
|
||||
| HDFS partitions=1/1 files=1 size=162.56MB
|
||||
| row-size=28B cardinality=1.50M
|
||||
|
|
||||
05:EXCHANGE [HASH(l_orderkey)]
|
||||
|
|
||||
01:SCAN HDFS [tpch.lineitem]
|
||||
partitions=1/1 files=1 size=718.94MB
|
||||
HDFS partitions=1/1 files=1 size=718.94MB
|
||||
row-size=8B cardinality=6.00M
|
||||
====
|
||||
# order does not become the leftmost input because of the outer join;
|
||||
@@ -785,7 +785,7 @@ PLAN-ROOT SINK
|
||||
| row-size=39B cardinality=60.00K
|
||||
|
|
||||
|--02:SCAN HDFS [tpch.nation]
|
||||
| partitions=1/1 files=1 size=2.15KB
|
||||
| HDFS partitions=1/1 files=1 size=2.15KB
|
||||
| predicates: n_name = 'x'
|
||||
| row-size=21B cardinality=1
|
||||
|
|
||||
@@ -795,12 +795,12 @@ PLAN-ROOT SINK
|
||||
| row-size=18B cardinality=1.50M
|
||||
|
|
||||
|--00:SCAN HDFS [tpch.customer]
|
||||
| partitions=1/1 files=1 size=23.08MB
|
||||
| HDFS partitions=1/1 files=1 size=23.08MB
|
||||
| runtime filters: RF000 -> c_nationkey
|
||||
| row-size=10B cardinality=150.00K
|
||||
|
|
||||
01:SCAN HDFS [tpch.orders]
|
||||
partitions=1/1 files=1 size=162.56MB
|
||||
HDFS partitions=1/1 files=1 size=162.56MB
|
||||
runtime filters: RF002 -> o_custkey
|
||||
row-size=8B cardinality=1.50M
|
||||
---- DISTRIBUTEDPLAN
|
||||
@@ -824,7 +824,7 @@ PLAN-ROOT SINK
|
||||
|--08:EXCHANGE [BROADCAST]
|
||||
| |
|
||||
| 02:SCAN HDFS [tpch.nation]
|
||||
| partitions=1/1 files=1 size=2.15KB
|
||||
| HDFS partitions=1/1 files=1 size=2.15KB
|
||||
| predicates: n_name = 'x'
|
||||
| row-size=21B cardinality=1
|
||||
|
|
||||
@@ -836,14 +836,14 @@ PLAN-ROOT SINK
|
||||
|--07:EXCHANGE [HASH(c_custkey)]
|
||||
| |
|
||||
| 00:SCAN HDFS [tpch.customer]
|
||||
| partitions=1/1 files=1 size=23.08MB
|
||||
| HDFS partitions=1/1 files=1 size=23.08MB
|
||||
| runtime filters: RF000 -> c_nationkey
|
||||
| row-size=10B cardinality=150.00K
|
||||
|
|
||||
06:EXCHANGE [HASH(o_custkey)]
|
||||
|
|
||||
01:SCAN HDFS [tpch.orders]
|
||||
partitions=1/1 files=1 size=162.56MB
|
||||
HDFS partitions=1/1 files=1 size=162.56MB
|
||||
runtime filters: RF002 -> o_custkey
|
||||
row-size=8B cardinality=1.50M
|
||||
====
|
||||
@@ -865,7 +865,7 @@ PLAN-ROOT SINK
|
||||
| row-size=23B cardinality=9.00G
|
||||
|
|
||||
|--01:SCAN HDFS [tpch.orders]
|
||||
| partitions=1/1 files=1 size=162.56MB
|
||||
| HDFS partitions=1/1 files=1 size=162.56MB
|
||||
| row-size=0B cardinality=1.50M
|
||||
|
|
||||
03:HASH JOIN [INNER JOIN]
|
||||
@@ -874,12 +874,12 @@ PLAN-ROOT SINK
|
||||
| row-size=23B cardinality=6.00K
|
||||
|
|
||||
|--02:SCAN HDFS [tpch.nation]
|
||||
| partitions=1/1 files=1 size=2.15KB
|
||||
| HDFS partitions=1/1 files=1 size=2.15KB
|
||||
| predicates: n_name = 'x'
|
||||
| row-size=21B cardinality=1
|
||||
|
|
||||
00:SCAN HDFS [tpch.customer]
|
||||
partitions=1/1 files=1 size=23.08MB
|
||||
HDFS partitions=1/1 files=1 size=23.08MB
|
||||
runtime filters: RF000 -> c_nationkey
|
||||
row-size=2B cardinality=150.00K
|
||||
---- DISTRIBUTEDPLAN
|
||||
@@ -901,7 +901,7 @@ PLAN-ROOT SINK
|
||||
|--07:EXCHANGE [BROADCAST]
|
||||
| |
|
||||
| 01:SCAN HDFS [tpch.orders]
|
||||
| partitions=1/1 files=1 size=162.56MB
|
||||
| HDFS partitions=1/1 files=1 size=162.56MB
|
||||
| row-size=0B cardinality=1.50M
|
||||
|
|
||||
03:HASH JOIN [INNER JOIN, BROADCAST]
|
||||
@@ -912,12 +912,12 @@ PLAN-ROOT SINK
|
||||
|--06:EXCHANGE [BROADCAST]
|
||||
| |
|
||||
| 02:SCAN HDFS [tpch.nation]
|
||||
| partitions=1/1 files=1 size=2.15KB
|
||||
| HDFS partitions=1/1 files=1 size=2.15KB
|
||||
| predicates: n_name = 'x'
|
||||
| row-size=21B cardinality=1
|
||||
|
|
||||
00:SCAN HDFS [tpch.customer]
|
||||
partitions=1/1 files=1 size=23.08MB
|
||||
HDFS partitions=1/1 files=1 size=23.08MB
|
||||
runtime filters: RF000 -> c_nationkey
|
||||
row-size=2B cardinality=150.00K
|
||||
====
|
||||
@@ -938,15 +938,15 @@ PLAN-ROOT SINK
|
||||
| | row-size=0B cardinality=64
|
||||
| |
|
||||
| |--01:SCAN HDFS [functional.alltypestiny b]
|
||||
| | partitions=4/4 files=4 size=460B
|
||||
| | HDFS partitions=4/4 files=4 size=460B
|
||||
| | row-size=0B cardinality=8
|
||||
| |
|
||||
| 00:SCAN HDFS [functional.alltypestiny a]
|
||||
| partitions=4/4 files=4 size=460B
|
||||
| HDFS partitions=4/4 files=4 size=460B
|
||||
| row-size=0B cardinality=8
|
||||
|
|
||||
02:SCAN HDFS [functional.alltypes c]
|
||||
partitions=24/24 files=24 size=478.45KB
|
||||
HDFS partitions=24/24 files=24 size=478.45KB
|
||||
row-size=4B cardinality=7.30K
|
||||
====
|
||||
# Test that tables are not re-ordered across outer/semi joins (IMPALA-860),
|
||||
@@ -976,7 +976,7 @@ PLAN-ROOT SINK
|
||||
| | row-size=20B cardinality=9
|
||||
| |
|
||||
| |--05:SCAN HDFS [functional.alltypestiny t6]
|
||||
| | partitions=4/4 files=4 size=460B
|
||||
| | HDFS partitions=4/4 files=4 size=460B
|
||||
| | row-size=4B cardinality=8
|
||||
| |
|
||||
| 08:HASH JOIN [LEFT OUTER JOIN]
|
||||
@@ -994,27 +994,27 @@ PLAN-ROOT SINK
|
||||
| | | | row-size=8B cardinality=8
|
||||
| | | |
|
||||
| | | |--00:SCAN HDFS [functional.alltypestiny t1]
|
||||
| | | | partitions=4/4 files=4 size=460B
|
||||
| | | | HDFS partitions=4/4 files=4 size=460B
|
||||
| | | | runtime filters: RF002 -> t1.id
|
||||
| | | | row-size=4B cardinality=8
|
||||
| | | |
|
||||
| | | 01:SCAN HDFS [functional.alltypes t2]
|
||||
| | | partitions=24/24 files=24 size=478.45KB
|
||||
| | | HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| | | runtime filters: RF002 -> t2.id, RF006 -> t2.id
|
||||
| | | row-size=4B cardinality=7.30K
|
||||
| | |
|
||||
| | 02:SCAN HDFS [functional.alltypessmall t3]
|
||||
| | partitions=4/4 files=4 size=6.32KB
|
||||
| | HDFS partitions=4/4 files=4 size=6.32KB
|
||||
| | runtime filters: RF002 -> t3.id, RF004 -> t3.id
|
||||
| | row-size=4B cardinality=100
|
||||
| |
|
||||
| 03:SCAN HDFS [functional.alltypesagg t4]
|
||||
| partitions=11/11 files=11 size=814.73KB
|
||||
| HDFS partitions=11/11 files=11 size=814.73KB
|
||||
| runtime filters: RF002 -> t4.id
|
||||
| row-size=4B cardinality=11.00K
|
||||
|
|
||||
04:SCAN HDFS [functional.alltypes t5]
|
||||
partitions=24/24 files=24 size=478.45KB
|
||||
HDFS partitions=24/24 files=24 size=478.45KB
|
||||
runtime filters: RF000 -> t5.id
|
||||
row-size=4B cardinality=7.30K
|
||||
====
|
||||
@@ -1059,31 +1059,31 @@ PLAN-ROOT SINK
|
||||
| | | | | row-size=8B cardinality=8
|
||||
| | | | |
|
||||
| | | | |--00:SCAN HDFS [functional.alltypestiny t1]
|
||||
| | | | | partitions=4/4 files=4 size=460B
|
||||
| | | | | HDFS partitions=4/4 files=4 size=460B
|
||||
| | | | | row-size=4B cardinality=8
|
||||
| | | | |
|
||||
| | | | 01:SCAN HDFS [functional.alltypes t2]
|
||||
| | | | partitions=24/24 files=24 size=478.45KB
|
||||
| | | | HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| | | | runtime filters: RF006 -> t2.id
|
||||
| | | | row-size=4B cardinality=7.30K
|
||||
| | | |
|
||||
| | | 02:SCAN HDFS [functional.alltypessmall t3]
|
||||
| | | partitions=4/4 files=4 size=6.32KB
|
||||
| | | HDFS partitions=4/4 files=4 size=6.32KB
|
||||
| | | runtime filters: RF004 -> t3.id
|
||||
| | | row-size=4B cardinality=100
|
||||
| | |
|
||||
| | 03:SCAN HDFS [functional.alltypesagg t4]
|
||||
| | partitions=11/11 files=11 size=814.73KB
|
||||
| | HDFS partitions=11/11 files=11 size=814.73KB
|
||||
| | runtime filters: RF002 -> t4.id
|
||||
| | row-size=4B cardinality=11.00K
|
||||
| |
|
||||
| 04:SCAN HDFS [functional.alltypes t5]
|
||||
| partitions=24/24 files=24 size=478.45KB
|
||||
| HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| runtime filters: RF000 -> t5.id
|
||||
| row-size=4B cardinality=7.30K
|
||||
|
|
||||
05:SCAN HDFS [functional.alltypestiny t6]
|
||||
partitions=4/4 files=4 size=460B
|
||||
HDFS partitions=4/4 files=4 size=460B
|
||||
row-size=4B cardinality=8
|
||||
====
|
||||
# Check that a join in between outer/semi joins is re-ordered correctly.
|
||||
@@ -1128,31 +1128,31 @@ PLAN-ROOT SINK
|
||||
| | | | | row-size=8B cardinality=8
|
||||
| | | | |
|
||||
| | | | |--00:SCAN HDFS [functional.alltypestiny t1]
|
||||
| | | | | partitions=4/4 files=4 size=460B
|
||||
| | | | | HDFS partitions=4/4 files=4 size=460B
|
||||
| | | | | row-size=4B cardinality=8
|
||||
| | | | |
|
||||
| | | | 01:SCAN HDFS [functional.alltypes t2]
|
||||
| | | | partitions=24/24 files=24 size=478.45KB
|
||||
| | | | HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| | | | runtime filters: RF008 -> t2.id
|
||||
| | | | row-size=4B cardinality=7.30K
|
||||
| | | |
|
||||
| | | 03:SCAN HDFS [functional.alltypessmall t4]
|
||||
| | | partitions=4/4 files=4 size=6.32KB
|
||||
| | | HDFS partitions=4/4 files=4 size=6.32KB
|
||||
| | | runtime filters: RF006 -> t4.id
|
||||
| | | row-size=4B cardinality=100
|
||||
| | |
|
||||
| | 02:SCAN HDFS [functional.alltypesagg t3]
|
||||
| | partitions=11/11 files=11 size=814.73KB
|
||||
| | HDFS partitions=11/11 files=11 size=814.73KB
|
||||
| | runtime filters: RF004 -> t3.id
|
||||
| | row-size=4B cardinality=11.00K
|
||||
| |
|
||||
| 04:SCAN HDFS [functional.alltypes t5]
|
||||
| partitions=24/24 files=24 size=478.45KB
|
||||
| HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| runtime filters: RF002 -> t5.id
|
||||
| row-size=4B cardinality=7.30K
|
||||
|
|
||||
05:SCAN HDFS [functional.alltypestiny t6]
|
||||
partitions=4/4 files=4 size=460B
|
||||
HDFS partitions=4/4 files=4 size=460B
|
||||
runtime filters: RF000 -> t6.id
|
||||
row-size=4B cardinality=8
|
||||
====
|
||||
@@ -1196,30 +1196,30 @@ PLAN-ROOT SINK
|
||||
| | | | | row-size=8B cardinality=8
|
||||
| | | | |
|
||||
| | | | |--00:SCAN HDFS [functional.alltypestiny t1]
|
||||
| | | | | partitions=4/4 files=4 size=460B
|
||||
| | | | | HDFS partitions=4/4 files=4 size=460B
|
||||
| | | | | row-size=4B cardinality=8
|
||||
| | | | |
|
||||
| | | | 01:SCAN HDFS [functional.alltypes t2]
|
||||
| | | | partitions=24/24 files=24 size=478.45KB
|
||||
| | | | HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| | | | runtime filters: RF006 -> t2.id
|
||||
| | | | row-size=4B cardinality=7.30K
|
||||
| | | |
|
||||
| | | 03:SCAN HDFS [functional.alltypessmall t4]
|
||||
| | | partitions=4/4 files=4 size=6.32KB
|
||||
| | | HDFS partitions=4/4 files=4 size=6.32KB
|
||||
| | | runtime filters: RF004 -> t4.id
|
||||
| | | row-size=4B cardinality=100
|
||||
| | |
|
||||
| | 02:SCAN HDFS [functional.alltypesagg t3]
|
||||
| | partitions=11/11 files=11 size=814.73KB
|
||||
| | HDFS partitions=11/11 files=11 size=814.73KB
|
||||
| | runtime filters: RF002 -> t3.id
|
||||
| | row-size=4B cardinality=11.00K
|
||||
| |
|
||||
| 04:SCAN HDFS [functional.alltypes t5]
|
||||
| partitions=24/24 files=24 size=478.45KB
|
||||
| HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| row-size=4B cardinality=7.30K
|
||||
|
|
||||
05:SCAN HDFS [functional.alltypestiny t6]
|
||||
partitions=4/4 files=4 size=460B
|
||||
HDFS partitions=4/4 files=4 size=460B
|
||||
runtime filters: RF000 -> t6.id
|
||||
row-size=4B cardinality=8
|
||||
====
|
||||
@@ -1247,7 +1247,7 @@ PLAN-ROOT SINK
|
||||
| row-size=28B cardinality=1
|
||||
|
|
||||
|--09:SCAN HDFS [functional.alltypestiny t4]
|
||||
| partitions=4/4 files=4 size=460B
|
||||
| HDFS partitions=4/4 files=4 size=460B
|
||||
| row-size=4B cardinality=8
|
||||
|
|
||||
11:HASH JOIN [RIGHT OUTER JOIN]
|
||||
@@ -1274,11 +1274,11 @@ PLAN-ROOT SINK
|
||||
| | | row-size=8B cardinality=8
|
||||
| | |
|
||||
| | |--01:SCAN HDFS [functional.alltypestiny b]
|
||||
| | | partitions=4/4 files=4 size=460B
|
||||
| | | HDFS partitions=4/4 files=4 size=460B
|
||||
| | | row-size=4B cardinality=8
|
||||
| | |
|
||||
| | 00:SCAN HDFS [functional.alltypestiny a]
|
||||
| | partitions=4/4 files=4 size=460B
|
||||
| | HDFS partitions=4/4 files=4 size=460B
|
||||
| | runtime filters: RF006 -> a.id
|
||||
| | row-size=4B cardinality=8
|
||||
| |
|
||||
@@ -1294,11 +1294,11 @@ PLAN-ROOT SINK
|
||||
| | row-size=8B cardinality=0
|
||||
| |
|
||||
| 06:SCAN HDFS [functional.alltypestiny b]
|
||||
| partitions=4/4 files=4 size=460B
|
||||
| HDFS partitions=4/4 files=4 size=460B
|
||||
| row-size=4B cardinality=8
|
||||
|
|
||||
08:SCAN HDFS [functional.alltypes t3]
|
||||
partitions=24/24 files=24 size=478.45KB
|
||||
HDFS partitions=24/24 files=24 size=478.45KB
|
||||
runtime filters: RF000 -> t3.id, RF002 -> t3.id
|
||||
row-size=4B cardinality=7.30K
|
||||
====
|
||||
@@ -1326,7 +1326,7 @@ PLAN-ROOT SINK
|
||||
| row-size=28B cardinality=9
|
||||
|
|
||||
|--09:SCAN HDFS [functional.alltypestiny t4]
|
||||
| partitions=4/4 files=4 size=460B
|
||||
| HDFS partitions=4/4 files=4 size=460B
|
||||
| row-size=4B cardinality=8
|
||||
|
|
||||
11:HASH JOIN [FULL OUTER JOIN]
|
||||
@@ -1352,11 +1352,11 @@ PLAN-ROOT SINK
|
||||
| | | row-size=8B cardinality=8
|
||||
| | |
|
||||
| | |--01:SCAN HDFS [functional.alltypestiny b]
|
||||
| | | partitions=4/4 files=4 size=460B
|
||||
| | | HDFS partitions=4/4 files=4 size=460B
|
||||
| | | row-size=4B cardinality=8
|
||||
| | |
|
||||
| | 00:SCAN HDFS [functional.alltypestiny a]
|
||||
| | partitions=4/4 files=4 size=460B
|
||||
| | HDFS partitions=4/4 files=4 size=460B
|
||||
| | runtime filters: RF004 -> a.id
|
||||
| | row-size=4B cardinality=8
|
||||
| |
|
||||
@@ -1372,11 +1372,11 @@ PLAN-ROOT SINK
|
||||
| | row-size=8B cardinality=0
|
||||
| |
|
||||
| 06:SCAN HDFS [functional.alltypestiny b]
|
||||
| partitions=4/4 files=4 size=460B
|
||||
| HDFS partitions=4/4 files=4 size=460B
|
||||
| row-size=4B cardinality=8
|
||||
|
|
||||
08:SCAN HDFS [functional.alltypes t3]
|
||||
partitions=24/24 files=24 size=478.45KB
|
||||
HDFS partitions=24/24 files=24 size=478.45KB
|
||||
runtime filters: RF000 -> t3.id
|
||||
row-size=4B cardinality=7.30K
|
||||
====
|
||||
@@ -1404,7 +1404,7 @@ PLAN-ROOT SINK
|
||||
| row-size=16B cardinality=1
|
||||
|
|
||||
|--09:SCAN HDFS [functional.alltypestiny t4]
|
||||
| partitions=4/4 files=4 size=460B
|
||||
| HDFS partitions=4/4 files=4 size=460B
|
||||
| row-size=4B cardinality=8
|
||||
|
|
||||
11:HASH JOIN [RIGHT SEMI JOIN]
|
||||
@@ -1431,11 +1431,11 @@ PLAN-ROOT SINK
|
||||
| | | row-size=8B cardinality=8
|
||||
| | |
|
||||
| | |--01:SCAN HDFS [functional.alltypestiny b]
|
||||
| | | partitions=4/4 files=4 size=460B
|
||||
| | | HDFS partitions=4/4 files=4 size=460B
|
||||
| | | row-size=4B cardinality=8
|
||||
| | |
|
||||
| | 00:SCAN HDFS [functional.alltypestiny a]
|
||||
| | partitions=4/4 files=4 size=460B
|
||||
| | HDFS partitions=4/4 files=4 size=460B
|
||||
| | runtime filters: RF008 -> a.id
|
||||
| | row-size=4B cardinality=8
|
||||
| |
|
||||
@@ -1445,18 +1445,18 @@ PLAN-ROOT SINK
|
||||
| | row-size=4B cardinality=8
|
||||
| |
|
||||
| |--05:SCAN HDFS [functional.alltypestiny a]
|
||||
| | partitions=4/4 files=4 size=460B
|
||||
| | HDFS partitions=4/4 files=4 size=460B
|
||||
| | runtime filters: RF000 -> a.id, RF004 -> a.id
|
||||
| | row-size=4B cardinality=8
|
||||
| |
|
||||
| 06:SCAN HDFS [functional.alltypes b]
|
||||
| partition predicates: b.month = 1
|
||||
| partitions=2/24 files=2 size=40.32KB
|
||||
| HDFS partitions=2/24 files=2 size=40.32KB
|
||||
| runtime filters: RF000 -> b.id, RF004 -> b.id, RF006 -> b.id
|
||||
| row-size=4B cardinality=620
|
||||
|
|
||||
08:SCAN HDFS [functional.alltypes t3]
|
||||
partitions=24/24 files=24 size=478.45KB
|
||||
HDFS partitions=24/24 files=24 size=478.45KB
|
||||
runtime filters: RF000 -> t3.id, RF002 -> t3.id
|
||||
row-size=4B cardinality=7.30K
|
||||
====
|
||||
@@ -1484,7 +1484,7 @@ PLAN-ROOT SINK
|
||||
| row-size=16B cardinality=1
|
||||
|
|
||||
|--09:SCAN HDFS [functional.alltypestiny t4]
|
||||
| partitions=4/4 files=4 size=460B
|
||||
| HDFS partitions=4/4 files=4 size=460B
|
||||
| row-size=4B cardinality=8
|
||||
|
|
||||
11:HASH JOIN [RIGHT ANTI JOIN]
|
||||
@@ -1510,11 +1510,11 @@ PLAN-ROOT SINK
|
||||
| | | row-size=8B cardinality=8
|
||||
| | |
|
||||
| | |--01:SCAN HDFS [functional.alltypestiny b]
|
||||
| | | partitions=4/4 files=4 size=460B
|
||||
| | | HDFS partitions=4/4 files=4 size=460B
|
||||
| | | row-size=4B cardinality=8
|
||||
| | |
|
||||
| | 00:SCAN HDFS [functional.alltypestiny a]
|
||||
| | partitions=4/4 files=4 size=460B
|
||||
| | HDFS partitions=4/4 files=4 size=460B
|
||||
| | runtime filters: RF004 -> a.id
|
||||
| | row-size=4B cardinality=8
|
||||
| |
|
||||
@@ -1523,18 +1523,18 @@ PLAN-ROOT SINK
|
||||
| | row-size=4B cardinality=620
|
||||
| |
|
||||
| |--05:SCAN HDFS [functional.alltypestiny a]
|
||||
| | partitions=4/4 files=4 size=460B
|
||||
| | HDFS partitions=4/4 files=4 size=460B
|
||||
| | runtime filters: RF000 -> a.id, RF002 -> a.id
|
||||
| | row-size=4B cardinality=8
|
||||
| |
|
||||
| 06:SCAN HDFS [functional.alltypes b]
|
||||
| partition predicates: b.month = 1
|
||||
| partitions=2/24 files=2 size=40.32KB
|
||||
| HDFS partitions=2/24 files=2 size=40.32KB
|
||||
| runtime filters: RF000 -> b.id, RF002 -> b.id
|
||||
| row-size=4B cardinality=620
|
||||
|
|
||||
08:SCAN HDFS [functional.alltypes t3]
|
||||
partitions=24/24 files=24 size=478.45KB
|
||||
HDFS partitions=24/24 files=24 size=478.45KB
|
||||
runtime filters: RF000 -> t3.id
|
||||
row-size=4B cardinality=7.30K
|
||||
====
|
||||
@@ -1557,17 +1557,21 @@ WHERE `$a$2`.`$c$1` > t4.id
|
||||
---- PLAN
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
10:AGGREGATE [FINALIZE]
|
||||
11:AGGREGATE [FINALIZE]
|
||||
| output: sum(t4.tinyint_col)
|
||||
| row-size=8B cardinality=1
|
||||
|
|
||||
09:HASH JOIN [LEFT SEMI JOIN]
|
||||
10:HASH JOIN [LEFT SEMI JOIN]
|
||||
| hash predicates: t4.bigint_col = tt1.int_col
|
||||
| runtime filters: RF000 <- tt1.int_col
|
||||
| row-size=31B cardinality=8
|
||||
|
|
||||
|--06:SCAN HDFS [functional.alltypestiny tt1]
|
||||
| partitions=4/4 files=4 size=460B
|
||||
|--09:AGGREGATE [FINALIZE]
|
||||
| | group by: tt1.int_col
|
||||
| | row-size=4B cardinality=2
|
||||
| |
|
||||
| 06:SCAN HDFS [functional.alltypestiny tt1]
|
||||
| HDFS partitions=4/4 files=4 size=460B
|
||||
| row-size=4B cardinality=8
|
||||
|
|
||||
08:NESTED LOOP JOIN [INNER JOIN]
|
||||
@@ -1580,7 +1584,7 @@ PLAN-ROOT SINK
|
||||
| | row-size=8B cardinality=1
|
||||
| |
|
||||
| 04:SCAN HDFS [functional.alltypesagg t1]
|
||||
| partitions=11/11 files=11 size=814.73KB
|
||||
| HDFS partitions=11/11 files=11 size=814.73KB
|
||||
| row-size=4B cardinality=11.00K
|
||||
|
|
||||
07:NESTED LOOP JOIN [CROSS JOIN]
|
||||
@@ -1593,16 +1597,16 @@ PLAN-ROOT SINK
|
||||
| | row-size=10B cardinality=1
|
||||
| |
|
||||
| |--02:SCAN HDFS [functional.alltypestiny t2]
|
||||
| | partitions=4/4 files=4 size=460B
|
||||
| | HDFS partitions=4/4 files=4 size=460B
|
||||
| | row-size=2B cardinality=8
|
||||
| |
|
||||
| 01:SCAN HDFS [functional.alltypes t1]
|
||||
| partitions=24/24 files=24 size=478.45KB
|
||||
| HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| runtime filters: RF002 -> t1.bigint_col
|
||||
| row-size=8B cardinality=7.30K
|
||||
|
|
||||
00:SCAN HDFS [functional.alltypestiny t4]
|
||||
partitions=4/4 files=4 size=460B
|
||||
HDFS partitions=4/4 files=4 size=460B
|
||||
runtime filters: RF000 -> t4.bigint_col
|
||||
row-size=13B cardinality=8
|
||||
====
|
||||
@@ -1623,11 +1627,11 @@ PLAN-ROOT SINK
|
||||
| row-size=25B cardinality=8
|
||||
|
|
||||
|--00:SCAN HDFS [functional.alltypestiny a]
|
||||
| partitions=4/4 files=4 size=460B
|
||||
| HDFS partitions=4/4 files=4 size=460B
|
||||
| row-size=9B cardinality=8
|
||||
|
|
||||
01:SCAN HDFS [functional.alltypessmall b]
|
||||
partitions=4/4 files=4 size=6.32KB
|
||||
HDFS partitions=4/4 files=4 size=6.32KB
|
||||
runtime filters: RF000 -> b.id, RF001 -> b.int_col
|
||||
row-size=16B cardinality=100
|
||||
====
|
||||
@@ -1656,16 +1660,16 @@ PLAN-ROOT SINK
|
||||
| | row-size=27B cardinality=8
|
||||
| |
|
||||
| |--00:SCAN HDFS [functional.alltypestiny a]
|
||||
| | partitions=4/4 files=4 size=460B
|
||||
| | HDFS partitions=4/4 files=4 size=460B
|
||||
| | row-size=9B cardinality=8
|
||||
| |
|
||||
| 01:SCAN HDFS [functional.alltypessmall b]
|
||||
| partitions=4/4 files=4 size=6.32KB
|
||||
| HDFS partitions=4/4 files=4 size=6.32KB
|
||||
| runtime filters: RF002 -> b.id
|
||||
| row-size=18B cardinality=100
|
||||
|
|
||||
02:SCAN HDFS [functional.alltypes c]
|
||||
partitions=24/24 files=24 size=478.45KB
|
||||
HDFS partitions=24/24 files=24 size=478.45KB
|
||||
runtime filters: RF000 -> c.tinyint_col
|
||||
row-size=6B cardinality=7.30K
|
||||
====
|
||||
@@ -1693,16 +1697,16 @@ PLAN-ROOT SINK
|
||||
| | row-size=10B cardinality=9
|
||||
| |
|
||||
| |--00:SCAN HDFS [functional.alltypestiny a]
|
||||
| | partitions=4/4 files=4 size=460B
|
||||
| | HDFS partitions=4/4 files=4 size=460B
|
||||
| | row-size=4B cardinality=8
|
||||
| |
|
||||
| 01:SCAN HDFS [functional.alltypessmall b]
|
||||
| partitions=4/4 files=4 size=6.32KB
|
||||
| HDFS partitions=4/4 files=4 size=6.32KB
|
||||
| runtime filters: RF002 -> b.id
|
||||
| row-size=6B cardinality=100
|
||||
|
|
||||
02:SCAN HDFS [functional.alltypes c]
|
||||
partitions=24/24 files=24 size=478.45KB
|
||||
HDFS partitions=24/24 files=24 size=478.45KB
|
||||
runtime filters: RF000 -> c.tinyint_col
|
||||
row-size=6B cardinality=7.30K
|
||||
====
|
||||
@@ -1727,7 +1731,7 @@ PLAN-ROOT SINK
|
||||
| row-size=81B cardinality=83.39K
|
||||
|
|
||||
|--03:SCAN HDFS [functional.alltypestiny t3]
|
||||
| partitions=4/4 files=4 size=460B
|
||||
| HDFS partitions=4/4 files=4 size=460B
|
||||
| row-size=33B cardinality=8
|
||||
|
|
||||
02:HASH JOIN [RIGHT OUTER JOIN]
|
||||
@@ -1737,11 +1741,11 @@ PLAN-ROOT SINK
|
||||
| row-size=48B cardinality=83.39K
|
||||
|
|
||||
|--00:SCAN HDFS [functional.alltypesagg t1]
|
||||
| partitions=11/11 files=11 size=814.73KB
|
||||
| HDFS partitions=11/11 files=11 size=814.73KB
|
||||
| row-size=15B cardinality=11.00K
|
||||
|
|
||||
01:SCAN HDFS [functional.alltypes t2]
|
||||
partitions=24/24 files=24 size=478.45KB
|
||||
HDFS partitions=24/24 files=24 size=478.45KB
|
||||
runtime filters: RF000 -> t2.date_string_col, RF001 -> t2.date_string_col
|
||||
row-size=33B cardinality=7.30K
|
||||
====
|
||||
@@ -1766,12 +1770,12 @@ PLAN-ROOT SINK
|
||||
| row-size=89B cardinality=1
|
||||
|
|
||||
|--01:SCAN HDFS [functional.alltypes]
|
||||
| partitions=24/24 files=24 size=478.45KB
|
||||
| HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| predicates: timestamp_col = TIMESTAMP '2016-11-20 00:00:00'
|
||||
| row-size=20B cardinality=1
|
||||
|
|
||||
00:SCAN HDFS [functional.alltypes a]
|
||||
partitions=24/24 files=24 size=478.45KB
|
||||
HDFS partitions=24/24 files=24 size=478.45KB
|
||||
predicates: a.date_string_col = ''
|
||||
runtime filters: RF000 -> a.id
|
||||
row-size=89B cardinality=10
|
||||
@@ -1791,12 +1795,12 @@ PLAN-ROOT SINK
|
||||
| row-size=89B cardinality=1
|
||||
|
|
||||
|--00:SCAN HDFS [functional.alltypes a]
|
||||
| partitions=24/24 files=24 size=478.45KB
|
||||
| HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| predicates: a.timestamp_col = TIMESTAMP '2016-11-20 00:00:00'
|
||||
| row-size=89B cardinality=1
|
||||
|
|
||||
01:SCAN HDFS [functional.alltypes]
|
||||
partitions=24/24 files=24 size=478.45KB
|
||||
HDFS partitions=24/24 files=24 size=478.45KB
|
||||
predicates: date_string_col = ''
|
||||
runtime filters: RF000 -> functional.alltypes.id
|
||||
row-size=24B cardinality=10
|
||||
@@ -1817,12 +1821,12 @@ PLAN-ROOT SINK
|
||||
| row-size=89B cardinality=1
|
||||
|
|
||||
|--01:SCAN HDFS [functional.alltypes]
|
||||
| partitions=24/24 files=24 size=478.45KB
|
||||
| HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| predicates: timestamp_col IS NOT DISTINCT FROM TIMESTAMP '2016-11-20 00:00:00'
|
||||
| row-size=20B cardinality=1
|
||||
|
|
||||
00:SCAN HDFS [functional.alltypes a]
|
||||
partitions=24/24 files=24 size=478.45KB
|
||||
HDFS partitions=24/24 files=24 size=478.45KB
|
||||
predicates: a.date_string_col IS NOT DISTINCT FROM ''
|
||||
runtime filters: RF000 -> a.id
|
||||
row-size=89B cardinality=10
|
||||
@@ -1842,12 +1846,12 @@ PLAN-ROOT SINK
|
||||
| row-size=89B cardinality=1
|
||||
|
|
||||
|--00:SCAN HDFS [functional.alltypes a]
|
||||
| partitions=24/24 files=24 size=478.45KB
|
||||
| HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| predicates: a.timestamp_col IS NOT DISTINCT FROM TIMESTAMP '2016-11-20 00:00:00'
|
||||
| row-size=89B cardinality=1
|
||||
|
|
||||
01:SCAN HDFS [functional.alltypes]
|
||||
partitions=24/24 files=24 size=478.45KB
|
||||
HDFS partitions=24/24 files=24 size=478.45KB
|
||||
predicates: date_string_col IS NOT DISTINCT FROM ''
|
||||
runtime filters: RF000 -> functional.alltypes.id
|
||||
row-size=24B cardinality=10
|
||||
|
||||
@@ -2181,24 +2181,34 @@ on (a.string_col = b.string_col and a.int_col = b.int_col)
|
||||
---- DISTRIBUTEDPLAN
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
07:EXCHANGE [UNPARTITIONED]
|
||||
10:EXCHANGE [UNPARTITIONED]
|
||||
|
|
||||
03:HASH JOIN [LEFT SEMI JOIN, PARTITIONED]
|
||||
04:HASH JOIN [LEFT SEMI JOIN, PARTITIONED]
|
||||
| hash predicates: int_col = b.int_col, string_col = b.string_col
|
||||
| runtime filters: RF000 <- b.int_col, RF001 <- b.string_col
|
||||
| row-size=17B cardinality=4
|
||||
|
|
||||
|--06:EXCHANGE [HASH(b.int_col,b.string_col)]
|
||||
|--09:EXCHANGE [HASH(b.int_col,b.string_col)]
|
||||
| |
|
||||
| 08:AGGREGATE [FINALIZE]
|
||||
| | group by: b.string_col, b.int_col
|
||||
| | row-size=17B cardinality=100
|
||||
| |
|
||||
| 07:EXCHANGE [HASH(b.string_col,b.int_col)]
|
||||
| |
|
||||
| 03:AGGREGATE [STREAMING]
|
||||
| | group by: b.string_col, b.int_col
|
||||
| | row-size=17B cardinality=100
|
||||
| |
|
||||
| 02:SCAN HDFS [functional.alltypes b]
|
||||
| HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| row-size=17B cardinality=7.30K
|
||||
|
|
||||
05:AGGREGATE [FINALIZE]
|
||||
06:AGGREGATE [FINALIZE]
|
||||
| group by: int_col, string_col
|
||||
| row-size=17B cardinality=4
|
||||
|
|
||||
04:EXCHANGE [HASH(int_col,string_col)]
|
||||
05:EXCHANGE [HASH(int_col,string_col)]
|
||||
|
|
||||
01:AGGREGATE [STREAMING]
|
||||
| group by: int_col, string_col
|
||||
@@ -2566,42 +2576,50 @@ where a.id < 10
|
||||
---- PLAN
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
08:NESTED LOOP JOIN [RIGHT ANTI JOIN]
|
||||
10:NESTED LOOP JOIN [LEFT ANTI JOIN]
|
||||
| join predicates: c.string_col != e.string_col
|
||||
| row-size=267B cardinality=100
|
||||
|
|
||||
|--07:NESTED LOOP JOIN [RIGHT SEMI JOIN]
|
||||
| | join predicates: b.bigint_col > d.bigint_col
|
||||
| | row-size=267B cardinality=100
|
||||
|--09:AGGREGATE [FINALIZE]
|
||||
| | group by: e.string_col
|
||||
| | row-size=15B cardinality=963
|
||||
| |
|
||||
| |--06:NESTED LOOP JOIN [RIGHT OUTER JOIN]
|
||||
| | | join predicates: a.int_col != c.int_col OR a.tinyint_col > c.tinyint_col
|
||||
| | | row-size=267B cardinality=100
|
||||
| | |
|
||||
| | |--05:NESTED LOOP JOIN [INNER JOIN]
|
||||
| | | | predicates: a.id < b.id
|
||||
| | | | row-size=178B cardinality=100
|
||||
| | | |
|
||||
| | | |--00:SCAN HDFS [functional.alltypestiny a]
|
||||
| | | | HDFS partitions=4/4 files=4 size=460B
|
||||
| | | | predicates: a.id < 10
|
||||
| | | | row-size=89B cardinality=1
|
||||
| | | |
|
||||
| | | 01:SCAN HDFS [functional.alltypessmall b]
|
||||
| | | HDFS partitions=4/4 files=4 size=6.32KB
|
||||
| | | row-size=89B cardinality=100
|
||||
| | |
|
||||
| | 02:SCAN HDFS [functional.alltypes c]
|
||||
| | HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| | row-size=89B cardinality=7.30K
|
||||
| 04:SCAN HDFS [functional.alltypesagg e]
|
||||
| HDFS partitions=11/11 files=11 size=814.73KB
|
||||
| row-size=15B cardinality=11.00K
|
||||
|
|
||||
08:NESTED LOOP JOIN [LEFT SEMI JOIN]
|
||||
| join predicates: b.bigint_col > d.bigint_col
|
||||
| row-size=267B cardinality=100
|
||||
|
|
||||
|--07:AGGREGATE [FINALIZE]
|
||||
| | group by: d.bigint_col
|
||||
| | row-size=8B cardinality=1.01K
|
||||
| |
|
||||
| 03:SCAN HDFS [functional.alltypesagg d]
|
||||
| HDFS partitions=11/11 files=11 size=814.73KB
|
||||
| row-size=8B cardinality=11.00K
|
||||
|
|
||||
04:SCAN HDFS [functional.alltypesagg e]
|
||||
HDFS partitions=11/11 files=11 size=814.73KB
|
||||
row-size=15B cardinality=11.00K
|
||||
06:NESTED LOOP JOIN [RIGHT OUTER JOIN]
|
||||
| join predicates: a.int_col != c.int_col OR a.tinyint_col > c.tinyint_col
|
||||
| row-size=267B cardinality=100
|
||||
|
|
||||
|--05:NESTED LOOP JOIN [INNER JOIN]
|
||||
| | predicates: a.id < b.id
|
||||
| | row-size=178B cardinality=100
|
||||
| |
|
||||
| |--00:SCAN HDFS [functional.alltypestiny a]
|
||||
| | HDFS partitions=4/4 files=4 size=460B
|
||||
| | predicates: a.id < 10
|
||||
| | row-size=89B cardinality=1
|
||||
| |
|
||||
| 01:SCAN HDFS [functional.alltypessmall b]
|
||||
| HDFS partitions=4/4 files=4 size=6.32KB
|
||||
| row-size=89B cardinality=100
|
||||
|
|
||||
02:SCAN HDFS [functional.alltypes c]
|
||||
HDFS partitions=24/24 files=24 size=478.45KB
|
||||
row-size=89B cardinality=7.30K
|
||||
====
|
||||
# Regression test for IMPALA-2495: Crash: impala::InPredicate::SetLookupPrepare
|
||||
select count(id) from functional.alltypestiny t1
|
||||
@@ -3067,7 +3085,7 @@ PLAN-ROOT SINK
|
||||
| row-size=96B cardinality=0
|
||||
|
|
||||
|--01:SCAN HDFS [functional_text_gzip.alltypes b]
|
||||
| HDFS partitions=24/24 files=24 size=123.32KB
|
||||
| HDFS partitions=24/24 files=24 size=77.88KB
|
||||
| row-size=80B cardinality=unavailable
|
||||
|
|
||||
00:SCAN HDFS [functional_text_gzip.emptytable a]
|
||||
|
||||
@@ -28,7 +28,7 @@ PLAN-ROOT SINK
|
||||
|
|
||||
01:AGGREGATE [FINALIZE]
|
||||
| output: count(f21)
|
||||
| row-size=8B cardinality=0
|
||||
| row-size=8B cardinality=1
|
||||
|
|
||||
00:SCAN HDFS [functional.allcomplextypes.complex_nested_struct_col.f2.f12]
|
||||
partitions=0/0 files=0 size=0B
|
||||
@@ -46,7 +46,7 @@ PLAN-ROOT SINK
|
||||
|
|
||||
03:AGGREGATE [FINALIZE]
|
||||
| output: count(*)
|
||||
| row-size=8B cardinality=0
|
||||
| row-size=8B cardinality=1
|
||||
|
|
||||
02:HASH JOIN [INNER JOIN]
|
||||
| hash predicates: a.f1 = b.f1
|
||||
@@ -94,7 +94,7 @@ PLAN-ROOT SINK
|
||||
| | row-size=0B cardinality=10
|
||||
| |
|
||||
| 00:SCAN HDFS [tpch_nested_parquet.region r]
|
||||
| partitions=1/1 files=1 size=3.44KB
|
||||
| HDFS partitions=1/1 files=1 size=3.59KB
|
||||
| predicates: !empty(r.r_nations)
|
||||
| row-size=12B cardinality=5
|
||||
|
|
||||
@@ -104,12 +104,12 @@ PLAN-ROOT SINK
|
||||
| row-size=163B cardinality=10.16K
|
||||
|
|
||||
|--06:SCAN HDFS [tpch_nested_parquet.supplier s]
|
||||
| partitions=1/1 files=1 size=41.79MB
|
||||
| HDFS partitions=1/1 files=1 size=41.80MB
|
||||
| runtime filters: RF000 -> s.s_nationkey, RF001 -> s_comment
|
||||
| row-size=77B cardinality=10.00K
|
||||
|
|
||||
05:SCAN HDFS [tpch_nested_parquet.customer c]
|
||||
partitions=1/1 files=4 size=288.99MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
runtime filters: RF000 -> c_nationkey, RF001 -> c.c_comment, RF004 -> c.c_nationkey, RF005 -> c_comment
|
||||
row-size=87B cardinality=150.00K
|
||||
====
|
||||
@@ -153,6 +153,7 @@ PLAN-ROOT SINK
|
||||
| | row-size=16B cardinality=1
|
||||
| |
|
||||
| 03:UNNEST [a.int_array_col b]
|
||||
| limit: 1
|
||||
| row-size=0B cardinality=10
|
||||
|
|
||||
00:SCAN HDFS [functional.allcomplextypes a]
|
||||
@@ -200,6 +201,7 @@ PLAN-ROOT SINK
|
||||
| | row-size=16B cardinality=1
|
||||
| |
|
||||
| 03:UNNEST [a.int_array_col b]
|
||||
| limit: 1
|
||||
| row-size=0B cardinality=10
|
||||
|
|
||||
00:SCAN HDFS [functional.allcomplextypes a]
|
||||
@@ -224,6 +226,7 @@ PLAN-ROOT SINK
|
||||
| | row-size=16B cardinality=1
|
||||
| |
|
||||
| 03:UNNEST [a.int_array_col]
|
||||
| limit: 1
|
||||
| row-size=0B cardinality=10
|
||||
|
|
||||
00:SCAN HDFS [functional.allcomplextypes a]
|
||||
@@ -836,24 +839,24 @@ where e.f1 < 10
|
||||
---- PLAN
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
11:HASH JOIN [INNER JOIN]
|
||||
12:HASH JOIN [INNER JOIN]
|
||||
| hash predicates: d.id = b.id
|
||||
| runtime filters: RF000 <- b.id
|
||||
| row-size=36B cardinality=0
|
||||
|
|
||||
|--10:SUBPLAN
|
||||
|--11:SUBPLAN
|
||||
| | row-size=32B cardinality=0
|
||||
| |
|
||||
| |--08:NESTED LOOP JOIN [CROSS JOIN]
|
||||
| |--09:NESTED LOOP JOIN [CROSS JOIN]
|
||||
| | | row-size=32B cardinality=10
|
||||
| | |
|
||||
| | |--06:SINGULAR ROW SRC
|
||||
| | |--07:SINGULAR ROW SRC
|
||||
| | | row-size=24B cardinality=1
|
||||
| | |
|
||||
| | 07:UNNEST [a.struct_array_col e]
|
||||
| | 08:UNNEST [a.struct_array_col e]
|
||||
| | row-size=0B cardinality=10
|
||||
| |
|
||||
| 09:HASH JOIN [RIGHT ANTI JOIN]
|
||||
| 10:HASH JOIN [RIGHT ANTI JOIN]
|
||||
| | hash predicates: c.int_col = b.int_col
|
||||
| | row-size=24B cardinality=0
|
||||
| |
|
||||
@@ -869,16 +872,20 @@ PLAN-ROOT SINK
|
||||
| | | row-size=16B cardinality=0
|
||||
| | |
|
||||
| | 01:SCAN HDFS [functional.alltypestiny b]
|
||||
| | partitions=4/4 files=4 size=460B
|
||||
| | HDFS partitions=4/4 files=4 size=460B
|
||||
| | runtime filters: RF002 -> b.id
|
||||
| | row-size=8B cardinality=8
|
||||
| |
|
||||
| 06:AGGREGATE [FINALIZE]
|
||||
| | group by: c.int_col
|
||||
| | row-size=4B cardinality=10
|
||||
| |
|
||||
| 02:SCAN HDFS [functional.alltypessmall c]
|
||||
| partitions=4/4 files=4 size=6.32KB
|
||||
| HDFS partitions=4/4 files=4 size=6.32KB
|
||||
| row-size=4B cardinality=100
|
||||
|
|
||||
03:SCAN HDFS [functional.alltypes d]
|
||||
partitions=24/24 files=24 size=478.45KB
|
||||
HDFS partitions=24/24 files=24 size=478.45KB
|
||||
runtime filters: RF000 -> d.id
|
||||
row-size=4B cardinality=7.30K
|
||||
====
|
||||
@@ -902,7 +909,7 @@ PLAN-ROOT SINK
|
||||
| | row-size=4B cardinality=100
|
||||
| |
|
||||
| |--02:SCAN HDFS [functional.alltypessmall c]
|
||||
| | partitions=4/4 files=4 size=6.32KB
|
||||
| | HDFS partitions=4/4 files=4 size=6.32KB
|
||||
| | row-size=4B cardinality=100
|
||||
| |
|
||||
| 09:SUBPLAN
|
||||
@@ -927,11 +934,11 @@ PLAN-ROOT SINK
|
||||
| | row-size=16B cardinality=0
|
||||
| |
|
||||
| 01:SCAN HDFS [functional.alltypestiny b]
|
||||
| partitions=4/4 files=4 size=460B
|
||||
| HDFS partitions=4/4 files=4 size=460B
|
||||
| row-size=8B cardinality=8
|
||||
|
|
||||
03:SCAN HDFS [functional.alltypes d]
|
||||
partitions=24/24 files=24 size=478.45KB
|
||||
HDFS partitions=24/24 files=24 size=478.45KB
|
||||
row-size=4B cardinality=7.30K
|
||||
====
|
||||
# Test subplans: Test joining relative refs with independent table refs.
|
||||
@@ -989,13 +996,13 @@ PLAN-ROOT SINK
|
||||
| | row-size=36B cardinality=0
|
||||
| |
|
||||
| 05:SCAN HDFS [functional.alltypessmall c]
|
||||
| partitions=4/4 files=4 size=6.32KB
|
||||
| HDFS partitions=4/4 files=4 size=6.32KB
|
||||
| predicates: c.id < 10, c.int_col > 30
|
||||
| runtime filters: RF002 -> c.id
|
||||
| row-size=89B cardinality=10
|
||||
|
|
||||
06:SCAN HDFS [functional.alltypes e]
|
||||
partitions=24/24 files=24 size=478.45KB
|
||||
HDFS partitions=24/24 files=24 size=478.45KB
|
||||
runtime filters: RF000 -> e.id
|
||||
row-size=4B cardinality=7.30K
|
||||
====
|
||||
@@ -1015,7 +1022,7 @@ PLAN-ROOT SINK
|
||||
| row-size=89B cardinality=8
|
||||
|
|
||||
|--06:SCAN HDFS [functional.alltypestiny e]
|
||||
| partitions=4/4 files=4 size=460B
|
||||
| HDFS partitions=4/4 files=4 size=460B
|
||||
| row-size=89B cardinality=8
|
||||
|
|
||||
11:SUBPLAN
|
||||
@@ -1053,7 +1060,7 @@ PLAN-ROOT SINK
|
||||
| row-size=36B cardinality=0
|
||||
|
|
||||
05:SCAN HDFS [functional.alltypessmall c]
|
||||
partitions=4/4 files=4 size=6.32KB
|
||||
HDFS partitions=4/4 files=4 size=6.32KB
|
||||
predicates: c.int_col > 30
|
||||
row-size=89B cardinality=10
|
||||
====
|
||||
@@ -1069,7 +1076,7 @@ PLAN-ROOT SINK
|
||||
| row-size=24B cardinality=0
|
||||
|
|
||||
|--05:NESTED LOOP JOIN [CROSS JOIN]
|
||||
| | row-size=24B cardinality=0
|
||||
| | row-size=24B cardinality=1
|
||||
| |
|
||||
| |--02:SINGULAR ROW SRC
|
||||
| | row-size=16B cardinality=1
|
||||
@@ -1077,7 +1084,7 @@ PLAN-ROOT SINK
|
||||
| 04:AGGREGATE [FINALIZE]
|
||||
| | output: count(*)
|
||||
| | having: count(*) < 10
|
||||
| | row-size=8B cardinality=0
|
||||
| | row-size=8B cardinality=1
|
||||
| |
|
||||
| 03:UNNEST [a.int_array_col]
|
||||
| row-size=0B cardinality=10
|
||||
@@ -1098,7 +1105,7 @@ PLAN-ROOT SINK
|
||||
| row-size=32B cardinality=0
|
||||
|
|
||||
|--05:NESTED LOOP JOIN [CROSS JOIN]
|
||||
| | row-size=32B cardinality=10
|
||||
| | row-size=32B cardinality=1
|
||||
| |
|
||||
| |--02:SINGULAR ROW SRC
|
||||
| | row-size=16B cardinality=1
|
||||
@@ -1107,7 +1114,7 @@ PLAN-ROOT SINK
|
||||
| | output: count(*)
|
||||
| | group by: f1
|
||||
| | having: count(*) < 10
|
||||
| | row-size=16B cardinality=10
|
||||
| | row-size=16B cardinality=1
|
||||
| |
|
||||
| 03:UNNEST [a.struct_array_col]
|
||||
| row-size=0B cardinality=10
|
||||
@@ -1229,6 +1236,7 @@ PLAN-ROOT SINK
|
||||
| | row-size=16B cardinality=1
|
||||
| |
|
||||
| 03:UNNEST [a.int_array_col]
|
||||
| limit: 1
|
||||
| row-size=4B cardinality=10
|
||||
|
|
||||
00:SCAN HDFS [functional.allcomplextypes a]
|
||||
@@ -1252,6 +1260,7 @@ PLAN-ROOT SINK
|
||||
| | row-size=16B cardinality=1
|
||||
| |
|
||||
| 03:UNNEST [a.int_array_col]
|
||||
| limit: 1
|
||||
| row-size=0B cardinality=10
|
||||
|
|
||||
00:SCAN HDFS [functional.allcomplextypes a]
|
||||
@@ -1648,7 +1657,7 @@ PLAN-ROOT SINK
|
||||
| row-size=0B cardinality=10
|
||||
|
|
||||
00:SCAN HDFS [tpch_nested_parquet.customer c]
|
||||
partitions=1/1 files=4 size=288.99MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
predicates: c_custkey < 10, !empty(c.c_orders)
|
||||
predicates on o: !empty(o.o_lineitems), o_orderkey < 5
|
||||
predicates on o_lineitems: l_linenumber < 3
|
||||
@@ -1780,7 +1789,7 @@ PLAN-ROOT SINK
|
||||
| row-size=0B cardinality=10
|
||||
|
|
||||
00:SCAN HDFS [tpch_nested_parquet.customer c]
|
||||
partitions=1/1 files=4 size=288.99MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
predicates: c.c_custkey = c.c_nationkey, !empty(c.c_orders)
|
||||
predicates on o: !empty(o.o_lineitems), o.o_orderkey = o.o_shippriority
|
||||
predicates on l: l.l_partkey = l.l_linenumber, l.l_partkey = l.l_suppkey
|
||||
@@ -1895,7 +1904,7 @@ PLAN-ROOT SINK
|
||||
| row-size=44B cardinality=1
|
||||
|
|
||||
00:SCAN HDFS [tpch_nested_parquet.supplier s]
|
||||
partitions=1/1 files=1 size=41.79MB
|
||||
HDFS partitions=1/1 files=1 size=41.80MB
|
||||
row-size=44B cardinality=10.00K
|
||||
====
|
||||
# IMPALA-2383: Test join ordering of relative collection ref after an outer join.
|
||||
@@ -2068,7 +2077,7 @@ PLAN-ROOT SINK
|
||||
| row-size=0B cardinality=10
|
||||
|
|
||||
00:SCAN HDFS [tpch_nested_parquet.customer c]
|
||||
partitions=1/1 files=4 size=288.99MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
row-size=44B cardinality=150.00K
|
||||
====
|
||||
# IMPALA-2412: Test join ordering in nested subplans. Same as above
|
||||
@@ -2127,7 +2136,7 @@ PLAN-ROOT SINK
|
||||
| row-size=0B cardinality=10
|
||||
|
|
||||
00:SCAN HDFS [tpch_nested_parquet.customer c]
|
||||
partitions=1/1 files=4 size=288.99MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
predicates: !empty(c.c_orders)
|
||||
row-size=44B cardinality=150.00K
|
||||
====
|
||||
@@ -2184,7 +2193,7 @@ PLAN-ROOT SINK
|
||||
| row-size=0B cardinality=10
|
||||
|
|
||||
00:SCAN HDFS [tpch_nested_parquet.customer c]
|
||||
partitions=1/1 files=4 size=288.99MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
row-size=44B cardinality=150.00K
|
||||
====
|
||||
# IMPALA-2446: Test predicate assignment when outer join has no conjuncts in
|
||||
@@ -2331,7 +2340,7 @@ PLAN-ROOT SINK
|
||||
| row-size=0B cardinality=10
|
||||
|
|
||||
00:SCAN HDFS [tpch_nested_parquet.customer c]
|
||||
partitions=1/1 files=4 size=288.99MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
row-size=12B cardinality=150.00K
|
||||
====
|
||||
# IMPALA-3065/IMPALA-3062: Test correct assignment of !empty() predicates.
|
||||
@@ -2349,7 +2358,7 @@ PLAN-ROOT SINK
|
||||
| row-size=28B cardinality=1.50M
|
||||
|
|
||||
|--05:SCAN HDFS [tpch_nested_parquet.customer c2]
|
||||
| partitions=1/1 files=4 size=288.99MB
|
||||
| HDFS partitions=1/1 files=4 size=289.08MB
|
||||
| row-size=8B cardinality=150.00K
|
||||
|
|
||||
01:SUBPLAN
|
||||
@@ -2365,7 +2374,7 @@ PLAN-ROOT SINK
|
||||
| row-size=0B cardinality=10
|
||||
|
|
||||
00:SCAN HDFS [tpch_nested_parquet.customer c1]
|
||||
partitions=1/1 files=4 size=288.99MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
runtime filters: RF000 -> c1.c_custkey
|
||||
row-size=20B cardinality=150.00K
|
||||
====
|
||||
@@ -2386,6 +2395,7 @@ PLAN-ROOT SINK
|
||||
| | row-size=40B cardinality=10
|
||||
| |
|
||||
| |--04:UNNEST [c2.c_orders o2]
|
||||
| | limit: 1
|
||||
| | row-size=0B cardinality=10
|
||||
| |
|
||||
| 05:NESTED LOOP JOIN [CROSS JOIN]
|
||||
@@ -2402,11 +2412,11 @@ PLAN-ROOT SINK
|
||||
| row-size=40B cardinality=300.00K
|
||||
|
|
||||
|--01:SCAN HDFS [tpch_nested_parquet.customer c2]
|
||||
| partitions=1/1 files=4 size=288.99MB
|
||||
| HDFS partitions=1/1 files=4 size=289.08MB
|
||||
| row-size=20B cardinality=150.00K
|
||||
|
|
||||
00:SCAN HDFS [tpch_nested_parquet.customer c1]
|
||||
partitions=1/1 files=4 size=288.99MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
row-size=20B cardinality=150.00K
|
||||
====
|
||||
# IMPALA-3084: Test correct assignment of NULL checking predicates
|
||||
@@ -2431,7 +2441,7 @@ PLAN-ROOT SINK
|
||||
| row-size=0B cardinality=10
|
||||
|
|
||||
00:SCAN HDFS [tpch_nested_parquet.customer c]
|
||||
partitions=1/1 files=4 size=288.99MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
row-size=230B cardinality=150.00K
|
||||
====
|
||||
# IMPALA-2540: Complex query mixing joins on base tables and nested collections.
|
||||
@@ -2486,21 +2496,21 @@ PLAN-ROOT SINK
|
||||
| | | row-size=61B cardinality=5
|
||||
| | |
|
||||
| | |--03:SCAN HDFS [tpch_nested_parquet.region t5]
|
||||
| | | partitions=1/1 files=1 size=3.44KB
|
||||
| | | HDFS partitions=1/1 files=1 size=3.59KB
|
||||
| | | row-size=2B cardinality=5
|
||||
| | |
|
||||
| | 01:SCAN HDFS [tpch_nested_parquet.customer t2]
|
||||
| | partitions=1/1 files=4 size=288.99MB
|
||||
| | HDFS partitions=1/1 files=4 size=289.08MB
|
||||
| | runtime filters: RF004 -> t2.c_custkey
|
||||
| | row-size=59B cardinality=150.00K
|
||||
| |
|
||||
| 02:SCAN HDFS [tpch_nested_parquet.region t3]
|
||||
| partitions=1/1 files=1 size=3.44KB
|
||||
| HDFS partitions=1/1 files=1 size=3.59KB
|
||||
| runtime filters: RF002 -> t3.r_comment
|
||||
| row-size=78B cardinality=5
|
||||
|
|
||||
00:SCAN HDFS [tpch_nested_parquet.region.r_nations t1]
|
||||
partitions=1/1 files=1 size=3.44KB
|
||||
HDFS partitions=1/1 files=1 size=3.59KB
|
||||
runtime filters: RF000 -> t1.pos
|
||||
row-size=8B cardinality=50
|
||||
====
|
||||
@@ -2533,7 +2543,7 @@ PLAN-ROOT SINK
|
||||
| row-size=8B cardinality=2
|
||||
|
|
||||
00:SCAN HDFS [tpch_nested_parquet.customer c]
|
||||
partitions=1/1 files=4 size=288.99MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
predicates on c_orders: o_orderkey = 6000000
|
||||
row-size=20B cardinality=150.00K
|
||||
====
|
||||
@@ -2584,6 +2594,48 @@ PLAN-ROOT SINK
|
||||
| row-size=0B cardinality=10
|
||||
|
|
||||
00:SCAN HDFS [tpch_nested_parquet.customer c]
|
||||
partitions=1/1 files=4 size=288.99MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
row-size=20B cardinality=150.00K
|
||||
====
|
||||
# IMPALA-1270: SEMI JOIN in subplan with distinct added by planner.
|
||||
# The single node planner needs to correctly set the table ref ids for
|
||||
# the left branch of the semi join for subplan generation to work.
|
||||
select a.id, e.key from functional_parquet.complextypestbl a
|
||||
left semi join functional.alltypessmall c on (a.id = c.int_col)
|
||||
inner join a.nested_struct.g e
|
||||
where length(e.key) > 0
|
||||
---- PLAN
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
07:SUBPLAN
|
||||
| row-size=32B cardinality=44.00K
|
||||
|
|
||||
|--05:NESTED LOOP JOIN [CROSS JOIN]
|
||||
| | row-size=32B cardinality=10
|
||||
| |
|
||||
| |--03:SINGULAR ROW SRC
|
||||
| | row-size=20B cardinality=1
|
||||
| |
|
||||
| 04:UNNEST [a.nested_struct.g e]
|
||||
| row-size=0B cardinality=10
|
||||
|
|
||||
06:HASH JOIN [LEFT SEMI JOIN]
|
||||
| hash predicates: a.id = c.int_col
|
||||
| runtime filters: RF000 <- c.int_col
|
||||
| row-size=20B cardinality=4.40K
|
||||
|
|
||||
|--02:AGGREGATE [FINALIZE]
|
||||
| | group by: c.int_col
|
||||
| | row-size=4B cardinality=10
|
||||
| |
|
||||
| 01:SCAN HDFS [functional.alltypessmall c]
|
||||
| HDFS partitions=4/4 files=4 size=6.32KB
|
||||
| row-size=4B cardinality=100
|
||||
|
|
||||
00:SCAN HDFS [functional_parquet.complextypestbl a]
|
||||
HDFS partitions=1/1 files=2 size=6.92KB
|
||||
predicates: !empty(a.nested_struct.g)
|
||||
predicates on e: length(e.`key`) > 0
|
||||
runtime filters: RF000 -> a.id
|
||||
row-size=20B cardinality=4.40K
|
||||
====
|
||||
|
||||
@@ -14,7 +14,7 @@ PLAN-ROOT SINK
|
||||
| row-size=273B cardinality=7.30K
|
||||
|
|
||||
|--02:SCAN HDFS [functional.alltypesagg c]
|
||||
| partitions=11/11 files=11 size=814.73KB
|
||||
| HDFS partitions=11/11 files=11 size=814.73KB
|
||||
| predicates: c.bigint_col = 10
|
||||
| row-size=95B cardinality=11
|
||||
|
|
||||
@@ -23,11 +23,11 @@ PLAN-ROOT SINK
|
||||
| row-size=178B cardinality=7.30K
|
||||
|
|
||||
|--01:SCAN HDFS [functional.alltypes b]
|
||||
| partitions=24/24 files=24 size=478.45KB
|
||||
| HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| row-size=89B cardinality=7.30K
|
||||
|
|
||||
00:SCAN HDFS [functional.alltypestiny a]
|
||||
partitions=4/4 files=4 size=460B
|
||||
HDFS partitions=4/4 files=4 size=460B
|
||||
predicates: a.id < 10
|
||||
row-size=89B cardinality=1
|
||||
---- DISTRIBUTEDPLAN
|
||||
@@ -48,7 +48,7 @@ PLAN-ROOT SINK
|
||||
| row-size=95B cardinality=1.10K
|
||||
|
|
||||
|--02:SCAN HDFS [functional.alltypesagg d]
|
||||
| partitions=11/11 files=11 size=814.73KB
|
||||
| HDFS partitions=11/11 files=11 size=814.73KB
|
||||
| predicates: d.bigint_col < 10
|
||||
| row-size=95B cardinality=1.10K
|
||||
|
|
||||
@@ -57,11 +57,11 @@ PLAN-ROOT SINK
|
||||
| row-size=1B cardinality=100
|
||||
|
|
||||
|--01:SCAN HDFS [functional.alltypessmall c]
|
||||
| partitions=4/4 files=4 size=6.32KB
|
||||
| HDFS partitions=4/4 files=4 size=6.32KB
|
||||
| row-size=1B cardinality=100
|
||||
|
|
||||
00:SCAN HDFS [functional.alltypestiny a]
|
||||
partitions=4/4 files=4 size=460B
|
||||
HDFS partitions=4/4 files=4 size=460B
|
||||
row-size=1B cardinality=8
|
||||
---- DISTRIBUTEDPLAN
|
||||
NotImplementedException: Error generating a valid execution plan for this query. A RIGHT SEMI JOIN type with no equi-join predicates can only be executed with a single node plan.
|
||||
@@ -84,7 +84,7 @@ PLAN-ROOT SINK
|
||||
| row-size=362B cardinality=18.40K
|
||||
|
|
||||
|--03:SCAN HDFS [functional.alltypes d]
|
||||
| partitions=24/24 files=24 size=478.45KB
|
||||
| HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| row-size=89B cardinality=7.30K
|
||||
|
|
||||
05:NESTED LOOP JOIN [FULL OUTER JOIN]
|
||||
@@ -92,7 +92,7 @@ PLAN-ROOT SINK
|
||||
| row-size=273B cardinality=11.10K
|
||||
|
|
||||
|--02:SCAN HDFS [functional.alltypesagg c]
|
||||
| partitions=11/11 files=11 size=814.73KB
|
||||
| HDFS partitions=11/11 files=11 size=814.73KB
|
||||
| row-size=95B cardinality=11.00K
|
||||
|
|
||||
04:NESTED LOOP JOIN [FULL OUTER JOIN]
|
||||
@@ -100,11 +100,11 @@ PLAN-ROOT SINK
|
||||
| row-size=178B cardinality=101
|
||||
|
|
||||
|--01:SCAN HDFS [functional.alltypessmall b]
|
||||
| partitions=4/4 files=4 size=6.32KB
|
||||
| HDFS partitions=4/4 files=4 size=6.32KB
|
||||
| row-size=89B cardinality=100
|
||||
|
|
||||
00:SCAN HDFS [functional.alltypestiny a]
|
||||
partitions=4/4 files=4 size=460B
|
||||
HDFS partitions=4/4 files=4 size=460B
|
||||
predicates: a.id < 10
|
||||
row-size=89B cardinality=1
|
||||
---- DISTRIBUTEDPLAN
|
||||
@@ -127,12 +127,12 @@ PLAN-ROOT SINK
|
||||
| row-size=8B cardinality=10
|
||||
|
|
||||
|--01:SCAN HDFS [functional.alltypessmall b]
|
||||
| partitions=4/4 files=4 size=6.32KB
|
||||
| HDFS partitions=4/4 files=4 size=6.32KB
|
||||
| predicates: b.int_col = 5
|
||||
| row-size=8B cardinality=10
|
||||
|
|
||||
00:SCAN HDFS [functional.alltypestiny a]
|
||||
partitions=4/4 files=4 size=460B
|
||||
HDFS partitions=4/4 files=4 size=460B
|
||||
row-size=4B cardinality=8
|
||||
---- DISTRIBUTEDPLAN
|
||||
NotImplementedException: Error generating a valid execution plan for this query. A RIGHT ANTI JOIN type with no equi-join predicates can only be executed with a single node plan.
|
||||
@@ -149,14 +149,14 @@ PLAN-ROOT SINK
|
||||
|
|
||||
09:AGGREGATE [FINALIZE]
|
||||
| output: count(*)
|
||||
| row-size=8B cardinality=0
|
||||
| row-size=8B cardinality=1
|
||||
|
|
||||
08:NESTED LOOP JOIN [RIGHT ANTI JOIN]
|
||||
| join predicates: d.tinyint_col > e.tinyint_col
|
||||
| row-size=5B cardinality=0
|
||||
|
|
||||
|--04:SCAN HDFS [functional.alltypesnopart e]
|
||||
| partitions=1/1 files=0 size=0B
|
||||
| HDFS partitions=1/1 files=0 size=0B
|
||||
| predicates: e.id < 10
|
||||
| row-size=5B cardinality=0
|
||||
|
|
||||
@@ -165,7 +165,7 @@ PLAN-ROOT SINK
|
||||
| row-size=1B cardinality=7.30K
|
||||
|
|
||||
|--03:SCAN HDFS [functional.alltypes d]
|
||||
| partitions=24/24 files=24 size=478.45KB
|
||||
| HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| row-size=1B cardinality=7.30K
|
||||
|
|
||||
06:NESTED LOOP JOIN [RIGHT OUTER JOIN]
|
||||
@@ -173,7 +173,7 @@ PLAN-ROOT SINK
|
||||
| row-size=17B cardinality=11.00K
|
||||
|
|
||||
|--02:SCAN HDFS [functional.alltypesagg c]
|
||||
| partitions=11/11 files=11 size=814.73KB
|
||||
| HDFS partitions=11/11 files=11 size=814.73KB
|
||||
| row-size=5B cardinality=11.00K
|
||||
|
|
||||
05:NESTED LOOP JOIN [INNER JOIN]
|
||||
@@ -181,11 +181,11 @@ PLAN-ROOT SINK
|
||||
| row-size=12B cardinality=8
|
||||
|
|
||||
|--01:SCAN HDFS [functional.alltypessmall b]
|
||||
| partitions=4/4 files=4 size=6.32KB
|
||||
| HDFS partitions=4/4 files=4 size=6.32KB
|
||||
| row-size=4B cardinality=100
|
||||
|
|
||||
00:SCAN HDFS [functional.alltypestiny a]
|
||||
partitions=4/4 files=4 size=460B
|
||||
HDFS partitions=4/4 files=4 size=460B
|
||||
row-size=8B cardinality=8
|
||||
---- DISTRIBUTEDPLAN
|
||||
NotImplementedException: Error generating a valid execution plan for this query. A RIGHT ANTI JOIN type with no equi-join predicates can only be executed with a single node plan.
|
||||
@@ -203,13 +203,13 @@ PLAN-ROOT SINK
|
||||
|
|
||||
15:AGGREGATE [FINALIZE]
|
||||
| output: count:merge(*)
|
||||
| row-size=8B cardinality=0
|
||||
| row-size=8B cardinality=1
|
||||
|
|
||||
14:EXCHANGE [UNPARTITIONED]
|
||||
|
|
||||
09:AGGREGATE
|
||||
| output: count(*)
|
||||
| row-size=8B cardinality=0
|
||||
| row-size=8B cardinality=1
|
||||
|
|
||||
08:NESTED LOOP JOIN [LEFT ANTI JOIN, BROADCAST]
|
||||
| join predicates: d.tinyint_col > e.tinyint_col
|
||||
@@ -236,23 +236,23 @@ PLAN-ROOT SINK
|
||||
| | | |--10:EXCHANGE [BROADCAST]
|
||||
| | | | |
|
||||
| | | | 00:SCAN HDFS [functional.alltypestiny a]
|
||||
| | | | partitions=4/4 files=4 size=460B
|
||||
| | | | HDFS partitions=4/4 files=4 size=460B
|
||||
| | | | row-size=8B cardinality=8
|
||||
| | | |
|
||||
| | | 01:SCAN HDFS [functional.alltypessmall b]
|
||||
| | | partitions=4/4 files=4 size=6.32KB
|
||||
| | | HDFS partitions=4/4 files=4 size=6.32KB
|
||||
| | | row-size=4B cardinality=100
|
||||
| | |
|
||||
| | 02:SCAN HDFS [functional.alltypesagg c]
|
||||
| | partitions=11/11 files=11 size=814.73KB
|
||||
| | HDFS partitions=11/11 files=11 size=814.73KB
|
||||
| | row-size=5B cardinality=11.00K
|
||||
| |
|
||||
| 03:SCAN HDFS [functional.alltypes d]
|
||||
| partitions=24/24 files=24 size=478.45KB
|
||||
| HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| row-size=1B cardinality=7.30K
|
||||
|
|
||||
04:SCAN HDFS [functional.alltypesnopart e]
|
||||
partitions=1/1 files=0 size=0B
|
||||
HDFS partitions=1/1 files=0 size=0B
|
||||
predicates: e.id < 10
|
||||
row-size=5B cardinality=0
|
||||
====
|
||||
@@ -272,7 +272,7 @@ PLAN-ROOT SINK
|
||||
|--06:EXCHANGE [BROADCAST]
|
||||
| |
|
||||
| 02:SCAN HDFS [functional.alltypes t2]
|
||||
| partitions=24/24 files=24 size=478.45KB
|
||||
| HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| predicates: (t2.bigint_col = 5)
|
||||
| row-size=89B cardinality=730
|
||||
|
|
||||
@@ -287,7 +287,7 @@ PLAN-ROOT SINK
|
||||
| row-size=4B cardinality=10
|
||||
|
|
||||
00:SCAN HDFS [functional.alltypes]
|
||||
partitions=24/24 files=24 size=478.45KB
|
||||
HDFS partitions=24/24 files=24 size=478.45KB
|
||||
row-size=4B cardinality=7.30K
|
||||
====
|
||||
# IMPALA-5689: Do not invert a left semi join with no equi-join predicates.
|
||||
@@ -298,16 +298,20 @@ left semi join functional.alltypes t2 on (t2.bigint_col=5)
|
||||
---- DISTRIBUTEDPLAN
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
07:EXCHANGE [UNPARTITIONED]
|
||||
08:EXCHANGE [UNPARTITIONED]
|
||||
|
|
||||
03:NESTED LOOP JOIN [LEFT SEMI JOIN, BROADCAST]
|
||||
| row-size=4B cardinality=10
|
||||
|
|
||||
|--06:EXCHANGE [BROADCAST]
|
||||
|--07:EXCHANGE [BROADCAST]
|
||||
| |
|
||||
| 06:EXCHANGE [UNPARTITIONED]
|
||||
| | limit: 1
|
||||
| |
|
||||
| 02:SCAN HDFS [functional.alltypes t2]
|
||||
| partitions=24/24 files=24 size=478.45KB
|
||||
| HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| predicates: (t2.bigint_col = 5)
|
||||
| limit: 1
|
||||
| row-size=8B cardinality=730
|
||||
|
|
||||
05:AGGREGATE [FINALIZE]
|
||||
@@ -321,6 +325,6 @@ PLAN-ROOT SINK
|
||||
| row-size=4B cardinality=10
|
||||
|
|
||||
00:SCAN HDFS [functional.alltypes]
|
||||
partitions=24/24 files=24 size=478.45KB
|
||||
HDFS partitions=24/24 files=24 size=478.45KB
|
||||
row-size=4B cardinality=7.30K
|
||||
====
|
||||
|
||||
@@ -1148,6 +1148,7 @@ PLAN-ROOT SINK
|
||||
|
|
||||
|--03:SCAN HDFS [functional.alltypes d]
|
||||
| HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| limit: 1
|
||||
| row-size=0B cardinality=7.30K
|
||||
|
|
||||
05:HASH JOIN [RIGHT OUTER JOIN]
|
||||
|
||||
973
testdata/workloads/functional-planner/queries/PlannerTest/semi-join-distinct.test
vendored
Normal file
973
testdata/workloads/functional-planner/queries/PlannerTest/semi-join-distinct.test
vendored
Normal file
@@ -0,0 +1,973 @@
|
||||
# IMPALA-1270: distinct should be added to subquery automatically because
|
||||
# it would reduce cardinality significantly.
|
||||
select * from functional.alltypestiny
|
||||
where int_col in (select int_col from functional.alltypes where id % 2 = 0)
|
||||
---- PLAN
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
03:HASH JOIN [LEFT SEMI JOIN]
|
||||
| hash predicates: int_col = functional.alltypes.int_col
|
||||
| runtime filters: RF000 <- functional.alltypes.int_col
|
||||
| row-size=89B cardinality=8
|
||||
|
|
||||
|--02:AGGREGATE [FINALIZE]
|
||||
| | group by: functional.alltypes.int_col
|
||||
| | row-size=4B cardinality=10
|
||||
| |
|
||||
| 01:SCAN HDFS [functional.alltypes]
|
||||
| HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| predicates: id % 2 = 0
|
||||
| row-size=8B cardinality=730
|
||||
|
|
||||
00:SCAN HDFS [functional.alltypestiny]
|
||||
HDFS partitions=4/4 files=4 size=460B
|
||||
runtime filters: RF000 -> int_col
|
||||
row-size=89B cardinality=8
|
||||
---- PARALLELPLANS
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
07:EXCHANGE [UNPARTITIONED]
|
||||
|
|
||||
03:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
|
||||
| hash predicates: int_col = functional.alltypes.int_col
|
||||
| row-size=89B cardinality=8
|
||||
|
|
||||
|--JOIN BUILD
|
||||
| | join-table-id=00 plan-id=01 cohort-id=01
|
||||
| | build expressions: functional.alltypes.int_col
|
||||
| | runtime filters: RF000 <- functional.alltypes.int_col
|
||||
| |
|
||||
| 06:EXCHANGE [BROADCAST]
|
||||
| |
|
||||
| 05:AGGREGATE [FINALIZE]
|
||||
| | group by: functional.alltypes.int_col
|
||||
| | row-size=4B cardinality=10
|
||||
| |
|
||||
| 04:EXCHANGE [HASH(functional.alltypes.int_col)]
|
||||
| |
|
||||
| 02:AGGREGATE [STREAMING]
|
||||
| | group by: functional.alltypes.int_col
|
||||
| | row-size=4B cardinality=10
|
||||
| |
|
||||
| 01:SCAN HDFS [functional.alltypes]
|
||||
| HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| predicates: id % 2 = 0
|
||||
| row-size=8B cardinality=730
|
||||
|
|
||||
00:SCAN HDFS [functional.alltypestiny]
|
||||
HDFS partitions=4/4 files=4 size=460B
|
||||
runtime filters: RF000 -> int_col
|
||||
row-size=89B cardinality=8
|
||||
====
|
||||
# IMPALA-1270: distinct should not be added to subquery when column stats
|
||||
# are missing (as they are on functional_parquet.alltypes).
|
||||
select * from functional.alltypestiny
|
||||
where int_col in (select int_col from functional_parquet.alltypes)
|
||||
---- PLAN
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
02:HASH JOIN [RIGHT SEMI JOIN]
|
||||
| hash predicates: int_col = int_col
|
||||
| runtime filters: RF000 <- int_col
|
||||
| row-size=89B cardinality=8
|
||||
|
|
||||
|--00:SCAN HDFS [functional.alltypestiny]
|
||||
| HDFS partitions=4/4 files=4 size=460B
|
||||
| row-size=89B cardinality=8
|
||||
|
|
||||
01:SCAN HDFS [functional_parquet.alltypes]
|
||||
HDFS partitions=24/24 files=24 size=202.01KB
|
||||
runtime filters: RF000 -> int_col
|
||||
row-size=4B cardinality=12.85K
|
||||
---- PARALLELPLANS
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
05:EXCHANGE [UNPARTITIONED]
|
||||
|
|
||||
02:HASH JOIN [RIGHT SEMI JOIN, PARTITIONED]
|
||||
| hash predicates: int_col = int_col
|
||||
| row-size=89B cardinality=8
|
||||
|
|
||||
|--JOIN BUILD
|
||||
| | join-table-id=00 plan-id=01 cohort-id=01
|
||||
| | build expressions: int_col
|
||||
| | runtime filters: RF000 <- int_col
|
||||
| |
|
||||
| 04:EXCHANGE [HASH(int_col)]
|
||||
| |
|
||||
| 00:SCAN HDFS [functional.alltypestiny]
|
||||
| HDFS partitions=4/4 files=4 size=460B
|
||||
| row-size=89B cardinality=8
|
||||
|
|
||||
03:EXCHANGE [HASH(int_col)]
|
||||
|
|
||||
01:SCAN HDFS [functional_parquet.alltypes]
|
||||
HDFS partitions=24/24 files=24 size=202.01KB
|
||||
runtime filters: RF000 -> int_col
|
||||
row-size=4B cardinality=12.85K
|
||||
====
|
||||
# IMPALA-1270: distinct should be added to subquery automatically because
|
||||
# it would reduce cardinality significantly.
|
||||
select * from functional.alltypestiny
|
||||
where int_col in (select int_col from functional.alltypes)
|
||||
---- PLAN
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
03:HASH JOIN [LEFT SEMI JOIN]
|
||||
| hash predicates: int_col = functional.alltypes.int_col
|
||||
| runtime filters: RF000 <- functional.alltypes.int_col
|
||||
| row-size=89B cardinality=8
|
||||
|
|
||||
|--02:AGGREGATE [FINALIZE]
|
||||
| | group by: functional.alltypes.int_col
|
||||
| | row-size=4B cardinality=10
|
||||
| |
|
||||
| 01:SCAN HDFS [functional.alltypes]
|
||||
| HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| row-size=4B cardinality=7.30K
|
||||
|
|
||||
00:SCAN HDFS [functional.alltypestiny]
|
||||
HDFS partitions=4/4 files=4 size=460B
|
||||
runtime filters: RF000 -> int_col
|
||||
row-size=89B cardinality=8
|
||||
---- PARALLELPLANS
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
07:EXCHANGE [UNPARTITIONED]
|
||||
|
|
||||
03:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
|
||||
| hash predicates: int_col = functional.alltypes.int_col
|
||||
| row-size=89B cardinality=8
|
||||
|
|
||||
|--JOIN BUILD
|
||||
| | join-table-id=00 plan-id=01 cohort-id=01
|
||||
| | build expressions: functional.alltypes.int_col
|
||||
| | runtime filters: RF000 <- functional.alltypes.int_col
|
||||
| |
|
||||
| 06:EXCHANGE [BROADCAST]
|
||||
| |
|
||||
| 05:AGGREGATE [FINALIZE]
|
||||
| | group by: functional.alltypes.int_col
|
||||
| | row-size=4B cardinality=10
|
||||
| |
|
||||
| 04:EXCHANGE [HASH(functional.alltypes.int_col)]
|
||||
| |
|
||||
| 02:AGGREGATE [STREAMING]
|
||||
| | group by: functional.alltypes.int_col
|
||||
| | row-size=4B cardinality=10
|
||||
| |
|
||||
| 01:SCAN HDFS [functional.alltypes]
|
||||
| HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| row-size=4B cardinality=7.30K
|
||||
|
|
||||
00:SCAN HDFS [functional.alltypestiny]
|
||||
HDFS partitions=4/4 files=4 size=460B
|
||||
runtime filters: RF000 -> int_col
|
||||
row-size=89B cardinality=8
|
||||
====
|
||||
# IMPALA-1270: distinct should not be added to subquery when it does not
|
||||
# reduce cardinality significantly.
|
||||
select * from functional.alltypestiny
|
||||
where int_col in (select int_col from functional.alltypes where id in (1,2,3))
|
||||
---- PLAN
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
02:HASH JOIN [LEFT SEMI JOIN]
|
||||
| hash predicates: int_col = int_col
|
||||
| runtime filters: RF000 <- int_col
|
||||
| row-size=89B cardinality=8
|
||||
|
|
||||
|--01:SCAN HDFS [functional.alltypes]
|
||||
| HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| predicates: id IN (1, 2, 3)
|
||||
| row-size=8B cardinality=3
|
||||
|
|
||||
00:SCAN HDFS [functional.alltypestiny]
|
||||
HDFS partitions=4/4 files=4 size=460B
|
||||
runtime filters: RF000 -> int_col
|
||||
row-size=89B cardinality=8
|
||||
---- PARALLELPLANS
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
04:EXCHANGE [UNPARTITIONED]
|
||||
|
|
||||
02:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
|
||||
| hash predicates: int_col = int_col
|
||||
| row-size=89B cardinality=8
|
||||
|
|
||||
|--JOIN BUILD
|
||||
| | join-table-id=00 plan-id=01 cohort-id=01
|
||||
| | build expressions: int_col
|
||||
| | runtime filters: RF000 <- int_col
|
||||
| |
|
||||
| 03:EXCHANGE [BROADCAST]
|
||||
| |
|
||||
| 01:SCAN HDFS [functional.alltypes]
|
||||
| HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| predicates: id IN (1, 2, 3)
|
||||
| row-size=8B cardinality=3
|
||||
|
|
||||
00:SCAN HDFS [functional.alltypestiny]
|
||||
HDFS partitions=4/4 files=4 size=460B
|
||||
runtime filters: RF000 -> int_col
|
||||
row-size=89B cardinality=8
|
||||
====
|
||||
# IMPALA-1270: distinct should not be added to subquery that returns 0 rows
|
||||
# reduce cardinality significantly.
|
||||
select * from functional.alltypestiny
|
||||
where int_col in (select int_col from functional.alltypes limit 0)
|
||||
---- PLAN
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
02:HASH JOIN [LEFT SEMI JOIN]
|
||||
| hash predicates: int_col = int_col
|
||||
| runtime filters: RF000 <- int_col
|
||||
| row-size=89B cardinality=0
|
||||
|
|
||||
|--01:EMPTYSET
|
||||
|
|
||||
00:SCAN HDFS [functional.alltypestiny]
|
||||
HDFS partitions=4/4 files=4 size=460B
|
||||
runtime filters: RF000 -> int_col
|
||||
row-size=89B cardinality=8
|
||||
---- PARALLELPLANS
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
04:EXCHANGE [UNPARTITIONED]
|
||||
|
|
||||
02:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
|
||||
| hash predicates: int_col = int_col
|
||||
| row-size=89B cardinality=0
|
||||
|
|
||||
|--JOIN BUILD
|
||||
| | join-table-id=00 plan-id=01 cohort-id=01
|
||||
| | build expressions: int_col
|
||||
| | runtime filters: RF000 <- int_col
|
||||
| |
|
||||
| 03:EXCHANGE [BROADCAST]
|
||||
| |
|
||||
| 01:EMPTYSET
|
||||
|
|
||||
00:SCAN HDFS [functional.alltypestiny]
|
||||
HDFS partitions=4/4 files=4 size=460B
|
||||
runtime filters: RF000 -> int_col
|
||||
row-size=89B cardinality=8
|
||||
====
|
||||
# IMPALA-1270: limit should be added to subquery that results in a semi
|
||||
# join with no join predicates.
|
||||
select * from functional.alltypestiny
|
||||
where exists (select int_col from functional.alltypes)
|
||||
---- PLAN
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
02:NESTED LOOP JOIN [LEFT SEMI JOIN]
|
||||
| row-size=89B cardinality=8
|
||||
|
|
||||
|--01:SCAN HDFS [functional.alltypes]
|
||||
| HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| limit: 1
|
||||
| row-size=0B cardinality=1
|
||||
|
|
||||
00:SCAN HDFS [functional.alltypestiny]
|
||||
HDFS partitions=4/4 files=4 size=460B
|
||||
row-size=89B cardinality=8
|
||||
---- PARALLELPLANS
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
05:EXCHANGE [UNPARTITIONED]
|
||||
|
|
||||
02:NESTED LOOP JOIN [LEFT SEMI JOIN, BROADCAST]
|
||||
| join table id: 00
|
||||
| row-size=89B cardinality=8
|
||||
|
|
||||
|--JOIN BUILD
|
||||
| | join-table-id=00 plan-id=01 cohort-id=01
|
||||
| |
|
||||
| 04:EXCHANGE [BROADCAST]
|
||||
| |
|
||||
| 03:EXCHANGE [UNPARTITIONED]
|
||||
| | limit: 1
|
||||
| |
|
||||
| 01:SCAN HDFS [functional.alltypes]
|
||||
| HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| limit: 1
|
||||
| row-size=0B cardinality=1
|
||||
|
|
||||
00:SCAN HDFS [functional.alltypestiny]
|
||||
HDFS partitions=4/4 files=4 size=460B
|
||||
row-size=89B cardinality=8
|
||||
====
|
||||
# IMPALA-1270: the added aggregation does not result in an extra exchange for
|
||||
# shuffle joins.
|
||||
select straight_join *
|
||||
from functional.alltypestiny t1
|
||||
left semi join /*+shuffle*/ functional.alltypes t2 on t1.int_col = t2.int_col
|
||||
---- PLAN
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
03:HASH JOIN [LEFT SEMI JOIN]
|
||||
| hash predicates: t1.int_col = t2.int_col
|
||||
| runtime filters: RF000 <- t2.int_col
|
||||
| row-size=89B cardinality=8
|
||||
|
|
||||
|--02:AGGREGATE [FINALIZE]
|
||||
| | group by: t2.int_col
|
||||
| | row-size=4B cardinality=10
|
||||
| |
|
||||
| 01:SCAN HDFS [functional.alltypes t2]
|
||||
| HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| row-size=4B cardinality=7.30K
|
||||
|
|
||||
00:SCAN HDFS [functional.alltypestiny t1]
|
||||
HDFS partitions=4/4 files=4 size=460B
|
||||
runtime filters: RF000 -> t1.int_col
|
||||
row-size=89B cardinality=8
|
||||
---- PARALLELPLANS
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
07:EXCHANGE [UNPARTITIONED]
|
||||
|
|
||||
03:HASH JOIN [LEFT SEMI JOIN, PARTITIONED]
|
||||
| hash predicates: t1.int_col = t2.int_col
|
||||
| row-size=89B cardinality=8
|
||||
|
|
||||
|--JOIN BUILD
|
||||
| | join-table-id=00 plan-id=01 cohort-id=01
|
||||
| | build expressions: t2.int_col
|
||||
| | runtime filters: RF000 <- t2.int_col
|
||||
| |
|
||||
| 05:AGGREGATE [FINALIZE]
|
||||
| | group by: t2.int_col
|
||||
| | row-size=4B cardinality=10
|
||||
| |
|
||||
| 04:EXCHANGE [HASH(t2.int_col)]
|
||||
| |
|
||||
| 02:AGGREGATE [STREAMING]
|
||||
| | group by: t2.int_col
|
||||
| | row-size=4B cardinality=10
|
||||
| |
|
||||
| 01:SCAN HDFS [functional.alltypes t2]
|
||||
| HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| row-size=4B cardinality=7.30K
|
||||
|
|
||||
06:EXCHANGE [HASH(t1.int_col)]
|
||||
|
|
||||
00:SCAN HDFS [functional.alltypestiny t1]
|
||||
HDFS partitions=4/4 files=4 size=460B
|
||||
runtime filters: RF000 -> t1.int_col
|
||||
row-size=89B cardinality=8
|
||||
====
|
||||
# IMPALA-1270: the distinct optimisation also applies to NULL AWARE LEFT ANTI JOIN
|
||||
select * from functional.alltypestiny
|
||||
where int_col not in (select int_col from functional.alltypes where id % 2 = 0)
|
||||
---- PLAN
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
03:HASH JOIN [NULL AWARE LEFT ANTI JOIN]
|
||||
| hash predicates: int_col = functional.alltypes.int_col
|
||||
| row-size=89B cardinality=8
|
||||
|
|
||||
|--02:AGGREGATE [FINALIZE]
|
||||
| | group by: functional.alltypes.int_col
|
||||
| | row-size=4B cardinality=10
|
||||
| |
|
||||
| 01:SCAN HDFS [functional.alltypes]
|
||||
| HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| predicates: id % 2 = 0
|
||||
| row-size=8B cardinality=730
|
||||
|
|
||||
00:SCAN HDFS [functional.alltypestiny]
|
||||
HDFS partitions=4/4 files=4 size=460B
|
||||
row-size=89B cardinality=8
|
||||
---- PARALLELPLANS
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
07:EXCHANGE [UNPARTITIONED]
|
||||
|
|
||||
03:HASH JOIN [NULL AWARE LEFT ANTI JOIN, BROADCAST]
|
||||
| hash predicates: int_col = functional.alltypes.int_col
|
||||
| row-size=89B cardinality=8
|
||||
|
|
||||
|--JOIN BUILD
|
||||
| | join-table-id=00 plan-id=01 cohort-id=01
|
||||
| | build expressions: functional.alltypes.int_col
|
||||
| |
|
||||
| 06:EXCHANGE [BROADCAST]
|
||||
| |
|
||||
| 05:AGGREGATE [FINALIZE]
|
||||
| | group by: functional.alltypes.int_col
|
||||
| | row-size=4B cardinality=10
|
||||
| |
|
||||
| 04:EXCHANGE [HASH(functional.alltypes.int_col)]
|
||||
| |
|
||||
| 02:AGGREGATE [STREAMING]
|
||||
| | group by: functional.alltypes.int_col
|
||||
| | row-size=4B cardinality=10
|
||||
| |
|
||||
| 01:SCAN HDFS [functional.alltypes]
|
||||
| HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| predicates: id % 2 = 0
|
||||
| row-size=8B cardinality=730
|
||||
|
|
||||
00:SCAN HDFS [functional.alltypestiny]
|
||||
HDFS partitions=4/4 files=4 size=460B
|
||||
row-size=89B cardinality=8
|
||||
====
|
||||
# IMPALA-1270: the distinct optimisation also applies to LEFT ANTI JOIN
|
||||
select * from functional.alltypestiny t1
|
||||
where not exists (
|
||||
select int_col from functional.alltypes t2
|
||||
where id % 2 = 0 and t1.int_col = t2.int_col)
|
||||
---- PLAN
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
03:HASH JOIN [LEFT ANTI JOIN]
|
||||
| hash predicates: t1.int_col = t2.int_col
|
||||
| row-size=89B cardinality=8
|
||||
|
|
||||
|--02:AGGREGATE [FINALIZE]
|
||||
| | group by: t2.int_col
|
||||
| | row-size=4B cardinality=10
|
||||
| |
|
||||
| 01:SCAN HDFS [functional.alltypes t2]
|
||||
| HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| predicates: id % 2 = 0
|
||||
| row-size=8B cardinality=730
|
||||
|
|
||||
00:SCAN HDFS [functional.alltypestiny t1]
|
||||
HDFS partitions=4/4 files=4 size=460B
|
||||
row-size=89B cardinality=8
|
||||
---- PARALLELPLANS
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
07:EXCHANGE [UNPARTITIONED]
|
||||
|
|
||||
03:HASH JOIN [LEFT ANTI JOIN, BROADCAST]
|
||||
| hash predicates: t1.int_col = t2.int_col
|
||||
| row-size=89B cardinality=8
|
||||
|
|
||||
|--JOIN BUILD
|
||||
| | join-table-id=00 plan-id=01 cohort-id=01
|
||||
| | build expressions: t2.int_col
|
||||
| |
|
||||
| 06:EXCHANGE [BROADCAST]
|
||||
| |
|
||||
| 05:AGGREGATE [FINALIZE]
|
||||
| | group by: t2.int_col
|
||||
| | row-size=4B cardinality=10
|
||||
| |
|
||||
| 04:EXCHANGE [HASH(t2.int_col)]
|
||||
| |
|
||||
| 02:AGGREGATE [STREAMING]
|
||||
| | group by: t2.int_col
|
||||
| | row-size=4B cardinality=10
|
||||
| |
|
||||
| 01:SCAN HDFS [functional.alltypes t2]
|
||||
| HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| predicates: id % 2 = 0
|
||||
| row-size=8B cardinality=730
|
||||
|
|
||||
00:SCAN HDFS [functional.alltypestiny t1]
|
||||
HDFS partitions=4/4 files=4 size=460B
|
||||
row-size=89B cardinality=8
|
||||
====
|
||||
# IMPALA-1270: multi-column join showing that unused slots are projected.
|
||||
select count(*) from functional.alltypesagg t1
|
||||
where int_col in (
|
||||
select int_col from functional.alltypes t2
|
||||
where t1.bool_col = t2.bool_col and id is not null)
|
||||
---- PLAN
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
04:AGGREGATE [FINALIZE]
|
||||
| output: count(*)
|
||||
| row-size=8B cardinality=1
|
||||
|
|
||||
03:HASH JOIN [LEFT SEMI JOIN]
|
||||
| hash predicates: int_col = t2.int_col, t1.bool_col = t2.bool_col
|
||||
| runtime filters: RF000 <- t2.int_col, RF001 <- t2.bool_col
|
||||
| row-size=5B cardinality=115
|
||||
|
|
||||
|--02:AGGREGATE [FINALIZE]
|
||||
| | group by: t2.bool_col, t2.int_col
|
||||
| | row-size=5B cardinality=20
|
||||
| |
|
||||
| 01:SCAN HDFS [functional.alltypes t2]
|
||||
| HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| predicates: id IS NOT NULL
|
||||
| row-size=9B cardinality=7.30K
|
||||
|
|
||||
00:SCAN HDFS [functional.alltypesagg t1]
|
||||
HDFS partitions=11/11 files=11 size=814.73KB
|
||||
runtime filters: RF000 -> int_col, RF001 -> t1.bool_col
|
||||
row-size=5B cardinality=11.00K
|
||||
---- PARALLELPLANS
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
09:AGGREGATE [FINALIZE]
|
||||
| output: count:merge(*)
|
||||
| row-size=8B cardinality=1
|
||||
|
|
||||
08:EXCHANGE [UNPARTITIONED]
|
||||
|
|
||||
04:AGGREGATE
|
||||
| output: count(*)
|
||||
| row-size=8B cardinality=1
|
||||
|
|
||||
03:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
|
||||
| hash predicates: int_col = t2.int_col, t1.bool_col = t2.bool_col
|
||||
| row-size=5B cardinality=115
|
||||
|
|
||||
|--JOIN BUILD
|
||||
| | join-table-id=00 plan-id=01 cohort-id=01
|
||||
| | build expressions: t2.int_col, t2.bool_col
|
||||
| | runtime filters: RF000 <- t2.int_col, RF001 <- t2.bool_col
|
||||
| |
|
||||
| 07:EXCHANGE [BROADCAST]
|
||||
| |
|
||||
| 06:AGGREGATE [FINALIZE]
|
||||
| | group by: t2.bool_col, t2.int_col
|
||||
| | row-size=5B cardinality=20
|
||||
| |
|
||||
| 05:EXCHANGE [HASH(t2.bool_col,t2.int_col)]
|
||||
| |
|
||||
| 02:AGGREGATE [STREAMING]
|
||||
| | group by: t2.bool_col, t2.int_col
|
||||
| | row-size=5B cardinality=20
|
||||
| |
|
||||
| 01:SCAN HDFS [functional.alltypes t2]
|
||||
| HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| predicates: id IS NOT NULL
|
||||
| row-size=9B cardinality=7.30K
|
||||
|
|
||||
00:SCAN HDFS [functional.alltypesagg t1]
|
||||
HDFS partitions=11/11 files=11 size=814.73KB
|
||||
runtime filters: RF000 -> int_col, RF001 -> t1.bool_col
|
||||
row-size=5B cardinality=11.00K
|
||||
====
|
||||
# IMPALA-1270: aggregation can be added on top of existing aggregation if it would
|
||||
# reduce cardinality enough.
|
||||
select count(*) from functional.alltypesagg t1
|
||||
where int_col in (
|
||||
select int_col from functional.alltypes t2
|
||||
group by int_col, id
|
||||
having sum(int_col) > 1)
|
||||
---- PLAN
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
05:AGGREGATE [FINALIZE]
|
||||
| output: count(*)
|
||||
| row-size=8B cardinality=1
|
||||
|
|
||||
04:HASH JOIN [LEFT SEMI JOIN]
|
||||
| hash predicates: int_col = int_col
|
||||
| runtime filters: RF000 <- int_col
|
||||
| row-size=4B cardinality=115
|
||||
|
|
||||
|--03:AGGREGATE [FINALIZE]
|
||||
| | group by: int_col
|
||||
| | row-size=4B cardinality=10
|
||||
| |
|
||||
| 02:AGGREGATE [FINALIZE]
|
||||
| | output: sum(int_col)
|
||||
| | group by: int_col, id
|
||||
| | having: sum(int_col) > 1
|
||||
| | row-size=16B cardinality=730
|
||||
| |
|
||||
| 01:SCAN HDFS [functional.alltypes t2]
|
||||
| HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| row-size=8B cardinality=7.30K
|
||||
|
|
||||
00:SCAN HDFS [functional.alltypesagg t1]
|
||||
HDFS partitions=11/11 files=11 size=814.73KB
|
||||
runtime filters: RF000 -> int_col
|
||||
row-size=4B cardinality=11.00K
|
||||
---- PARALLELPLANS
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
12:AGGREGATE [FINALIZE]
|
||||
| output: count:merge(*)
|
||||
| row-size=8B cardinality=1
|
||||
|
|
||||
11:EXCHANGE [UNPARTITIONED]
|
||||
|
|
||||
05:AGGREGATE
|
||||
| output: count(*)
|
||||
| row-size=8B cardinality=1
|
||||
|
|
||||
04:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
|
||||
| hash predicates: int_col = int_col
|
||||
| row-size=4B cardinality=115
|
||||
|
|
||||
|--JOIN BUILD
|
||||
| | join-table-id=00 plan-id=01 cohort-id=01
|
||||
| | build expressions: int_col
|
||||
| | runtime filters: RF000 <- int_col
|
||||
| |
|
||||
| 10:EXCHANGE [BROADCAST]
|
||||
| |
|
||||
| 09:AGGREGATE [FINALIZE]
|
||||
| | group by: int_col
|
||||
| | row-size=4B cardinality=10
|
||||
| |
|
||||
| 08:EXCHANGE [HASH(int_col)]
|
||||
| |
|
||||
| 03:AGGREGATE [STREAMING]
|
||||
| | group by: int_col
|
||||
| | row-size=4B cardinality=10
|
||||
| |
|
||||
| 07:AGGREGATE [FINALIZE]
|
||||
| | output: sum:merge(int_col)
|
||||
| | group by: int_col, id
|
||||
| | having: sum(int_col) > 1
|
||||
| | row-size=16B cardinality=730
|
||||
| |
|
||||
| 06:EXCHANGE [HASH(int_col,id)]
|
||||
| |
|
||||
| 02:AGGREGATE [STREAMING]
|
||||
| | output: sum(int_col)
|
||||
| | group by: int_col, id
|
||||
| | row-size=16B cardinality=7.30K
|
||||
| |
|
||||
| 01:SCAN HDFS [functional.alltypes t2]
|
||||
| HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| row-size=8B cardinality=7.30K
|
||||
|
|
||||
00:SCAN HDFS [functional.alltypesagg t1]
|
||||
HDFS partitions=11/11 files=11 size=814.73KB
|
||||
runtime filters: RF000 -> int_col
|
||||
row-size=4B cardinality=11.00K
|
||||
====
|
||||
# IMPALA-1270: aggregation will not be added on top of existing aggregation if it does
|
||||
# not reduce cardinality enough.
|
||||
select count(*) from functional.alltypesagg t1
|
||||
where int_col in (
|
||||
select int_col from functional.alltypes t2
|
||||
group by int_col, tinyint_col
|
||||
having sum(int_col) > 1)
|
||||
---- PLAN
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
04:AGGREGATE [FINALIZE]
|
||||
| output: count(*)
|
||||
| row-size=8B cardinality=1
|
||||
|
|
||||
03:HASH JOIN [LEFT SEMI JOIN]
|
||||
| hash predicates: int_col = int_col
|
||||
| runtime filters: RF000 <- int_col
|
||||
| row-size=4B cardinality=115
|
||||
|
|
||||
|--02:AGGREGATE [FINALIZE]
|
||||
| | output: sum(int_col)
|
||||
| | group by: int_col, tinyint_col
|
||||
| | having: sum(int_col) > 1
|
||||
| | row-size=13B cardinality=10
|
||||
| |
|
||||
| 01:SCAN HDFS [functional.alltypes t2]
|
||||
| HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| row-size=5B cardinality=7.30K
|
||||
|
|
||||
00:SCAN HDFS [functional.alltypesagg t1]
|
||||
HDFS partitions=11/11 files=11 size=814.73KB
|
||||
runtime filters: RF000 -> int_col
|
||||
row-size=4B cardinality=11.00K
|
||||
---- PARALLELPLANS
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
09:AGGREGATE [FINALIZE]
|
||||
| output: count:merge(*)
|
||||
| row-size=8B cardinality=1
|
||||
|
|
||||
08:EXCHANGE [UNPARTITIONED]
|
||||
|
|
||||
04:AGGREGATE
|
||||
| output: count(*)
|
||||
| row-size=8B cardinality=1
|
||||
|
|
||||
03:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
|
||||
| hash predicates: int_col = int_col
|
||||
| row-size=4B cardinality=115
|
||||
|
|
||||
|--JOIN BUILD
|
||||
| | join-table-id=00 plan-id=01 cohort-id=01
|
||||
| | build expressions: int_col
|
||||
| | runtime filters: RF000 <- int_col
|
||||
| |
|
||||
| 07:EXCHANGE [BROADCAST]
|
||||
| |
|
||||
| 06:AGGREGATE [FINALIZE]
|
||||
| | output: sum:merge(int_col)
|
||||
| | group by: int_col, tinyint_col
|
||||
| | having: sum(int_col) > 1
|
||||
| | row-size=13B cardinality=10
|
||||
| |
|
||||
| 05:EXCHANGE [HASH(int_col,tinyint_col)]
|
||||
| |
|
||||
| 02:AGGREGATE [STREAMING]
|
||||
| | output: sum(int_col)
|
||||
| | group by: int_col, tinyint_col
|
||||
| | row-size=13B cardinality=100
|
||||
| |
|
||||
| 01:SCAN HDFS [functional.alltypes t2]
|
||||
| HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| row-size=5B cardinality=7.30K
|
||||
|
|
||||
00:SCAN HDFS [functional.alltypesagg t1]
|
||||
HDFS partitions=11/11 files=11 size=814.73KB
|
||||
runtime filters: RF000 -> int_col
|
||||
row-size=4B cardinality=11.00K
|
||||
====
|
||||
# IMPALA-1270: planner is not able to coalesce redundant aggregations yet.
|
||||
# The left input of the SEMI JOIN could be more efficiently executed with
|
||||
# a single aggregation by int_col, but the bottom-up plan generation process
|
||||
# first generates an aggregation by int_col, tinyint_col and the distinct
|
||||
# aggregation is placed on top of that.
|
||||
select count(*) from functional.alltypesagg t1
|
||||
where int_col in (
|
||||
select int_col from functional.alltypes t2
|
||||
group by int_col, tinyint_col)
|
||||
---- PLAN
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
05:AGGREGATE [FINALIZE]
|
||||
| output: count(*)
|
||||
| row-size=8B cardinality=1
|
||||
|
|
||||
04:HASH JOIN [LEFT SEMI JOIN]
|
||||
| hash predicates: int_col = int_col
|
||||
| runtime filters: RF000 <- int_col
|
||||
| row-size=4B cardinality=115
|
||||
|
|
||||
|--03:AGGREGATE [FINALIZE]
|
||||
| | group by: int_col
|
||||
| | row-size=4B cardinality=10
|
||||
| |
|
||||
| 02:AGGREGATE [FINALIZE]
|
||||
| | group by: int_col, tinyint_col
|
||||
| | row-size=5B cardinality=100
|
||||
| |
|
||||
| 01:SCAN HDFS [functional.alltypes t2]
|
||||
| HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| row-size=5B cardinality=7.30K
|
||||
|
|
||||
00:SCAN HDFS [functional.alltypesagg t1]
|
||||
HDFS partitions=11/11 files=11 size=814.73KB
|
||||
runtime filters: RF000 -> int_col
|
||||
row-size=4B cardinality=11.00K
|
||||
---- PARALLELPLANS
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
12:AGGREGATE [FINALIZE]
|
||||
| output: count:merge(*)
|
||||
| row-size=8B cardinality=1
|
||||
|
|
||||
11:EXCHANGE [UNPARTITIONED]
|
||||
|
|
||||
05:AGGREGATE
|
||||
| output: count(*)
|
||||
| row-size=8B cardinality=1
|
||||
|
|
||||
04:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
|
||||
| hash predicates: int_col = int_col
|
||||
| row-size=4B cardinality=115
|
||||
|
|
||||
|--JOIN BUILD
|
||||
| | join-table-id=00 plan-id=01 cohort-id=01
|
||||
| | build expressions: int_col
|
||||
| | runtime filters: RF000 <- int_col
|
||||
| |
|
||||
| 10:EXCHANGE [BROADCAST]
|
||||
| |
|
||||
| 09:AGGREGATE [FINALIZE]
|
||||
| | group by: int_col
|
||||
| | row-size=4B cardinality=10
|
||||
| |
|
||||
| 08:EXCHANGE [HASH(int_col)]
|
||||
| |
|
||||
| 03:AGGREGATE [STREAMING]
|
||||
| | group by: int_col
|
||||
| | row-size=4B cardinality=10
|
||||
| |
|
||||
| 07:AGGREGATE [FINALIZE]
|
||||
| | group by: int_col, tinyint_col
|
||||
| | row-size=5B cardinality=100
|
||||
| |
|
||||
| 06:EXCHANGE [HASH(int_col,tinyint_col)]
|
||||
| |
|
||||
| 02:AGGREGATE [STREAMING]
|
||||
| | group by: int_col, tinyint_col
|
||||
| | row-size=5B cardinality=100
|
||||
| |
|
||||
| 01:SCAN HDFS [functional.alltypes t2]
|
||||
| HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| row-size=5B cardinality=7.30K
|
||||
|
|
||||
00:SCAN HDFS [functional.alltypesagg t1]
|
||||
HDFS partitions=11/11 files=11 size=814.73KB
|
||||
runtime filters: RF000 -> int_col
|
||||
row-size=4B cardinality=11.00K
|
||||
====
|
||||
# IMPALA-1270: aggregate function in select list of subquery is eligible for
|
||||
# distinct subquery optimization.
|
||||
select id from functional.alltypesagg t1
|
||||
where int_col in (
|
||||
select count(*)
|
||||
from functional.alltypes t2
|
||||
group by int_col, tinyint_col)
|
||||
---- PLAN
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
04:HASH JOIN [LEFT SEMI JOIN]
|
||||
| hash predicates: int_col = count(*)
|
||||
| runtime filters: RF000 <- count(*)
|
||||
| row-size=8B cardinality=11
|
||||
|
|
||||
|--03:AGGREGATE [FINALIZE]
|
||||
| | group by: count(*)
|
||||
| | row-size=8B cardinality=1
|
||||
| |
|
||||
| 02:AGGREGATE [FINALIZE]
|
||||
| | output: count(*)
|
||||
| | group by: int_col, tinyint_col
|
||||
| | row-size=13B cardinality=100
|
||||
| |
|
||||
| 01:SCAN HDFS [functional.alltypes t2]
|
||||
| HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| row-size=5B cardinality=7.30K
|
||||
|
|
||||
00:SCAN HDFS [functional.alltypesagg t1]
|
||||
HDFS partitions=11/11 files=11 size=814.73KB
|
||||
runtime filters: RF000 -> int_col
|
||||
row-size=8B cardinality=11.00K
|
||||
---- PARALLELPLANS
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
10:EXCHANGE [UNPARTITIONED]
|
||||
|
|
||||
04:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
|
||||
| hash predicates: int_col = count(*)
|
||||
| row-size=8B cardinality=11
|
||||
|
|
||||
|--JOIN BUILD
|
||||
| | join-table-id=00 plan-id=01 cohort-id=01
|
||||
| | build expressions: count(*)
|
||||
| | runtime filters: RF000 <- count(*)
|
||||
| |
|
||||
| 09:EXCHANGE [BROADCAST]
|
||||
| |
|
||||
| 08:AGGREGATE [FINALIZE]
|
||||
| | group by: count(*)
|
||||
| | row-size=8B cardinality=1
|
||||
| |
|
||||
| 07:EXCHANGE [HASH(count(*))]
|
||||
| |
|
||||
| 03:AGGREGATE [STREAMING]
|
||||
| | group by: count(*)
|
||||
| | row-size=8B cardinality=1
|
||||
| |
|
||||
| 06:AGGREGATE [FINALIZE]
|
||||
| | output: count:merge(*)
|
||||
| | group by: int_col, tinyint_col
|
||||
| | row-size=13B cardinality=100
|
||||
| |
|
||||
| 05:EXCHANGE [HASH(int_col,tinyint_col)]
|
||||
| |
|
||||
| 02:AGGREGATE [STREAMING]
|
||||
| | output: count(*)
|
||||
| | group by: int_col, tinyint_col
|
||||
| | row-size=13B cardinality=100
|
||||
| |
|
||||
| 01:SCAN HDFS [functional.alltypes t2]
|
||||
| HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| row-size=5B cardinality=7.30K
|
||||
|
|
||||
00:SCAN HDFS [functional.alltypesagg t1]
|
||||
HDFS partitions=11/11 files=11 size=814.73KB
|
||||
runtime filters: RF000 -> int_col
|
||||
row-size=8B cardinality=11.00K
|
||||
====
|
||||
# IMPALA-1270: analytic function in select list of subquery is eligible for
|
||||
# distinct subquery optimization.
|
||||
select id from functional.alltypesagg t1
|
||||
where int_col in (
|
||||
select rank() over (partition by int_col order by id)
|
||||
from functional.alltypes t2)
|
||||
---- PLAN
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
05:HASH JOIN [LEFT SEMI JOIN]
|
||||
| hash predicates: int_col = rank()
|
||||
| runtime filters: RF000 <- rank()
|
||||
| row-size=8B cardinality=11
|
||||
|
|
||||
|--04:AGGREGATE [FINALIZE]
|
||||
| | group by: rank()
|
||||
| | row-size=8B cardinality=1
|
||||
| |
|
||||
| 03:ANALYTIC
|
||||
| | functions: rank()
|
||||
| | partition by: int_col
|
||||
| | order by: id ASC
|
||||
| | window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
|
||||
| | row-size=16B cardinality=7.30K
|
||||
| |
|
||||
| 02:SORT
|
||||
| | order by: int_col ASC NULLS FIRST, id ASC
|
||||
| | row-size=8B cardinality=7.30K
|
||||
| |
|
||||
| 01:SCAN HDFS [functional.alltypes t2]
|
||||
| HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| row-size=8B cardinality=7.30K
|
||||
|
|
||||
00:SCAN HDFS [functional.alltypesagg t1]
|
||||
HDFS partitions=11/11 files=11 size=814.73KB
|
||||
runtime filters: RF000 -> int_col
|
||||
row-size=8B cardinality=11.00K
|
||||
---- PARALLELPLANS
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
10:EXCHANGE [UNPARTITIONED]
|
||||
|
|
||||
05:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
|
||||
| hash predicates: int_col = rank()
|
||||
| row-size=8B cardinality=11
|
||||
|
|
||||
|--JOIN BUILD
|
||||
| | join-table-id=00 plan-id=01 cohort-id=01
|
||||
| | build expressions: rank()
|
||||
| | runtime filters: RF000 <- rank()
|
||||
| |
|
||||
| 09:EXCHANGE [BROADCAST]
|
||||
| |
|
||||
| 08:AGGREGATE [FINALIZE]
|
||||
| | group by: rank()
|
||||
| | row-size=8B cardinality=1
|
||||
| |
|
||||
| 07:EXCHANGE [HASH(rank())]
|
||||
| |
|
||||
| 04:AGGREGATE [STREAMING]
|
||||
| | group by: rank()
|
||||
| | row-size=8B cardinality=1
|
||||
| |
|
||||
| 03:ANALYTIC
|
||||
| | functions: rank()
|
||||
| | partition by: int_col
|
||||
| | order by: id ASC
|
||||
| | window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
|
||||
| | row-size=16B cardinality=7.30K
|
||||
| |
|
||||
| 02:SORT
|
||||
| | order by: int_col ASC NULLS FIRST, id ASC
|
||||
| | row-size=8B cardinality=7.30K
|
||||
| |
|
||||
| 06:EXCHANGE [HASH(int_col)]
|
||||
| |
|
||||
| 01:SCAN HDFS [functional.alltypes t2]
|
||||
| HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| row-size=8B cardinality=7.30K
|
||||
|
|
||||
00:SCAN HDFS [functional.alltypesagg t1]
|
||||
HDFS partitions=11/11 files=11 size=814.73KB
|
||||
runtime filters: RF000 -> int_col
|
||||
row-size=8B cardinality=11.00K
|
||||
====
|
||||
@@ -13,19 +13,21 @@ PLAN-ROOT SINK
|
||||
| row-size=89B cardinality=7.30K
|
||||
|
|
||||
|--03:NESTED LOOP JOIN [RIGHT SEMI JOIN]
|
||||
| | limit: 1
|
||||
| | row-size=4B cardinality=1
|
||||
| |
|
||||
| |--01:SCAN HDFS [functional.tinyinttable]
|
||||
| | partitions=1/1 files=1 size=20B
|
||||
| | HDFS partitions=1/1 files=1 size=20B
|
||||
| | predicates: 1 = functional.tinyinttable.int_col
|
||||
| | row-size=4B cardinality=1
|
||||
| |
|
||||
| 02:SCAN HDFS [functional.alltypestiny]
|
||||
| partitions=4/4 files=4 size=460B
|
||||
| HDFS partitions=4/4 files=4 size=460B
|
||||
| predicates: 1 = functional.alltypestiny.int_col
|
||||
| limit: 1
|
||||
| row-size=4B cardinality=4
|
||||
|
|
||||
00:SCAN HDFS [functional.alltypes t]
|
||||
partitions=24/24 files=24 size=478.45KB
|
||||
HDFS partitions=24/24 files=24 size=478.45KB
|
||||
row-size=89B cardinality=7.30K
|
||||
====
|
||||
|
||||
@@ -126,12 +126,16 @@ where t1.int_col in
|
||||
---- PLAN
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
02:HASH JOIN [LEFT SEMI JOIN]
|
||||
03:HASH JOIN [LEFT SEMI JOIN]
|
||||
| hash predicates: t1.int_col = t2.int_col
|
||||
| runtime filters: RF000 <- t2.int_col
|
||||
| row-size=89B cardinality=7.30K
|
||||
|
|
||||
|--01:SCAN HDFS [functional.alltypes t2]
|
||||
|--02:AGGREGATE [FINALIZE]
|
||||
| | group by: t2.int_col
|
||||
| | row-size=4B cardinality=10
|
||||
| |
|
||||
| 01:SCAN HDFS [functional.alltypes t2]
|
||||
| HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| predicates: (t2.int_col IS NOT NULL AND (t2.int_col < 0 OR t2.int_col > 10) OR t2.bigint_col IS NOT NULL AND (t2.bigint_col < 0 OR t2.bigint_col > 10))
|
||||
| row-size=12B cardinality=730
|
||||
@@ -173,11 +177,15 @@ and t.bigint_col < 1000
|
||||
---- PLAN
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
04:HASH JOIN [NULL AWARE LEFT ANTI JOIN]
|
||||
| hash predicates: t.tinyint_col = tinyint_col
|
||||
05:HASH JOIN [NULL AWARE LEFT ANTI JOIN]
|
||||
| hash predicates: t.tinyint_col = functional.alltypestiny.tinyint_col
|
||||
| row-size=89B cardinality=730
|
||||
|
|
||||
|--02:SCAN HDFS [functional.alltypestiny]
|
||||
|--04:AGGREGATE [FINALIZE]
|
||||
| | group by: functional.alltypestiny.tinyint_col
|
||||
| | row-size=1B cardinality=2
|
||||
| |
|
||||
| 02:SCAN HDFS [functional.alltypestiny]
|
||||
| HDFS partitions=4/4 files=4 size=460B
|
||||
| row-size=1B cardinality=8
|
||||
|
|
||||
@@ -360,20 +368,24 @@ where a.int_col in
|
||||
---- PLAN
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
08:AGGREGATE [FINALIZE]
|
||||
09:AGGREGATE [FINALIZE]
|
||||
| output: count(id)
|
||||
| row-size=8B cardinality=1
|
||||
|
|
||||
07:AGGREGATE
|
||||
08:AGGREGATE
|
||||
| group by: id
|
||||
| row-size=4B cardinality=115
|
||||
|
|
||||
06:HASH JOIN [LEFT SEMI JOIN]
|
||||
07:HASH JOIN [LEFT SEMI JOIN]
|
||||
| hash predicates: a.int_col = t.int_col
|
||||
| runtime filters: RF000 <- t.int_col
|
||||
| row-size=8B cardinality=115
|
||||
|
|
||||
|--05:HASH JOIN [INNER JOIN]
|
||||
|--06:AGGREGATE [FINALIZE]
|
||||
| | group by: t.int_col
|
||||
| | row-size=4B cardinality=10
|
||||
| |
|
||||
| 05:HASH JOIN [INNER JOIN]
|
||||
| | hash predicates: s.bigint_col = n.bigint_col
|
||||
| | runtime filters: RF002 <- n.bigint_col
|
||||
| | row-size=29B cardinality=40
|
||||
@@ -476,12 +488,16 @@ and s.bool_col = false
|
||||
---- PLAN
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
08:HASH JOIN [LEFT SEMI JOIN]
|
||||
| hash predicates: a.int_col = int_col
|
||||
| runtime filters: RF000 <- int_col
|
||||
09:HASH JOIN [LEFT SEMI JOIN]
|
||||
| hash predicates: a.int_col = functional.alltypessmall.int_col
|
||||
| runtime filters: RF000 <- functional.alltypessmall.int_col
|
||||
| row-size=109B cardinality=91
|
||||
|
|
||||
|--06:SCAN HDFS [functional.alltypessmall]
|
||||
|--08:AGGREGATE [FINALIZE]
|
||||
| | group by: functional.alltypessmall.int_col
|
||||
| | row-size=4B cardinality=10
|
||||
| |
|
||||
| 06:SCAN HDFS [functional.alltypessmall]
|
||||
| HDFS partitions=4/4 files=4 size=6.32KB
|
||||
| row-size=4B cardinality=100
|
||||
|
|
||||
@@ -554,17 +570,21 @@ and bigint_col < 1000
|
||||
---- PLAN
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
04:HASH JOIN [LEFT SEMI JOIN]
|
||||
05:HASH JOIN [LEFT SEMI JOIN]
|
||||
| hash predicates: id = id
|
||||
| runtime filters: RF000 <- id
|
||||
| row-size=89B cardinality=11
|
||||
|
|
||||
|--03:HASH JOIN [LEFT SEMI JOIN]
|
||||
| | hash predicates: int_col = int_col
|
||||
| | runtime filters: RF002 <- int_col
|
||||
|--04:HASH JOIN [LEFT SEMI JOIN]
|
||||
| | hash predicates: int_col = functional.alltypestiny.int_col
|
||||
| | runtime filters: RF002 <- functional.alltypestiny.int_col
|
||||
| | row-size=9B cardinality=11
|
||||
| |
|
||||
| |--02:SCAN HDFS [functional.alltypestiny]
|
||||
| |--03:AGGREGATE [FINALIZE]
|
||||
| | | group by: functional.alltypestiny.int_col
|
||||
| | | row-size=4B cardinality=2
|
||||
| | |
|
||||
| | 02:SCAN HDFS [functional.alltypestiny]
|
||||
| | HDFS partitions=4/4 files=4 size=460B
|
||||
| | row-size=4B cardinality=8
|
||||
| |
|
||||
@@ -744,12 +764,16 @@ and tinyint_col < 10
|
||||
---- PLAN
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
03:HASH JOIN [LEFT SEMI JOIN]
|
||||
04:HASH JOIN [LEFT SEMI JOIN]
|
||||
| hash predicates: a.tinyint_col = b.tinyint_col
|
||||
| runtime filters: RF000 <- b.tinyint_col
|
||||
| row-size=1B cardinality=244
|
||||
|
|
||||
|--02:AGGREGATE [FINALIZE]
|
||||
|--03:AGGREGATE [FINALIZE]
|
||||
| | group by: b.tinyint_col
|
||||
| | row-size=1B cardinality=2
|
||||
| |
|
||||
| 02:AGGREGATE [FINALIZE]
|
||||
| | group by: id, int_col, bool_col, b.tinyint_col
|
||||
| | row-size=10B cardinality=8
|
||||
| |
|
||||
@@ -771,21 +795,25 @@ where not exists
|
||||
---- PLAN
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
03:AGGREGATE [FINALIZE]
|
||||
04:AGGREGATE [FINALIZE]
|
||||
| output: count(*)
|
||||
| row-size=8B cardinality=1
|
||||
|
|
||||
02:HASH JOIN [RIGHT ANTI JOIN]
|
||||
| hash predicates: a.int_col = t.int_col
|
||||
03:HASH JOIN [LEFT ANTI JOIN]
|
||||
| hash predicates: t.int_col = a.int_col
|
||||
| row-size=4B cardinality=7.30K
|
||||
|
|
||||
|--00:SCAN HDFS [functional.alltypes t]
|
||||
| HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| row-size=4B cardinality=7.30K
|
||||
|--02:AGGREGATE [FINALIZE]
|
||||
| | group by: a.int_col
|
||||
| | row-size=4B cardinality=957
|
||||
| |
|
||||
| 01:SCAN HDFS [functional.alltypesagg a]
|
||||
| HDFS partitions=11/11 files=11 size=814.73KB
|
||||
| row-size=4B cardinality=11.00K
|
||||
|
|
||||
01:SCAN HDFS [functional.alltypesagg a]
|
||||
HDFS partitions=11/11 files=11 size=814.73KB
|
||||
row-size=4B cardinality=11.00K
|
||||
00:SCAN HDFS [functional.alltypes t]
|
||||
HDFS partitions=24/24 files=24 size=478.45KB
|
||||
row-size=4B cardinality=7.30K
|
||||
====
|
||||
# Correlated NOT EXISTS with an analytic function and a group by clause
|
||||
select count(*)
|
||||
@@ -799,15 +827,19 @@ and bool_col = false
|
||||
---- PLAN
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
06:AGGREGATE [FINALIZE]
|
||||
07:AGGREGATE [FINALIZE]
|
||||
| output: count(*)
|
||||
| row-size=8B cardinality=1
|
||||
|
|
||||
05:HASH JOIN [LEFT ANTI JOIN]
|
||||
06:HASH JOIN [LEFT ANTI JOIN]
|
||||
| hash predicates: a.int_col = b.int_col
|
||||
| row-size=5B cardinality=5.50K
|
||||
|
|
||||
|--04:AGGREGATE [FINALIZE]
|
||||
|--05:AGGREGATE [FINALIZE]
|
||||
| | group by: b.int_col
|
||||
| | row-size=4B cardinality=10
|
||||
| |
|
||||
| 04:AGGREGATE [FINALIZE]
|
||||
| | group by: b.id, b.int_col, b.bigint_col
|
||||
| | row-size=16B cardinality=50
|
||||
| |
|
||||
@@ -978,16 +1010,16 @@ where exists
|
||||
---- PLAN
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
05:AGGREGATE [FINALIZE]
|
||||
06:AGGREGATE [FINALIZE]
|
||||
| output: count(*)
|
||||
| row-size=8B cardinality=1
|
||||
|
|
||||
04:HASH JOIN [LEFT SEMI JOIN]
|
||||
05:HASH JOIN [LEFT SEMI JOIN]
|
||||
| hash predicates: a.id = t.id
|
||||
| runtime filters: RF000 <- t.id
|
||||
| row-size=4B cardinality=8
|
||||
|
|
||||
|--03:HASH JOIN [RIGHT SEMI JOIN]
|
||||
|--04:HASH JOIN [RIGHT SEMI JOIN]
|
||||
| | hash predicates: g.int_col = t.int_col
|
||||
| | runtime filters: RF002 <- t.int_col
|
||||
| | row-size=8B cardinality=8
|
||||
@@ -996,6 +1028,10 @@ PLAN-ROOT SINK
|
||||
| | HDFS partitions=4/4 files=4 size=460B
|
||||
| | row-size=8B cardinality=8
|
||||
| |
|
||||
| 03:AGGREGATE [FINALIZE]
|
||||
| | group by: g.int_col
|
||||
| | row-size=4B cardinality=957
|
||||
| |
|
||||
| 02:SCAN HDFS [functional.alltypesagg g]
|
||||
| HDFS partitions=11/11 files=11 size=814.73KB
|
||||
| predicates: g.bool_col = FALSE
|
||||
@@ -2489,6 +2525,7 @@ PLAN-ROOT SINK
|
||||
|--01:SCAN HDFS [functional.alltypestiny]
|
||||
| HDFS partitions=4/4 files=4 size=460B
|
||||
| predicates: 1 = functional.alltypestiny.int_col
|
||||
| limit: 1
|
||||
| row-size=4B cardinality=4
|
||||
|
|
||||
00:SCAN HDFS [functional.alltypessmall]
|
||||
@@ -2546,6 +2583,7 @@ PLAN-ROOT SINK
|
||||
|
|
||||
|--02:AGGREGATE [FINALIZE]
|
||||
| | group by: int_col
|
||||
| | limit: 1
|
||||
| | row-size=4B cardinality=2
|
||||
| |
|
||||
| 01:SCAN HDFS [functional.alltypestiny]
|
||||
@@ -2592,6 +2630,7 @@ PLAN-ROOT SINK
|
||||
|--02:AGGREGATE [FINALIZE]
|
||||
| | output: max(int_col)
|
||||
| | having: 1 = max(int_col)
|
||||
| | limit: 1
|
||||
| | row-size=4B cardinality=1
|
||||
| |
|
||||
| 01:SCAN HDFS [functional.alltypestiny]
|
||||
@@ -2635,6 +2674,7 @@ PLAN-ROOT SINK
|
||||
|
|
||||
|--02:SELECT
|
||||
| | predicates: 1 = int_col
|
||||
| | limit: 1
|
||||
| | row-size=4B cardinality=1
|
||||
| |
|
||||
| 01:SCAN HDFS [functional.alltypestiny]
|
||||
@@ -2679,11 +2719,13 @@ PLAN-ROOT SINK
|
||||
| row-size=89B cardinality=7.30K
|
||||
|
|
||||
|--03:NESTED LOOP JOIN [LEFT SEMI JOIN]
|
||||
| | limit: 1
|
||||
| | row-size=4B cardinality=unavailable
|
||||
| |
|
||||
| |--02:SCAN HDFS [functional.alltypestiny]
|
||||
| | HDFS partitions=4/4 files=4 size=460B
|
||||
| | predicates: 1 = functional.alltypestiny.int_col
|
||||
| | limit: 1
|
||||
| | row-size=4B cardinality=4
|
||||
| |
|
||||
| 01:SCAN HDFS [functional.tinyinttable]
|
||||
@@ -2708,6 +2750,7 @@ PLAN-ROOT SINK
|
||||
|--03:HASH JOIN [LEFT SEMI JOIN]
|
||||
| | hash predicates: bigint_col = bigint_col, t.id = id
|
||||
| | runtime filters: RF000 <- bigint_col, RF001 <- id
|
||||
| | limit: 1
|
||||
| | row-size=16B cardinality=2
|
||||
| |
|
||||
| |--02:SCAN HDFS [functional.alltypestiny]
|
||||
@@ -4688,7 +4731,7 @@ and tinyint_col < 10
|
||||
---- PLAN
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
04:HASH JOIN [RIGHT SEMI JOIN]
|
||||
05:HASH JOIN [RIGHT SEMI JOIN]
|
||||
| hash predicates: CASE valid_tid(2,3,4,5) WHEN 2 THEN b.tinyint_col WHEN 3 THEN b.tinyint_col WHEN 4 THEN b.tinyint_col WHEN 5 THEN b.tinyint_col END = a.tinyint_col, CASE valid_tid(2,3,4,5) WHEN 2 THEN b.string_col WHEN 3 THEN b.string_col WHEN 4 THEN b.string_col WHEN 5 THEN b.string_col END = a.string_col
|
||||
| row-size=14B cardinality=1
|
||||
|
|
||||
@@ -4697,6 +4740,10 @@ PLAN-ROOT SINK
|
||||
| predicates: tinyint_col < 10
|
||||
| row-size=14B cardinality=1
|
||||
|
|
||||
04:AGGREGATE [FINALIZE]
|
||||
| group by: CASE valid_tid(2,3,4,5) WHEN 2 THEN b.tinyint_col WHEN 3 THEN b.tinyint_col WHEN 4 THEN b.tinyint_col WHEN 5 THEN b.tinyint_col END, CASE valid_tid(2,3,4,5) WHEN 2 THEN b.string_col WHEN 3 THEN b.string_col WHEN 4 THEN b.string_col WHEN 5 THEN b.string_col END
|
||||
| row-size=13B cardinality=9.64K
|
||||
|
|
||||
03:AGGREGATE [FINALIZE]
|
||||
| group by: CASE valid_tid(2,3,4,5) WHEN 2 THEN id WHEN 3 THEN id WHEN 4 THEN id WHEN 5 THEN NULL END, CASE valid_tid(2,3,4,5) WHEN 2 THEN int_col WHEN 3 THEN int_col WHEN 4 THEN NULL WHEN 5 THEN NULL END, CASE valid_tid(2,3,4,5) WHEN 2 THEN bool_col WHEN 3 THEN NULL WHEN 4 THEN NULL WHEN 5 THEN NULL END, CASE valid_tid(2,3,4,5) WHEN 2 THEN b.tinyint_col WHEN 3 THEN b.tinyint_col WHEN 4 THEN b.tinyint_col WHEN 5 THEN b.tinyint_col END, CASE valid_tid(2,3,4,5) WHEN 2 THEN b.string_col WHEN 3 THEN b.string_col WHEN 4 THEN b.string_col WHEN 5 THEN b.string_col END, CASE valid_tid(2,3,4,5) WHEN 2 THEN 2 WHEN 3 THEN 3 WHEN 4 THEN 4 WHEN 5 THEN 5 END
|
||||
| row-size=26B cardinality=41.67K
|
||||
|
||||
File diff suppressed because it is too large
Load Diff
@@ -4227,12 +4227,12 @@ Max Per-Host Resource Reservation: Memory=71.75MB Threads=6
|
||||
Per-Host Resource Estimates: Memory=612MB
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
10:SORT
|
||||
11:SORT
|
||||
| order by: s_name ASC
|
||||
| row-size=67B cardinality=400
|
||||
|
|
||||
09:HASH JOIN [RIGHT SEMI JOIN]
|
||||
| hash predicates: ps_suppkey = s_suppkey
|
||||
10:HASH JOIN [RIGHT SEMI JOIN]
|
||||
| hash predicates: tpch.partsupp.ps_suppkey = s_suppkey
|
||||
| runtime filters: RF000 <- s_suppkey
|
||||
| row-size=98B cardinality=400
|
||||
|
|
||||
@@ -4251,6 +4251,10 @@ PLAN-ROOT SINK
|
||||
| runtime filters: RF008 -> s_nationkey
|
||||
| row-size=77B cardinality=10.00K
|
||||
|
|
||||
09:AGGREGATE [FINALIZE]
|
||||
| group by: tpch.partsupp.ps_suppkey
|
||||
| row-size=8B cardinality=9.71K
|
||||
|
|
||||
07:HASH JOIN [RIGHT SEMI JOIN]
|
||||
| hash predicates: l_partkey = ps_partkey, l_suppkey = ps_suppkey
|
||||
| other join predicates: ps_availqty > 0.5 * sum(l_quantity)
|
||||
@@ -4269,7 +4273,7 @@ PLAN-ROOT SINK
|
||||
| |
|
||||
| 02:SCAN HDFS [tpch.partsupp]
|
||||
| HDFS partitions=1/1 files=1 size=112.71MB
|
||||
| runtime filters: RF000 -> ps_suppkey, RF006 -> ps_partkey
|
||||
| runtime filters: RF000 -> tpch.partsupp.ps_suppkey, RF006 -> ps_partkey
|
||||
| row-size=20B cardinality=800.00K
|
||||
|
|
||||
05:AGGREGATE [FINALIZE]
|
||||
@@ -4283,30 +4287,30 @@ PLAN-ROOT SINK
|
||||
runtime filters: RF000 -> tpch.lineitem.l_suppkey, RF002 -> tpch.lineitem.l_partkey, RF003 -> tpch.lineitem.l_suppkey
|
||||
row-size=46B cardinality=600.12K
|
||||
---- DISTRIBUTEDPLAN
|
||||
Max Per-Host Resource Reservation: Memory=107.63MB Threads=13
|
||||
Per-Host Resource Estimates: Memory=660MB
|
||||
Max Per-Host Resource Reservation: Memory=111.57MB Threads=13
|
||||
Per-Host Resource Estimates: Memory=679MB
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
18:MERGING-EXCHANGE [UNPARTITIONED]
|
||||
20:MERGING-EXCHANGE [UNPARTITIONED]
|
||||
| order by: s_name ASC
|
||||
|
|
||||
10:SORT
|
||||
11:SORT
|
||||
| order by: s_name ASC
|
||||
| row-size=67B cardinality=400
|
||||
|
|
||||
09:HASH JOIN [RIGHT SEMI JOIN, PARTITIONED]
|
||||
| hash predicates: ps_suppkey = s_suppkey
|
||||
10:HASH JOIN [RIGHT SEMI JOIN, PARTITIONED]
|
||||
| hash predicates: tpch.partsupp.ps_suppkey = s_suppkey
|
||||
| runtime filters: RF000 <- s_suppkey
|
||||
| row-size=98B cardinality=400
|
||||
|
|
||||
|--17:EXCHANGE [HASH(s_suppkey)]
|
||||
|--19:EXCHANGE [HASH(s_suppkey)]
|
||||
| |
|
||||
| 08:HASH JOIN [INNER JOIN, BROADCAST]
|
||||
| | hash predicates: s_nationkey = n_nationkey
|
||||
| | runtime filters: RF008 <- n_nationkey
|
||||
| | row-size=98B cardinality=400
|
||||
| |
|
||||
| |--15:EXCHANGE [BROADCAST]
|
||||
| |--18:EXCHANGE [BROADCAST]
|
||||
| | |
|
||||
| | 01:SCAN HDFS [tpch.nation]
|
||||
| | HDFS partitions=1/1 files=1 size=2.15KB
|
||||
@@ -4318,7 +4322,15 @@ PLAN-ROOT SINK
|
||||
| runtime filters: RF008 -> s_nationkey
|
||||
| row-size=77B cardinality=10.00K
|
||||
|
|
||||
16:EXCHANGE [HASH(ps_suppkey)]
|
||||
17:AGGREGATE [FINALIZE]
|
||||
| group by: tpch.partsupp.ps_suppkey
|
||||
| row-size=8B cardinality=9.71K
|
||||
|
|
||||
16:EXCHANGE [HASH(tpch.partsupp.ps_suppkey)]
|
||||
|
|
||||
09:AGGREGATE [STREAMING]
|
||||
| group by: tpch.partsupp.ps_suppkey
|
||||
| row-size=8B cardinality=9.71K
|
||||
|
|
||||
07:HASH JOIN [RIGHT SEMI JOIN, PARTITIONED]
|
||||
| hash predicates: l_partkey = ps_partkey, l_suppkey = ps_suppkey
|
||||
@@ -4326,14 +4338,14 @@ PLAN-ROOT SINK
|
||||
| runtime filters: RF002 <- ps_partkey, RF003 <- ps_suppkey
|
||||
| row-size=20B cardinality=79.79K
|
||||
|
|
||||
|--14:EXCHANGE [HASH(ps_partkey,ps_suppkey)]
|
||||
|--15:EXCHANGE [HASH(ps_partkey,ps_suppkey)]
|
||||
| |
|
||||
| 06:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
|
||||
| | hash predicates: ps_partkey = p_partkey
|
||||
| | runtime filters: RF006 <- p_partkey
|
||||
| | row-size=20B cardinality=79.79K
|
||||
| |
|
||||
| |--13:EXCHANGE [BROADCAST]
|
||||
| |--14:EXCHANGE [BROADCAST]
|
||||
| | |
|
||||
| | 03:SCAN HDFS [tpch.part]
|
||||
| | HDFS partitions=1/1 files=1 size=22.83MB
|
||||
@@ -4342,15 +4354,15 @@ PLAN-ROOT SINK
|
||||
| |
|
||||
| 02:SCAN HDFS [tpch.partsupp]
|
||||
| HDFS partitions=1/1 files=1 size=112.71MB
|
||||
| runtime filters: RF000 -> ps_suppkey, RF006 -> ps_partkey
|
||||
| runtime filters: RF000 -> tpch.partsupp.ps_suppkey, RF006 -> ps_partkey
|
||||
| row-size=20B cardinality=800.00K
|
||||
|
|
||||
12:AGGREGATE [FINALIZE]
|
||||
13:AGGREGATE [FINALIZE]
|
||||
| output: sum:merge(l_quantity)
|
||||
| group by: l_partkey, l_suppkey
|
||||
| row-size=32B cardinality=600.12K
|
||||
|
|
||||
11:EXCHANGE [HASH(l_partkey,l_suppkey)]
|
||||
12:EXCHANGE [HASH(l_partkey,l_suppkey)]
|
||||
|
|
||||
05:AGGREGATE [STREAMING]
|
||||
| output: sum(l_quantity)
|
||||
@@ -4363,19 +4375,19 @@ PLAN-ROOT SINK
|
||||
runtime filters: RF000 -> tpch.lineitem.l_suppkey, RF002 -> tpch.lineitem.l_partkey, RF003 -> tpch.lineitem.l_suppkey
|
||||
row-size=46B cardinality=600.12K
|
||||
---- PARALLELPLANS
|
||||
Max Per-Host Resource Reservation: Memory=123.45MB Threads=13
|
||||
Per-Host Resource Estimates: Memory=430MB
|
||||
Max Per-Host Resource Reservation: Memory=127.38MB Threads=13
|
||||
Per-Host Resource Estimates: Memory=449MB
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
18:MERGING-EXCHANGE [UNPARTITIONED]
|
||||
20:MERGING-EXCHANGE [UNPARTITIONED]
|
||||
| order by: s_name ASC
|
||||
|
|
||||
10:SORT
|
||||
11:SORT
|
||||
| order by: s_name ASC
|
||||
| row-size=67B cardinality=400
|
||||
|
|
||||
09:HASH JOIN [RIGHT SEMI JOIN, PARTITIONED]
|
||||
| hash predicates: ps_suppkey = s_suppkey
|
||||
10:HASH JOIN [RIGHT SEMI JOIN, PARTITIONED]
|
||||
| hash predicates: tpch.partsupp.ps_suppkey = s_suppkey
|
||||
| row-size=98B cardinality=400
|
||||
|
|
||||
|--JOIN BUILD
|
||||
@@ -4383,7 +4395,7 @@ PLAN-ROOT SINK
|
||||
| | build expressions: s_suppkey
|
||||
| | runtime filters: RF000 <- s_suppkey
|
||||
| |
|
||||
| 17:EXCHANGE [HASH(s_suppkey)]
|
||||
| 19:EXCHANGE [HASH(s_suppkey)]
|
||||
| |
|
||||
| 08:HASH JOIN [INNER JOIN, BROADCAST]
|
||||
| | hash predicates: s_nationkey = n_nationkey
|
||||
@@ -4394,7 +4406,7 @@ PLAN-ROOT SINK
|
||||
| | | build expressions: n_nationkey
|
||||
| | | runtime filters: RF008 <- n_nationkey
|
||||
| | |
|
||||
| | 15:EXCHANGE [BROADCAST]
|
||||
| | 18:EXCHANGE [BROADCAST]
|
||||
| | |
|
||||
| | 01:SCAN HDFS [tpch.nation]
|
||||
| | HDFS partitions=1/1 files=1 size=2.15KB
|
||||
@@ -4406,7 +4418,15 @@ PLAN-ROOT SINK
|
||||
| runtime filters: RF008 -> s_nationkey
|
||||
| row-size=77B cardinality=10.00K
|
||||
|
|
||||
16:EXCHANGE [HASH(ps_suppkey)]
|
||||
17:AGGREGATE [FINALIZE]
|
||||
| group by: tpch.partsupp.ps_suppkey
|
||||
| row-size=8B cardinality=9.71K
|
||||
|
|
||||
16:EXCHANGE [HASH(tpch.partsupp.ps_suppkey)]
|
||||
|
|
||||
09:AGGREGATE [STREAMING]
|
||||
| group by: tpch.partsupp.ps_suppkey
|
||||
| row-size=8B cardinality=9.71K
|
||||
|
|
||||
07:HASH JOIN [RIGHT SEMI JOIN, PARTITIONED]
|
||||
| hash predicates: l_partkey = ps_partkey, l_suppkey = ps_suppkey
|
||||
@@ -4418,7 +4438,7 @@ PLAN-ROOT SINK
|
||||
| | build expressions: ps_partkey, ps_suppkey
|
||||
| | runtime filters: RF002 <- ps_partkey, RF003 <- ps_suppkey
|
||||
| |
|
||||
| 14:EXCHANGE [HASH(ps_partkey,ps_suppkey)]
|
||||
| 15:EXCHANGE [HASH(ps_partkey,ps_suppkey)]
|
||||
| |
|
||||
| 06:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
|
||||
| | hash predicates: ps_partkey = p_partkey
|
||||
@@ -4429,7 +4449,7 @@ PLAN-ROOT SINK
|
||||
| | | build expressions: p_partkey
|
||||
| | | runtime filters: RF006 <- p_partkey
|
||||
| | |
|
||||
| | 13:EXCHANGE [BROADCAST]
|
||||
| | 14:EXCHANGE [BROADCAST]
|
||||
| | |
|
||||
| | 03:SCAN HDFS [tpch.part]
|
||||
| | HDFS partitions=1/1 files=1 size=22.83MB
|
||||
@@ -4438,15 +4458,15 @@ PLAN-ROOT SINK
|
||||
| |
|
||||
| 02:SCAN HDFS [tpch.partsupp]
|
||||
| HDFS partitions=1/1 files=1 size=112.71MB
|
||||
| runtime filters: RF000 -> ps_suppkey, RF006 -> ps_partkey
|
||||
| runtime filters: RF000 -> tpch.partsupp.ps_suppkey, RF006 -> ps_partkey
|
||||
| row-size=20B cardinality=800.00K
|
||||
|
|
||||
12:AGGREGATE [FINALIZE]
|
||||
13:AGGREGATE [FINALIZE]
|
||||
| output: sum:merge(l_quantity)
|
||||
| group by: l_partkey, l_suppkey
|
||||
| row-size=32B cardinality=600.12K
|
||||
|
|
||||
11:EXCHANGE [HASH(l_partkey,l_suppkey)]
|
||||
12:EXCHANGE [HASH(l_partkey,l_suppkey)]
|
||||
|
|
||||
05:AGGREGATE [STREAMING]
|
||||
| output: sum(l_quantity)
|
||||
@@ -4815,21 +4835,21 @@ group by
|
||||
order by
|
||||
cntrycode
|
||||
---- PLAN
|
||||
Max Per-Host Resource Reservation: Memory=24.00MB Threads=4
|
||||
Per-Host Resource Estimates: Memory=314MB
|
||||
Max Per-Host Resource Reservation: Memory=25.94MB Threads=4
|
||||
Per-Host Resource Estimates: Memory=324MB
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
07:SORT
|
||||
08:SORT
|
||||
| order by: cntrycode ASC
|
||||
| row-size=36B cardinality=15.00K
|
||||
|
|
||||
06:AGGREGATE [FINALIZE]
|
||||
07:AGGREGATE [FINALIZE]
|
||||
| output: count(*), sum(c_acctbal)
|
||||
| group by: substr(c_phone, 1, 2)
|
||||
| row-size=36B cardinality=15.00K
|
||||
|
|
||||
05:HASH JOIN [RIGHT ANTI JOIN]
|
||||
| hash predicates: o_custkey = c_custkey
|
||||
06:HASH JOIN [RIGHT ANTI JOIN]
|
||||
| hash predicates: tpch.orders.o_custkey = c_custkey
|
||||
| row-size=51B cardinality=15.00K
|
||||
|
|
||||
|--04:NESTED LOOP JOIN [INNER JOIN]
|
||||
@@ -4850,50 +4870,54 @@ PLAN-ROOT SINK
|
||||
| predicates: substr(c_phone, 1, 2) IN ('13', '31', '23', '29', '30', '18', '17')
|
||||
| row-size=43B cardinality=15.00K
|
||||
|
|
||||
05:AGGREGATE [FINALIZE]
|
||||
| group by: tpch.orders.o_custkey
|
||||
| row-size=8B cardinality=98.39K
|
||||
|
|
||||
03:SCAN HDFS [tpch.orders]
|
||||
HDFS partitions=1/1 files=1 size=162.56MB
|
||||
row-size=8B cardinality=1.50M
|
||||
---- DISTRIBUTEDPLAN
|
||||
Max Per-Host Resource Reservation: Memory=41.88MB Threads=10
|
||||
Per-Host Resource Estimates: Memory=365MB
|
||||
Max Per-Host Resource Reservation: Memory=45.81MB Threads=10
|
||||
Per-Host Resource Estimates: Memory=380MB
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
15:MERGING-EXCHANGE [UNPARTITIONED]
|
||||
17:MERGING-EXCHANGE [UNPARTITIONED]
|
||||
| order by: cntrycode ASC
|
||||
|
|
||||
07:SORT
|
||||
08:SORT
|
||||
| order by: cntrycode ASC
|
||||
| row-size=36B cardinality=15.00K
|
||||
|
|
||||
14:AGGREGATE [FINALIZE]
|
||||
16:AGGREGATE [FINALIZE]
|
||||
| output: count:merge(*), sum:merge(c_acctbal)
|
||||
| group by: cntrycode
|
||||
| row-size=36B cardinality=15.00K
|
||||
|
|
||||
13:EXCHANGE [HASH(cntrycode)]
|
||||
15:EXCHANGE [HASH(cntrycode)]
|
||||
|
|
||||
06:AGGREGATE [STREAMING]
|
||||
07:AGGREGATE [STREAMING]
|
||||
| output: count(*), sum(c_acctbal)
|
||||
| group by: substr(c_phone, 1, 2)
|
||||
| row-size=36B cardinality=15.00K
|
||||
|
|
||||
05:HASH JOIN [RIGHT ANTI JOIN, PARTITIONED]
|
||||
| hash predicates: o_custkey = c_custkey
|
||||
06:HASH JOIN [RIGHT ANTI JOIN, PARTITIONED]
|
||||
| hash predicates: tpch.orders.o_custkey = c_custkey
|
||||
| row-size=51B cardinality=15.00K
|
||||
|
|
||||
|--12:EXCHANGE [HASH(c_custkey)]
|
||||
|--14:EXCHANGE [HASH(c_custkey)]
|
||||
| |
|
||||
| 04:NESTED LOOP JOIN [INNER JOIN, BROADCAST]
|
||||
| | predicates: c_acctbal > avg(c_acctbal)
|
||||
| | row-size=51B cardinality=15.00K
|
||||
| |
|
||||
| |--10:EXCHANGE [BROADCAST]
|
||||
| |--13:EXCHANGE [BROADCAST]
|
||||
| | |
|
||||
| | 09:AGGREGATE [FINALIZE]
|
||||
| | 12:AGGREGATE [FINALIZE]
|
||||
| | | output: avg:merge(c_acctbal)
|
||||
| | | row-size=8B cardinality=1
|
||||
| | |
|
||||
| | 08:EXCHANGE [UNPARTITIONED]
|
||||
| | 11:EXCHANGE [UNPARTITIONED]
|
||||
| | |
|
||||
| | 02:AGGREGATE
|
||||
| | | output: avg(c_acctbal)
|
||||
@@ -4909,44 +4933,52 @@ PLAN-ROOT SINK
|
||||
| predicates: substr(c_phone, 1, 2) IN ('13', '31', '23', '29', '30', '18', '17')
|
||||
| row-size=43B cardinality=15.00K
|
||||
|
|
||||
11:EXCHANGE [HASH(o_custkey)]
|
||||
10:AGGREGATE [FINALIZE]
|
||||
| group by: tpch.orders.o_custkey
|
||||
| row-size=8B cardinality=98.39K
|
||||
|
|
||||
09:EXCHANGE [HASH(tpch.orders.o_custkey)]
|
||||
|
|
||||
05:AGGREGATE [STREAMING]
|
||||
| group by: tpch.orders.o_custkey
|
||||
| row-size=8B cardinality=98.39K
|
||||
|
|
||||
03:SCAN HDFS [tpch.orders]
|
||||
HDFS partitions=1/1 files=1 size=162.56MB
|
||||
row-size=8B cardinality=1.50M
|
||||
---- PARALLELPLANS
|
||||
Max Per-Host Resource Reservation: Memory=41.88MB Threads=9
|
||||
Per-Host Resource Estimates: Memory=213MB
|
||||
Max Per-Host Resource Reservation: Memory=45.81MB Threads=9
|
||||
Per-Host Resource Estimates: Memory=228MB
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
15:MERGING-EXCHANGE [UNPARTITIONED]
|
||||
17:MERGING-EXCHANGE [UNPARTITIONED]
|
||||
| order by: cntrycode ASC
|
||||
|
|
||||
07:SORT
|
||||
08:SORT
|
||||
| order by: cntrycode ASC
|
||||
| row-size=36B cardinality=15.00K
|
||||
|
|
||||
14:AGGREGATE [FINALIZE]
|
||||
16:AGGREGATE [FINALIZE]
|
||||
| output: count:merge(*), sum:merge(c_acctbal)
|
||||
| group by: cntrycode
|
||||
| row-size=36B cardinality=15.00K
|
||||
|
|
||||
13:EXCHANGE [HASH(cntrycode)]
|
||||
15:EXCHANGE [HASH(cntrycode)]
|
||||
|
|
||||
06:AGGREGATE [STREAMING]
|
||||
07:AGGREGATE [STREAMING]
|
||||
| output: count(*), sum(c_acctbal)
|
||||
| group by: substr(c_phone, 1, 2)
|
||||
| row-size=36B cardinality=15.00K
|
||||
|
|
||||
05:HASH JOIN [RIGHT ANTI JOIN, PARTITIONED]
|
||||
| hash predicates: o_custkey = c_custkey
|
||||
06:HASH JOIN [RIGHT ANTI JOIN, PARTITIONED]
|
||||
| hash predicates: tpch.orders.o_custkey = c_custkey
|
||||
| row-size=51B cardinality=15.00K
|
||||
|
|
||||
|--JOIN BUILD
|
||||
| | join-table-id=00 plan-id=01 cohort-id=01
|
||||
| | build expressions: c_custkey
|
||||
| |
|
||||
| 12:EXCHANGE [HASH(c_custkey)]
|
||||
| 14:EXCHANGE [HASH(c_custkey)]
|
||||
| |
|
||||
| 04:NESTED LOOP JOIN [INNER JOIN, BROADCAST]
|
||||
| | join table id: 01
|
||||
@@ -4956,13 +4988,13 @@ PLAN-ROOT SINK
|
||||
| |--JOIN BUILD
|
||||
| | | join-table-id=01 plan-id=02 cohort-id=02
|
||||
| | |
|
||||
| | 10:EXCHANGE [BROADCAST]
|
||||
| | 13:EXCHANGE [BROADCAST]
|
||||
| | |
|
||||
| | 09:AGGREGATE [FINALIZE]
|
||||
| | 12:AGGREGATE [FINALIZE]
|
||||
| | | output: avg:merge(c_acctbal)
|
||||
| | | row-size=8B cardinality=1
|
||||
| | |
|
||||
| | 08:EXCHANGE [UNPARTITIONED]
|
||||
| | 11:EXCHANGE [UNPARTITIONED]
|
||||
| | |
|
||||
| | 02:AGGREGATE
|
||||
| | | output: avg(c_acctbal)
|
||||
@@ -4978,7 +5010,15 @@ PLAN-ROOT SINK
|
||||
| predicates: substr(c_phone, 1, 2) IN ('13', '31', '23', '29', '30', '18', '17')
|
||||
| row-size=43B cardinality=15.00K
|
||||
|
|
||||
11:EXCHANGE [HASH(o_custkey)]
|
||||
10:AGGREGATE [FINALIZE]
|
||||
| group by: tpch.orders.o_custkey
|
||||
| row-size=8B cardinality=98.39K
|
||||
|
|
||||
09:EXCHANGE [HASH(tpch.orders.o_custkey)]
|
||||
|
|
||||
05:AGGREGATE [STREAMING]
|
||||
| group by: tpch.orders.o_custkey
|
||||
| row-size=8B cardinality=98.39K
|
||||
|
|
||||
03:SCAN HDFS [tpch.orders]
|
||||
HDFS partitions=1/1 files=1 size=162.56MB
|
||||
|
||||
@@ -1475,16 +1475,16 @@ where
|
||||
order by
|
||||
s_name
|
||||
---- PLAN
|
||||
Max Per-Host Resource Reservation: Memory=55.81MB Threads=6
|
||||
Max Per-Host Resource Reservation: Memory=48.62MB Threads=6
|
||||
Per-Host Resource Estimates: Memory=65MB
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
10:SORT
|
||||
11:SORT
|
||||
| order by: s_name ASC
|
||||
| row-size=67B cardinality=400
|
||||
|
|
||||
09:HASH JOIN [RIGHT SEMI JOIN]
|
||||
| hash predicates: ps_suppkey = s_suppkey
|
||||
10:HASH JOIN [RIGHT SEMI JOIN]
|
||||
| hash predicates: tpch_kudu.partsupp.ps_suppkey = s_suppkey
|
||||
| runtime filters: RF000 <- s_suppkey, RF001 <- s_suppkey
|
||||
| row-size=87B cardinality=400
|
||||
|
|
||||
@@ -1501,6 +1501,10 @@ PLAN-ROOT SINK
|
||||
| runtime filters: RF008 -> s_nationkey, RF009 -> s_nationkey
|
||||
| row-size=85B cardinality=10.00K
|
||||
|
|
||||
09:AGGREGATE [FINALIZE]
|
||||
| group by: tpch_kudu.partsupp.ps_suppkey
|
||||
| row-size=8B cardinality=9.71K
|
||||
|
|
||||
07:HASH JOIN [RIGHT SEMI JOIN]
|
||||
| hash predicates: l_partkey = ps_partkey, l_suppkey = ps_suppkey
|
||||
| other join predicates: ps_availqty > 0.5 * sum(l_quantity)
|
||||
@@ -1517,7 +1521,7 @@ PLAN-ROOT SINK
|
||||
| | row-size=57B cardinality=20.00K
|
||||
| |
|
||||
| 02:SCAN KUDU [tpch_kudu.partsupp]
|
||||
| runtime filters: RF000 -> ps_suppkey, RF001 -> ps_suppkey, RF006 -> ps_partkey, RF007 -> ps_partkey
|
||||
| runtime filters: RF000 -> tpch_kudu.partsupp.ps_suppkey, RF001 -> tpch_kudu.partsupp.ps_suppkey, RF006 -> ps_partkey, RF007 -> ps_partkey
|
||||
| row-size=24B cardinality=800.00K
|
||||
|
|
||||
05:AGGREGATE [FINALIZE]
|
||||
@@ -1673,39 +1677,43 @@ order by
|
||||
cntrycode
|
||||
---- PLAN
|
||||
Max Per-Host Resource Reservation: Memory=13.94MB Threads=4
|
||||
Per-Host Resource Estimates: Memory=31MB
|
||||
Per-Host Resource Estimates: Memory=41MB
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
07:SORT
|
||||
08:SORT
|
||||
| order by: cntrycode ASC
|
||||
| row-size=36B cardinality=15.00K
|
||||
|
|
||||
06:AGGREGATE [FINALIZE]
|
||||
07:AGGREGATE [FINALIZE]
|
||||
| output: count(*), sum(c_acctbal)
|
||||
| group by: substr(c_phone, 1, 2)
|
||||
| row-size=36B cardinality=15.00K
|
||||
|
|
||||
05:HASH JOIN [RIGHT ANTI JOIN]
|
||||
| hash predicates: o_custkey = c_custkey
|
||||
06:HASH JOIN [LEFT ANTI JOIN]
|
||||
| hash predicates: c_custkey = tpch_kudu.orders.o_custkey
|
||||
| row-size=55B cardinality=15.00K
|
||||
|
|
||||
|--04:NESTED LOOP JOIN [INNER JOIN]
|
||||
| | predicates: c_acctbal > round(avg(c_acctbal), 1)
|
||||
| | row-size=55B cardinality=15.00K
|
||||
|--05:AGGREGATE [FINALIZE]
|
||||
| | group by: tpch_kudu.orders.o_custkey
|
||||
| | row-size=8B cardinality=98.39K
|
||||
| |
|
||||
| |--02:AGGREGATE [FINALIZE]
|
||||
| | | output: avg(c_acctbal)
|
||||
| | | row-size=8B cardinality=1
|
||||
| | |
|
||||
| | 01:SCAN KUDU [tpch_kudu.customer]
|
||||
| | predicates: substr(c_phone, 1, 2) IN ('13', '31', '23', '29', '30', '18', '17')
|
||||
| | kudu predicates: c_acctbal > 0
|
||||
| | row-size=39B cardinality=15.00K
|
||||
| |
|
||||
| 00:SCAN KUDU [tpch_kudu.customer]
|
||||
| predicates: substr(c_phone, 1, 2) IN ('13', '31', '23', '29', '30', '18', '17')
|
||||
| row-size=47B cardinality=15.00K
|
||||
| 03:SCAN KUDU [tpch_kudu.orders]
|
||||
| row-size=8B cardinality=1.50M
|
||||
|
|
||||
03:SCAN KUDU [tpch_kudu.orders]
|
||||
row-size=8B cardinality=1.50M
|
||||
04:NESTED LOOP JOIN [INNER JOIN]
|
||||
| predicates: c_acctbal > round(avg(c_acctbal), 1)
|
||||
| row-size=55B cardinality=15.00K
|
||||
|
|
||||
|--02:AGGREGATE [FINALIZE]
|
||||
| | output: avg(c_acctbal)
|
||||
| | row-size=8B cardinality=1
|
||||
| |
|
||||
| 01:SCAN KUDU [tpch_kudu.customer]
|
||||
| predicates: substr(c_phone, 1, 2) IN ('13', '31', '23', '29', '30', '18', '17')
|
||||
| kudu predicates: c_acctbal > 0
|
||||
| row-size=39B cardinality=15.00K
|
||||
|
|
||||
00:SCAN KUDU [tpch_kudu.customer]
|
||||
predicates: substr(c_phone, 1, 2) IN ('13', '31', '23', '29', '30', '18', '17')
|
||||
row-size=47B cardinality=15.00K
|
||||
====
|
||||
|
||||
@@ -36,7 +36,7 @@ PLAN-ROOT SINK
|
||||
| row-size=120B cardinality=1.50M
|
||||
|
|
||||
00:SCAN HDFS [tpch_nested_parquet.customer.c_orders.o_lineitems]
|
||||
HDFS partitions=1/1 files=4 size=289.14MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
predicates: l_shipdate <= '1998-09-02'
|
||||
row-size=68B cardinality=1.50M
|
||||
---- DISTRIBUTEDPLAN
|
||||
@@ -64,7 +64,7 @@ PLAN-ROOT SINK
|
||||
| row-size=120B cardinality=1.50M
|
||||
|
|
||||
00:SCAN HDFS [tpch_nested_parquet.customer.c_orders.o_lineitems]
|
||||
HDFS partitions=1/1 files=4 size=289.14MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
predicates: l_shipdate <= '1998-09-02'
|
||||
row-size=68B cardinality=1.50M
|
||||
====
|
||||
@@ -147,7 +147,7 @@ PLAN-ROOT SINK
|
||||
| | | row-size=0B cardinality=10
|
||||
| | |
|
||||
| | 16:SCAN HDFS [tpch_nested_parquet.region r]
|
||||
| | HDFS partitions=1/1 files=1 size=3.50KB
|
||||
| | HDFS partitions=1/1 files=1 size=3.59KB
|
||||
| | predicates: r_name = 'EUROPE', !empty(r.r_nations)
|
||||
| | row-size=31B cardinality=1
|
||||
| |
|
||||
@@ -187,7 +187,7 @@ PLAN-ROOT SINK
|
||||
| | row-size=0B cardinality=10
|
||||
| |
|
||||
| 06:SCAN HDFS [tpch_nested_parquet.region r]
|
||||
| HDFS partitions=1/1 files=1 size=3.50KB
|
||||
| HDFS partitions=1/1 files=1 size=3.59KB
|
||||
| predicates: r_name = 'EUROPE', !empty(r.r_nations)
|
||||
| row-size=31B cardinality=1
|
||||
|
|
||||
@@ -270,7 +270,7 @@ PLAN-ROOT SINK
|
||||
| | | row-size=0B cardinality=10
|
||||
| | |
|
||||
| | 16:SCAN HDFS [tpch_nested_parquet.region r]
|
||||
| | HDFS partitions=1/1 files=1 size=3.50KB
|
||||
| | HDFS partitions=1/1 files=1 size=3.59KB
|
||||
| | predicates: r_name = 'EUROPE', !empty(r.r_nations)
|
||||
| | row-size=31B cardinality=1
|
||||
| |
|
||||
@@ -312,7 +312,7 @@ PLAN-ROOT SINK
|
||||
| | row-size=0B cardinality=10
|
||||
| |
|
||||
| 06:SCAN HDFS [tpch_nested_parquet.region r]
|
||||
| HDFS partitions=1/1 files=1 size=3.50KB
|
||||
| HDFS partitions=1/1 files=1 size=3.59KB
|
||||
| predicates: r_name = 'EUROPE', !empty(r.r_nations)
|
||||
| row-size=31B cardinality=1
|
||||
|
|
||||
@@ -408,7 +408,7 @@ PLAN-ROOT SINK
|
||||
| row-size=0B cardinality=10
|
||||
|
|
||||
00:SCAN HDFS [tpch_nested_parquet.customer c]
|
||||
HDFS partitions=1/1 files=4 size=289.14MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
predicates: c_mktsegment = 'BUILDING', !empty(c.c_orders)
|
||||
predicates on o: !empty(o.o_lineitems), o_orderdate < '1995-03-15'
|
||||
predicates on l: l_shipdate > '1995-03-15'
|
||||
@@ -463,7 +463,7 @@ PLAN-ROOT SINK
|
||||
| row-size=0B cardinality=10
|
||||
|
|
||||
00:SCAN HDFS [tpch_nested_parquet.customer c]
|
||||
HDFS partitions=1/1 files=4 size=289.14MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
predicates: c_mktsegment = 'BUILDING', !empty(c.c_orders)
|
||||
predicates on o: !empty(o.o_lineitems), o_orderdate < '1995-03-15'
|
||||
predicates on l: l_shipdate > '1995-03-15'
|
||||
@@ -519,6 +519,7 @@ PLAN-ROOT SINK
|
||||
| | | row-size=48B cardinality=1
|
||||
| | |
|
||||
| | 05:UNNEST [o.o_lineitems]
|
||||
| | limit: 1
|
||||
| | row-size=24B cardinality=10
|
||||
| |
|
||||
| 07:NESTED LOOP JOIN [CROSS JOIN]
|
||||
@@ -531,7 +532,7 @@ PLAN-ROOT SINK
|
||||
| row-size=0B cardinality=10
|
||||
|
|
||||
00:SCAN HDFS [tpch_nested_parquet.customer c]
|
||||
HDFS partitions=1/1 files=4 size=289.14MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
predicates: !empty(c.c_orders)
|
||||
predicates on o: o_orderdate >= '1993-07-01', o_orderdate < '1993-10-01'
|
||||
predicates on o_lineitems: l_commitdate < l_receiptdate
|
||||
@@ -573,6 +574,7 @@ PLAN-ROOT SINK
|
||||
| | | row-size=48B cardinality=1
|
||||
| | |
|
||||
| | 05:UNNEST [o.o_lineitems]
|
||||
| | limit: 1
|
||||
| | row-size=24B cardinality=10
|
||||
| |
|
||||
| 07:NESTED LOOP JOIN [CROSS JOIN]
|
||||
@@ -585,7 +587,7 @@ PLAN-ROOT SINK
|
||||
| row-size=0B cardinality=10
|
||||
|
|
||||
00:SCAN HDFS [tpch_nested_parquet.customer c]
|
||||
HDFS partitions=1/1 files=4 size=289.14MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
predicates: !empty(c.c_orders)
|
||||
predicates on o: o_orderdate >= '1993-07-01', o_orderdate < '1993-10-01'
|
||||
predicates on o_lineitems: l_commitdate < l_receiptdate
|
||||
@@ -655,7 +657,7 @@ PLAN-ROOT SINK
|
||||
| | row-size=0B cardinality=10
|
||||
| |
|
||||
| 10:SCAN HDFS [tpch_nested_parquet.region r]
|
||||
| HDFS partitions=1/1 files=1 size=3.50KB
|
||||
| HDFS partitions=1/1 files=1 size=3.59KB
|
||||
| predicates: r_name = 'ASIA', !empty(r.r_nations)
|
||||
| row-size=31B cardinality=1
|
||||
|
|
||||
@@ -684,7 +686,7 @@ PLAN-ROOT SINK
|
||||
| row-size=0B cardinality=10
|
||||
|
|
||||
00:SCAN HDFS [tpch_nested_parquet.customer c]
|
||||
HDFS partitions=1/1 files=4 size=289.14MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
predicates: !empty(c.c_orders)
|
||||
predicates on o: !empty(o.o_lineitems), o_orderdate >= '1994-01-01', o_orderdate < '1995-01-01'
|
||||
runtime filters: RF000 -> c_nationkey, RF004 -> c.c_nationkey
|
||||
@@ -744,7 +746,7 @@ PLAN-ROOT SINK
|
||||
| | row-size=0B cardinality=10
|
||||
| |
|
||||
| 10:SCAN HDFS [tpch_nested_parquet.region r]
|
||||
| HDFS partitions=1/1 files=1 size=3.50KB
|
||||
| HDFS partitions=1/1 files=1 size=3.59KB
|
||||
| predicates: r_name = 'ASIA', !empty(r.r_nations)
|
||||
| row-size=31B cardinality=1
|
||||
|
|
||||
@@ -773,7 +775,7 @@ PLAN-ROOT SINK
|
||||
| row-size=0B cardinality=10
|
||||
|
|
||||
00:SCAN HDFS [tpch_nested_parquet.customer c]
|
||||
HDFS partitions=1/1 files=4 size=289.14MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
predicates: !empty(c.c_orders)
|
||||
predicates on o: !empty(o.o_lineitems), o_orderdate >= '1994-01-01', o_orderdate < '1995-01-01'
|
||||
runtime filters: RF000 -> c_nationkey, RF004 -> c.c_nationkey
|
||||
@@ -800,7 +802,7 @@ PLAN-ROOT SINK
|
||||
| row-size=16B cardinality=1
|
||||
|
|
||||
00:SCAN HDFS [tpch_nested_parquet.customer.c_orders.o_lineitems]
|
||||
HDFS partitions=1/1 files=4 size=289.14MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
predicates: l_discount <= 0.07, l_discount >= 0.05, l_quantity < 24, l_shipdate < '1995-01-01', l_shipdate >= '1994-01-01'
|
||||
row-size=36B cardinality=1.50M
|
||||
---- DISTRIBUTEDPLAN
|
||||
@@ -819,7 +821,7 @@ PLAN-ROOT SINK
|
||||
| row-size=16B cardinality=1
|
||||
|
|
||||
00:SCAN HDFS [tpch_nested_parquet.customer.c_orders.o_lineitems]
|
||||
HDFS partitions=1/1 files=4 size=289.14MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
predicates: l_discount <= 0.07, l_discount >= 0.05, l_quantity < 24, l_shipdate < '1995-01-01', l_shipdate >= '1994-01-01'
|
||||
row-size=36B cardinality=1.50M
|
||||
====
|
||||
@@ -882,7 +884,7 @@ PLAN-ROOT SINK
|
||||
| row-size=100B cardinality=15.00M
|
||||
|
|
||||
|--11:SCAN HDFS [tpch_nested_parquet.region.r_nations n2]
|
||||
| HDFS partitions=1/1 files=1 size=3.50KB
|
||||
| HDFS partitions=1/1 files=1 size=3.59KB
|
||||
| predicates: n2.n_name IN ('GERMANY', 'FRANCE')
|
||||
| row-size=14B cardinality=5
|
||||
|
|
||||
@@ -892,7 +894,7 @@ PLAN-ROOT SINK
|
||||
| row-size=86B cardinality=15.00M
|
||||
|
|
||||
|--10:SCAN HDFS [tpch_nested_parquet.region.r_nations n1]
|
||||
| HDFS partitions=1/1 files=1 size=3.50KB
|
||||
| HDFS partitions=1/1 files=1 size=3.59KB
|
||||
| predicates: n1.n_name IN ('FRANCE', 'GERMANY')
|
||||
| row-size=14B cardinality=5
|
||||
|
|
||||
@@ -930,7 +932,7 @@ PLAN-ROOT SINK
|
||||
| row-size=0B cardinality=10
|
||||
|
|
||||
00:SCAN HDFS [tpch_nested_parquet.customer c]
|
||||
HDFS partitions=1/1 files=4 size=289.14MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
predicates: !empty(c.c_orders)
|
||||
predicates on o: !empty(o.o_lineitems)
|
||||
predicates on l: l_shipdate >= '1995-01-01', l_shipdate <= '1996-12-31'
|
||||
@@ -969,7 +971,7 @@ PLAN-ROOT SINK
|
||||
|--19:EXCHANGE [BROADCAST]
|
||||
| |
|
||||
| 11:SCAN HDFS [tpch_nested_parquet.region.r_nations n2]
|
||||
| HDFS partitions=1/1 files=1 size=3.50KB
|
||||
| HDFS partitions=1/1 files=1 size=3.59KB
|
||||
| predicates: n2.n_name IN ('GERMANY', 'FRANCE')
|
||||
| row-size=14B cardinality=5
|
||||
|
|
||||
@@ -981,7 +983,7 @@ PLAN-ROOT SINK
|
||||
|--18:EXCHANGE [BROADCAST]
|
||||
| |
|
||||
| 10:SCAN HDFS [tpch_nested_parquet.region.r_nations n1]
|
||||
| HDFS partitions=1/1 files=1 size=3.50KB
|
||||
| HDFS partitions=1/1 files=1 size=3.59KB
|
||||
| predicates: n1.n_name IN ('FRANCE', 'GERMANY')
|
||||
| row-size=14B cardinality=5
|
||||
|
|
||||
@@ -1021,7 +1023,7 @@ PLAN-ROOT SINK
|
||||
| row-size=0B cardinality=10
|
||||
|
|
||||
00:SCAN HDFS [tpch_nested_parquet.customer c]
|
||||
HDFS partitions=1/1 files=4 size=289.14MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
predicates: !empty(c.c_orders)
|
||||
predicates on o: !empty(o.o_lineitems)
|
||||
predicates on l: l_shipdate >= '1995-01-01', l_shipdate <= '1996-12-31'
|
||||
@@ -1084,7 +1086,7 @@ PLAN-ROOT SINK
|
||||
| row-size=167B cardinality=15.00M
|
||||
|
|
||||
|--16:SCAN HDFS [tpch_nested_parquet.region.r_nations n2]
|
||||
| HDFS partitions=1/1 files=1 size=3.50KB
|
||||
| HDFS partitions=1/1 files=1 size=3.59KB
|
||||
| row-size=14B cardinality=50
|
||||
|
|
||||
19:HASH JOIN [INNER JOIN]
|
||||
@@ -1105,7 +1107,7 @@ PLAN-ROOT SINK
|
||||
| | row-size=0B cardinality=10
|
||||
| |
|
||||
| 11:SCAN HDFS [tpch_nested_parquet.region r]
|
||||
| HDFS partitions=1/1 files=1 size=3.50KB
|
||||
| HDFS partitions=1/1 files=1 size=3.59KB
|
||||
| predicates: r_name = 'AMERICA', !empty(r.r_nations)
|
||||
| row-size=31B cardinality=1
|
||||
|
|
||||
@@ -1152,7 +1154,7 @@ PLAN-ROOT SINK
|
||||
| row-size=0B cardinality=10
|
||||
|
|
||||
00:SCAN HDFS [tpch_nested_parquet.customer c]
|
||||
HDFS partitions=1/1 files=4 size=289.14MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
predicates: !empty(c.c_orders)
|
||||
predicates on o: !empty(o.o_lineitems), o_orderdate >= '1995-01-01', o_orderdate <= '1996-12-31'
|
||||
runtime filters: RF002 -> c_nationkey
|
||||
@@ -1189,7 +1191,7 @@ PLAN-ROOT SINK
|
||||
|--26:EXCHANGE [BROADCAST]
|
||||
| |
|
||||
| 16:SCAN HDFS [tpch_nested_parquet.region.r_nations n2]
|
||||
| HDFS partitions=1/1 files=1 size=3.50KB
|
||||
| HDFS partitions=1/1 files=1 size=3.59KB
|
||||
| row-size=14B cardinality=50
|
||||
|
|
||||
19:HASH JOIN [INNER JOIN, BROADCAST]
|
||||
@@ -1212,7 +1214,7 @@ PLAN-ROOT SINK
|
||||
| | row-size=0B cardinality=10
|
||||
| |
|
||||
| 11:SCAN HDFS [tpch_nested_parquet.region r]
|
||||
| HDFS partitions=1/1 files=1 size=3.50KB
|
||||
| HDFS partitions=1/1 files=1 size=3.59KB
|
||||
| predicates: r_name = 'AMERICA', !empty(r.r_nations)
|
||||
| row-size=31B cardinality=1
|
||||
|
|
||||
@@ -1263,7 +1265,7 @@ PLAN-ROOT SINK
|
||||
| row-size=0B cardinality=10
|
||||
|
|
||||
00:SCAN HDFS [tpch_nested_parquet.customer c]
|
||||
HDFS partitions=1/1 files=4 size=289.14MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
predicates: !empty(c.c_orders)
|
||||
predicates on o: !empty(o.o_lineitems), o_orderdate >= '1995-01-01', o_orderdate <= '1996-12-31'
|
||||
runtime filters: RF002 -> c_nationkey
|
||||
@@ -1320,7 +1322,7 @@ PLAN-ROOT SINK
|
||||
| row-size=169B cardinality=15.00M
|
||||
|
|
||||
|--11:SCAN HDFS [tpch_nested_parquet.region.r_nations n]
|
||||
| HDFS partitions=1/1 files=1 size=3.50KB
|
||||
| HDFS partitions=1/1 files=1 size=3.59KB
|
||||
| row-size=14B cardinality=50
|
||||
|
|
||||
13:HASH JOIN [INNER JOIN]
|
||||
@@ -1367,7 +1369,7 @@ PLAN-ROOT SINK
|
||||
| row-size=0B cardinality=10
|
||||
|
|
||||
00:SCAN HDFS [tpch_nested_parquet.customer.c_orders o]
|
||||
HDFS partitions=1/1 files=4 size=289.14MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
predicates: !empty(o.o_lineitems)
|
||||
row-size=24B cardinality=1.50M
|
||||
---- DISTRIBUTEDPLAN
|
||||
@@ -1402,7 +1404,7 @@ PLAN-ROOT SINK
|
||||
|--19:EXCHANGE [BROADCAST]
|
||||
| |
|
||||
| 11:SCAN HDFS [tpch_nested_parquet.region.r_nations n]
|
||||
| HDFS partitions=1/1 files=1 size=3.50KB
|
||||
| HDFS partitions=1/1 files=1 size=3.59KB
|
||||
| row-size=14B cardinality=50
|
||||
|
|
||||
13:HASH JOIN [INNER JOIN, BROADCAST]
|
||||
@@ -1453,7 +1455,7 @@ PLAN-ROOT SINK
|
||||
| row-size=0B cardinality=10
|
||||
|
|
||||
00:SCAN HDFS [tpch_nested_parquet.customer.c_orders o]
|
||||
HDFS partitions=1/1 files=4 size=289.14MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
predicates: !empty(o.o_lineitems)
|
||||
row-size=24B cardinality=1.50M
|
||||
====
|
||||
@@ -1510,7 +1512,7 @@ PLAN-ROOT SINK
|
||||
| row-size=275B cardinality=15.00M
|
||||
|
|
||||
|--09:SCAN HDFS [tpch_nested_parquet.region.r_nations n]
|
||||
| HDFS partitions=1/1 files=1 size=3.50KB
|
||||
| HDFS partitions=1/1 files=1 size=3.59KB
|
||||
| row-size=14B cardinality=50
|
||||
|
|
||||
01:SUBPLAN
|
||||
@@ -1538,7 +1540,7 @@ PLAN-ROOT SINK
|
||||
| row-size=0B cardinality=10
|
||||
|
|
||||
00:SCAN HDFS [tpch_nested_parquet.customer c]
|
||||
HDFS partitions=1/1 files=4 size=289.14MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
predicates: !empty(c.c_orders)
|
||||
predicates on o: !empty(o.o_lineitems), o_orderdate >= '1993-10-01', o_orderdate < '1994-01-01'
|
||||
predicates on l: l_returnflag = 'R'
|
||||
@@ -1577,7 +1579,7 @@ PLAN-ROOT SINK
|
||||
|--13:EXCHANGE [BROADCAST]
|
||||
| |
|
||||
| 09:SCAN HDFS [tpch_nested_parquet.region.r_nations n]
|
||||
| HDFS partitions=1/1 files=1 size=3.50KB
|
||||
| HDFS partitions=1/1 files=1 size=3.59KB
|
||||
| row-size=14B cardinality=50
|
||||
|
|
||||
01:SUBPLAN
|
||||
@@ -1605,7 +1607,7 @@ PLAN-ROOT SINK
|
||||
| row-size=0B cardinality=10
|
||||
|
|
||||
00:SCAN HDFS [tpch_nested_parquet.customer c]
|
||||
HDFS partitions=1/1 files=4 size=289.14MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
predicates: !empty(c.c_orders)
|
||||
predicates on o: !empty(o.o_lineitems), o_orderdate >= '1993-10-01', o_orderdate < '1994-01-01'
|
||||
predicates on l: l_returnflag = 'R'
|
||||
@@ -1668,7 +1670,7 @@ PLAN-ROOT SINK
|
||||
| | row-size=40B cardinality=100.00K
|
||||
| |
|
||||
| |--13:SCAN HDFS [tpch_nested_parquet.region.r_nations n]
|
||||
| | HDFS partitions=1/1 files=1 size=3.50KB
|
||||
| | HDFS partitions=1/1 files=1 size=3.59KB
|
||||
| | predicates: n_name = 'GERMANY'
|
||||
| | row-size=14B cardinality=5
|
||||
| |
|
||||
@@ -1701,7 +1703,7 @@ PLAN-ROOT SINK
|
||||
| row-size=48B cardinality=100.00K
|
||||
|
|
||||
|--05:SCAN HDFS [tpch_nested_parquet.region.r_nations n]
|
||||
| HDFS partitions=1/1 files=1 size=3.50KB
|
||||
| HDFS partitions=1/1 files=1 size=3.59KB
|
||||
| predicates: n_name = 'GERMANY'
|
||||
| row-size=14B cardinality=5
|
||||
|
|
||||
@@ -1758,7 +1760,7 @@ PLAN-ROOT SINK
|
||||
| |--21:EXCHANGE [BROADCAST]
|
||||
| | |
|
||||
| | 13:SCAN HDFS [tpch_nested_parquet.region.r_nations n]
|
||||
| | HDFS partitions=1/1 files=1 size=3.50KB
|
||||
| | HDFS partitions=1/1 files=1 size=3.59KB
|
||||
| | predicates: n_name = 'GERMANY'
|
||||
| | row-size=14B cardinality=5
|
||||
| |
|
||||
@@ -1800,7 +1802,7 @@ PLAN-ROOT SINK
|
||||
|--18:EXCHANGE [BROADCAST]
|
||||
| |
|
||||
| 05:SCAN HDFS [tpch_nested_parquet.region.r_nations n]
|
||||
| HDFS partitions=1/1 files=1 size=3.50KB
|
||||
| HDFS partitions=1/1 files=1 size=3.59KB
|
||||
| predicates: n_name = 'GERMANY'
|
||||
| row-size=14B cardinality=5
|
||||
|
|
||||
@@ -1878,7 +1880,7 @@ PLAN-ROOT SINK
|
||||
| row-size=0B cardinality=10
|
||||
|
|
||||
00:SCAN HDFS [tpch_nested_parquet.customer.c_orders o]
|
||||
HDFS partitions=1/1 files=4 size=289.14MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
predicates: !empty(o.o_lineitems)
|
||||
predicates on l: l_shipmode IN ('MAIL', 'SHIP'), l_commitdate < l_receiptdate, l_shipdate < l_commitdate, l_receiptdate >= '1994-01-01', l_receiptdate < '1995-01-01'
|
||||
row-size=24B cardinality=1.50M
|
||||
@@ -1919,7 +1921,7 @@ PLAN-ROOT SINK
|
||||
| row-size=0B cardinality=10
|
||||
|
|
||||
00:SCAN HDFS [tpch_nested_parquet.customer.c_orders o]
|
||||
HDFS partitions=1/1 files=4 size=289.14MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
predicates: !empty(o.o_lineitems)
|
||||
predicates on l: l_shipmode IN ('MAIL', 'SHIP'), l_commitdate < l_receiptdate, l_shipdate < l_commitdate, l_receiptdate >= '1994-01-01', l_receiptdate < '1995-01-01'
|
||||
row-size=24B cardinality=1.50M
|
||||
@@ -1977,7 +1979,7 @@ PLAN-ROOT SINK
|
||||
| row-size=0B cardinality=10
|
||||
|
|
||||
00:SCAN HDFS [tpch_nested_parquet.customer c]
|
||||
HDFS partitions=1/1 files=4 size=289.14MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
predicates on c_orders: (NOT o_comment LIKE '%special%requests%')
|
||||
row-size=20B cardinality=150.00K
|
||||
---- DISTRIBUTEDPLAN
|
||||
@@ -2029,7 +2031,7 @@ PLAN-ROOT SINK
|
||||
| row-size=0B cardinality=10
|
||||
|
|
||||
00:SCAN HDFS [tpch_nested_parquet.customer c]
|
||||
HDFS partitions=1/1 files=4 size=289.14MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
predicates on c_orders: (NOT o_comment LIKE '%special%requests%')
|
||||
row-size=20B cardinality=150.00K
|
||||
====
|
||||
@@ -2067,7 +2069,7 @@ PLAN-ROOT SINK
|
||||
| row-size=41B cardinality=200.00K
|
||||
|
|
||||
00:SCAN HDFS [tpch_nested_parquet.customer.c_orders.o_lineitems l]
|
||||
HDFS partitions=1/1 files=4 size=289.14MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
predicates: l_shipdate < '1995-10-01', l_shipdate >= '1995-09-01'
|
||||
runtime filters: RF000 -> l_partkey
|
||||
row-size=36B cardinality=1.50M
|
||||
@@ -2098,7 +2100,7 @@ PLAN-ROOT SINK
|
||||
| row-size=41B cardinality=200.00K
|
||||
|
|
||||
00:SCAN HDFS [tpch_nested_parquet.customer.c_orders.o_lineitems l]
|
||||
HDFS partitions=1/1 files=4 size=289.14MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
predicates: l_shipdate < '1995-10-01', l_shipdate >= '1995-09-01'
|
||||
runtime filters: RF000 -> l_partkey
|
||||
row-size=36B cardinality=1.50M
|
||||
@@ -2158,7 +2160,7 @@ PLAN-ROOT SINK
|
||||
| | row-size=24B cardinality=1.50M
|
||||
| |
|
||||
| 03:SCAN HDFS [tpch_nested_parquet.customer.c_orders.o_lineitems l]
|
||||
| HDFS partitions=1/1 files=4 size=289.14MB
|
||||
| HDFS partitions=1/1 files=4 size=289.08MB
|
||||
| predicates: l_shipdate < '1996-04-01', l_shipdate >= '1996-01-01'
|
||||
| row-size=36B cardinality=1.50M
|
||||
|
|
||||
@@ -2177,7 +2179,7 @@ PLAN-ROOT SINK
|
||||
| row-size=24B cardinality=1.50M
|
||||
|
|
||||
01:SCAN HDFS [tpch_nested_parquet.customer.c_orders.o_lineitems l]
|
||||
HDFS partitions=1/1 files=4 size=289.14MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
predicates: l_shipdate < '1996-04-01', l_shipdate >= '1996-01-01'
|
||||
runtime filters: RF000 -> l.l_suppkey
|
||||
row-size=36B cardinality=1.50M
|
||||
@@ -2222,7 +2224,7 @@ PLAN-ROOT SINK
|
||||
| | row-size=24B cardinality=1.50M
|
||||
| |
|
||||
| 03:SCAN HDFS [tpch_nested_parquet.customer.c_orders.o_lineitems l]
|
||||
| HDFS partitions=1/1 files=4 size=289.14MB
|
||||
| HDFS partitions=1/1 files=4 size=289.08MB
|
||||
| predicates: l_shipdate < '1996-04-01', l_shipdate >= '1996-01-01'
|
||||
| row-size=36B cardinality=1.50M
|
||||
|
|
||||
@@ -2250,7 +2252,7 @@ PLAN-ROOT SINK
|
||||
| row-size=24B cardinality=1.50M
|
||||
|
|
||||
01:SCAN HDFS [tpch_nested_parquet.customer.c_orders.o_lineitems l]
|
||||
HDFS partitions=1/1 files=4 size=289.14MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
predicates: l_shipdate < '1996-04-01', l_shipdate >= '1996-01-01'
|
||||
runtime filters: RF000 -> l.l_suppkey
|
||||
row-size=36B cardinality=1.50M
|
||||
@@ -2426,7 +2428,7 @@ PLAN-ROOT SINK
|
||||
| | row-size=16B cardinality=15.00M
|
||||
| |
|
||||
| 02:SCAN HDFS [tpch_nested_parquet.customer.c_orders.o_lineitems l]
|
||||
| HDFS partitions=1/1 files=4 size=289.14MB
|
||||
| HDFS partitions=1/1 files=4 size=289.08MB
|
||||
| row-size=16B cardinality=15.00M
|
||||
|
|
||||
04:HASH JOIN [INNER JOIN]
|
||||
@@ -2441,7 +2443,7 @@ PLAN-ROOT SINK
|
||||
| row-size=48B cardinality=1.00K
|
||||
|
|
||||
00:SCAN HDFS [tpch_nested_parquet.customer.c_orders.o_lineitems l]
|
||||
HDFS partitions=1/1 files=4 size=289.14MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
runtime filters: RF000 -> l.l_partkey, RF002 -> l_partkey
|
||||
row-size=24B cardinality=15.00M
|
||||
---- DISTRIBUTEDPLAN
|
||||
@@ -2478,7 +2480,7 @@ PLAN-ROOT SINK
|
||||
| | row-size=16B cardinality=15.00M
|
||||
| |
|
||||
| 02:SCAN HDFS [tpch_nested_parquet.customer.c_orders.o_lineitems l]
|
||||
| HDFS partitions=1/1 files=4 size=289.14MB
|
||||
| HDFS partitions=1/1 files=4 size=289.08MB
|
||||
| row-size=16B cardinality=15.00M
|
||||
|
|
||||
10:EXCHANGE [HASH(p_partkey)]
|
||||
@@ -2497,7 +2499,7 @@ PLAN-ROOT SINK
|
||||
| row-size=48B cardinality=1.00K
|
||||
|
|
||||
00:SCAN HDFS [tpch_nested_parquet.customer.c_orders.o_lineitems l]
|
||||
HDFS partitions=1/1 files=4 size=289.14MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
runtime filters: RF000 -> l.l_partkey, RF002 -> l_partkey
|
||||
row-size=24B cardinality=15.00M
|
||||
====
|
||||
@@ -2559,7 +2561,7 @@ PLAN-ROOT SINK
|
||||
| row-size=0B cardinality=10
|
||||
|
|
||||
00:SCAN HDFS [tpch_nested_parquet.customer c]
|
||||
HDFS partitions=1/1 files=4 size=289.14MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
predicates: !empty(c.c_orders)
|
||||
row-size=50B cardinality=150.00K
|
||||
---- DISTRIBUTEDPLAN
|
||||
@@ -2605,7 +2607,7 @@ PLAN-ROOT SINK
|
||||
| row-size=0B cardinality=10
|
||||
|
|
||||
00:SCAN HDFS [tpch_nested_parquet.customer c]
|
||||
HDFS partitions=1/1 files=4 size=289.14MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
predicates: !empty(c.c_orders)
|
||||
row-size=50B cardinality=150.00K
|
||||
====
|
||||
@@ -2667,7 +2669,7 @@ PLAN-ROOT SINK
|
||||
| row-size=52B cardinality=1.43K
|
||||
|
|
||||
00:SCAN HDFS [tpch_nested_parquet.customer.c_orders.o_lineitems l]
|
||||
HDFS partitions=1/1 files=4 size=289.14MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
predicates: l_shipmode IN ('AIR', 'AIR REG'), l_quantity <= 11 OR l_quantity <= 20 OR l_quantity <= 30, l_quantity <= 11 OR l_quantity <= 20 OR l_quantity >= 20, l_quantity <= 11 OR l_quantity >= 10 OR l_quantity <= 30, l_quantity <= 11 OR l_quantity >= 10 OR l_quantity >= 20, l_quantity >= 1 OR l_quantity <= 20 OR l_quantity <= 30, l_quantity >= 1 OR l_quantity <= 20 OR l_quantity >= 20, l_quantity >= 1 OR l_quantity >= 10 OR l_quantity <= 30, l_quantity >= 1 OR l_quantity >= 10 OR l_quantity >= 20, l_shipinstruct = 'DELIVER IN PERSON'
|
||||
runtime filters: RF000 -> l_partkey
|
||||
row-size=56B cardinality=1.50M
|
||||
@@ -2700,7 +2702,7 @@ PLAN-ROOT SINK
|
||||
| row-size=52B cardinality=1.43K
|
||||
|
|
||||
00:SCAN HDFS [tpch_nested_parquet.customer.c_orders.o_lineitems l]
|
||||
HDFS partitions=1/1 files=4 size=289.14MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
predicates: l_shipmode IN ('AIR', 'AIR REG'), l_quantity <= 11 OR l_quantity <= 20 OR l_quantity <= 30, l_quantity <= 11 OR l_quantity <= 20 OR l_quantity >= 20, l_quantity <= 11 OR l_quantity >= 10 OR l_quantity <= 30, l_quantity <= 11 OR l_quantity >= 10 OR l_quantity >= 20, l_quantity >= 1 OR l_quantity <= 20 OR l_quantity <= 30, l_quantity >= 1 OR l_quantity <= 20 OR l_quantity >= 20, l_quantity >= 1 OR l_quantity >= 10 OR l_quantity <= 30, l_quantity >= 1 OR l_quantity >= 10 OR l_quantity >= 20, l_shipinstruct = 'DELIVER IN PERSON'
|
||||
runtime filters: RF000 -> l_partkey
|
||||
row-size=56B cardinality=1.50M
|
||||
@@ -2773,7 +2775,7 @@ PLAN-ROOT SINK
|
||||
| | row-size=115B cardinality=100.00K
|
||||
| |
|
||||
| |--05:SCAN HDFS [tpch_nested_parquet.region.r_nations n]
|
||||
| | HDFS partitions=1/1 files=1 size=3.50KB
|
||||
| | HDFS partitions=1/1 files=1 size=3.59KB
|
||||
| | predicates: n_name = 'CANADA'
|
||||
| | row-size=14B cardinality=5
|
||||
| |
|
||||
@@ -2801,7 +2803,7 @@ PLAN-ROOT SINK
|
||||
| row-size=32B cardinality=1.50M
|
||||
|
|
||||
07:SCAN HDFS [tpch_nested_parquet.customer.c_orders.o_lineitems l]
|
||||
HDFS partitions=1/1 files=4 size=289.14MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
predicates: l_shipdate < '1995-01-01', l_shipdate >= '1994-01-01'
|
||||
runtime filters: RF000 -> l.l_partkey, RF001 -> l.l_suppkey
|
||||
row-size=36B cardinality=1.50M
|
||||
@@ -2854,7 +2856,7 @@ PLAN-ROOT SINK
|
||||
| |--16:EXCHANGE [BROADCAST]
|
||||
| | |
|
||||
| | 05:SCAN HDFS [tpch_nested_parquet.region.r_nations n]
|
||||
| | HDFS partitions=1/1 files=1 size=3.50KB
|
||||
| | HDFS partitions=1/1 files=1 size=3.59KB
|
||||
| | predicates: n_name = 'CANADA'
|
||||
| | row-size=14B cardinality=5
|
||||
| |
|
||||
@@ -2889,7 +2891,7 @@ PLAN-ROOT SINK
|
||||
| row-size=32B cardinality=1.50M
|
||||
|
|
||||
07:SCAN HDFS [tpch_nested_parquet.customer.c_orders.o_lineitems l]
|
||||
HDFS partitions=1/1 files=4 size=289.14MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
predicates: l_shipdate < '1995-01-01', l_shipdate >= '1994-01-01'
|
||||
runtime filters: RF000 -> l.l_partkey, RF001 -> l.l_suppkey
|
||||
row-size=36B cardinality=1.50M
|
||||
@@ -2974,7 +2976,7 @@ PLAN-ROOT SINK
|
||||
| row-size=146B cardinality=15.00M
|
||||
|
|
||||
|--10:SCAN HDFS [tpch_nested_parquet.region.r_nations n]
|
||||
| HDFS partitions=1/1 files=1 size=3.50KB
|
||||
| HDFS partitions=1/1 files=1 size=3.59KB
|
||||
| predicates: n_name = 'SAUDI ARABIA'
|
||||
| row-size=14B cardinality=5
|
||||
|
|
||||
@@ -3012,7 +3014,7 @@ PLAN-ROOT SINK
|
||||
| row-size=0B cardinality=10
|
||||
|
|
||||
01:SCAN HDFS [tpch_nested_parquet.customer c]
|
||||
HDFS partitions=1/1 files=4 size=289.14MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
predicates: !empty(c.c_orders)
|
||||
predicates on o: !empty(o.o_lineitems), o_orderstatus = 'F'
|
||||
predicates on l1: l1.l_receiptdate > l1.l_commitdate
|
||||
@@ -3071,7 +3073,7 @@ PLAN-ROOT SINK
|
||||
|--22:EXCHANGE [BROADCAST]
|
||||
| |
|
||||
| 10:SCAN HDFS [tpch_nested_parquet.region.r_nations n]
|
||||
| HDFS partitions=1/1 files=1 size=3.50KB
|
||||
| HDFS partitions=1/1 files=1 size=3.59KB
|
||||
| predicates: n_name = 'SAUDI ARABIA'
|
||||
| row-size=14B cardinality=5
|
||||
|
|
||||
@@ -3111,7 +3113,7 @@ PLAN-ROOT SINK
|
||||
| row-size=0B cardinality=10
|
||||
|
|
||||
01:SCAN HDFS [tpch_nested_parquet.customer c]
|
||||
HDFS partitions=1/1 files=4 size=289.14MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
predicates: !empty(c.c_orders)
|
||||
predicates on o: !empty(o.o_lineitems), o_orderstatus = 'F'
|
||||
predicates on l1: l1.l_receiptdate > l1.l_commitdate
|
||||
@@ -3176,6 +3178,7 @@ PLAN-ROOT SINK
|
||||
| | row-size=55B cardinality=1
|
||||
| |
|
||||
| 04:UNNEST [c.c_orders]
|
||||
| limit: 1
|
||||
| row-size=0B cardinality=10
|
||||
|
|
||||
06:NESTED LOOP JOIN [INNER JOIN]
|
||||
@@ -3187,12 +3190,12 @@ PLAN-ROOT SINK
|
||||
| | row-size=8B cardinality=1
|
||||
| |
|
||||
| 01:SCAN HDFS [tpch_nested_parquet.customer c]
|
||||
| HDFS partitions=1/1 files=4 size=289.14MB
|
||||
| HDFS partitions=1/1 files=4 size=289.08MB
|
||||
| predicates: c_acctbal > 0, substr(c_phone, 1, 2) IN ('13', '31', '23', '29', '30', '18', '17')
|
||||
| row-size=35B cardinality=15.00K
|
||||
|
|
||||
00:SCAN HDFS [tpch_nested_parquet.customer c]
|
||||
HDFS partitions=1/1 files=4 size=289.14MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
predicates: substr(c_phone, 1, 2) IN ('13', '31', '23', '29', '30', '18', '17')
|
||||
row-size=47B cardinality=15.00K
|
||||
---- DISTRIBUTEDPLAN
|
||||
@@ -3229,6 +3232,7 @@ PLAN-ROOT SINK
|
||||
| | row-size=55B cardinality=1
|
||||
| |
|
||||
| 04:UNNEST [c.c_orders]
|
||||
| limit: 1
|
||||
| row-size=0B cardinality=10
|
||||
|
|
||||
06:NESTED LOOP JOIN [INNER JOIN, BROADCAST]
|
||||
@@ -3248,12 +3252,12 @@ PLAN-ROOT SINK
|
||||
| | row-size=8B cardinality=1
|
||||
| |
|
||||
| 01:SCAN HDFS [tpch_nested_parquet.customer c]
|
||||
| HDFS partitions=1/1 files=4 size=289.14MB
|
||||
| HDFS partitions=1/1 files=4 size=289.08MB
|
||||
| predicates: c_acctbal > 0, substr(c_phone, 1, 2) IN ('13', '31', '23', '29', '30', '18', '17')
|
||||
| row-size=35B cardinality=15.00K
|
||||
|
|
||||
00:SCAN HDFS [tpch_nested_parquet.customer c]
|
||||
HDFS partitions=1/1 files=4 size=289.14MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
predicates: substr(c_phone, 1, 2) IN ('13', '31', '23', '29', '30', '18', '17')
|
||||
row-size=47B cardinality=15.00K
|
||||
====
|
||||
|
||||
@@ -1477,11 +1477,11 @@ order by
|
||||
---- PLAN
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
10:SORT
|
||||
11:SORT
|
||||
| order by: s_name ASC
|
||||
| row-size=67B cardinality=400
|
||||
|
|
||||
09:HASH JOIN [RIGHT SEMI JOIN]
|
||||
10:HASH JOIN [RIGHT SEMI JOIN]
|
||||
| hash predicates: tpch.partsupp.ps_suppkey = tpch.supplier.s_suppkey
|
||||
| runtime filters: RF000 <- tpch.supplier.s_suppkey
|
||||
| row-size=98B cardinality=400
|
||||
@@ -1501,6 +1501,10 @@ PLAN-ROOT SINK
|
||||
| runtime filters: RF008 -> tpch.supplier.s_nationkey
|
||||
| row-size=77B cardinality=10.00K
|
||||
|
|
||||
09:AGGREGATE [FINALIZE]
|
||||
| group by: tpch.partsupp.ps_suppkey
|
||||
| row-size=8B cardinality=9.71K
|
||||
|
|
||||
07:HASH JOIN [RIGHT SEMI JOIN]
|
||||
| hash predicates: l_partkey = tpch.partsupp.ps_partkey, l_suppkey = tpch.partsupp.ps_suppkey
|
||||
| other join predicates: tpch.partsupp.ps_availqty > 0.5 * sum(l_quantity)
|
||||
@@ -1683,16 +1687,16 @@ order by
|
||||
---- PLAN
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
07:SORT
|
||||
08:SORT
|
||||
| order by: cntrycode ASC
|
||||
| row-size=36B cardinality=15.00K
|
||||
|
|
||||
06:AGGREGATE [FINALIZE]
|
||||
07:AGGREGATE [FINALIZE]
|
||||
| output: count(*), sum(tpch.customer.c_acctbal)
|
||||
| group by: substr(tpch.customer.c_phone, 1, 2)
|
||||
| row-size=36B cardinality=15.00K
|
||||
|
|
||||
05:HASH JOIN [RIGHT ANTI JOIN]
|
||||
06:HASH JOIN [RIGHT ANTI JOIN]
|
||||
| hash predicates: tpch.orders.o_custkey = tpch.customer.c_custkey
|
||||
| row-size=51B cardinality=15.00K
|
||||
|
|
||||
@@ -1714,6 +1718,10 @@ PLAN-ROOT SINK
|
||||
| predicates: substr(tpch.customer.c_phone, 1, 2) IN ('13', '31', '23', '29', '30', '18', '17')
|
||||
| row-size=43B cardinality=15.00K
|
||||
|
|
||||
05:AGGREGATE [FINALIZE]
|
||||
| group by: tpch.orders.o_custkey
|
||||
| row-size=8B cardinality=98.39K
|
||||
|
|
||||
03:SCAN HDFS [tpch.orders]
|
||||
HDFS partitions=1/1 files=1 size=162.56MB
|
||||
row-size=8B cardinality=1.50M
|
||||
|
||||
@@ -4005,43 +4005,43 @@ select t1.bigint_col from functional.alltypestiny t1 inner join
|
||||
---- PLAN
|
||||
PLAN-ROOT SINK
|
||||
|
|
||||
11:AGGREGATE [FINALIZE]
|
||||
12:AGGREGATE [FINALIZE]
|
||||
| group by: bigint_col
|
||||
| row-size=8B cardinality=8
|
||||
|
|
||||
00:UNION
|
||||
| pass-through-operands: 01,04
|
||||
| pass-through-operands: 01,05
|
||||
| row-size=8B cardinality=11.70K
|
||||
|
|
||||
|--10:HASH JOIN [INNER JOIN]
|
||||
|--11:HASH JOIN [INNER JOIN]
|
||||
| | hash predicates: t2.bigint_col = t1.bigint_col
|
||||
| | runtime filters: RF004 <- t1.bigint_col
|
||||
| | row-size=16B cardinality=5.84K
|
||||
| |
|
||||
| |--08:SCAN HDFS [functional.alltypestiny t1]
|
||||
| |--09:SCAN HDFS [functional.alltypestiny t1]
|
||||
| | HDFS partitions=4/4 files=4 size=460B
|
||||
| | row-size=8B cardinality=8
|
||||
| |
|
||||
| 09:SCAN HDFS [functional.alltypes t2]
|
||||
| 10:SCAN HDFS [functional.alltypes t2]
|
||||
| HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| runtime filters: RF004 -> t2.bigint_col
|
||||
| row-size=8B cardinality=7.30K
|
||||
|
|
||||
|--07:HASH JOIN [RIGHT OUTER JOIN]
|
||||
|--08:HASH JOIN [RIGHT OUTER JOIN]
|
||||
| | hash predicates: t2.bigint_col = t1.bigint_col
|
||||
| | runtime filters: RF002 <- t1.bigint_col
|
||||
| | row-size=16B cardinality=5.84K
|
||||
| |
|
||||
| |--05:SCAN HDFS [functional.alltypestiny t1]
|
||||
| |--06:SCAN HDFS [functional.alltypestiny t1]
|
||||
| | HDFS partitions=4/4 files=4 size=460B
|
||||
| | row-size=8B cardinality=8
|
||||
| |
|
||||
| 06:SCAN HDFS [functional.alltypes t2]
|
||||
| 07:SCAN HDFS [functional.alltypes t2]
|
||||
| HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| runtime filters: RF002 -> t2.bigint_col
|
||||
| row-size=8B cardinality=7.30K
|
||||
|
|
||||
|--04:HASH JOIN [RIGHT SEMI JOIN]
|
||||
|--05:HASH JOIN [RIGHT SEMI JOIN]
|
||||
| | hash predicates: t2.bigint_col = t1.bigint_col
|
||||
| | runtime filters: RF000 <- t1.bigint_col
|
||||
| | row-size=8B cardinality=8
|
||||
@@ -4050,6 +4050,10 @@ PLAN-ROOT SINK
|
||||
| | HDFS partitions=4/4 files=4 size=460B
|
||||
| | row-size=8B cardinality=8
|
||||
| |
|
||||
| 04:AGGREGATE [FINALIZE]
|
||||
| | group by: t2.bigint_col
|
||||
| | row-size=8B cardinality=10
|
||||
| |
|
||||
| 03:SCAN HDFS [functional.alltypes t2]
|
||||
| HDFS partitions=24/24 files=24 size=478.45KB
|
||||
| runtime filters: RF000 -> t2.bigint_col
|
||||
@@ -4103,7 +4107,7 @@ PLAN-ROOT SINK
|
||||
| row-size=0B cardinality=10
|
||||
|
|
||||
00:SCAN HDFS [tpch_nested_parquet.customer c]
|
||||
HDFS partitions=1/1 files=4 size=288.99MB
|
||||
HDFS partitions=1/1 files=4 size=289.08MB
|
||||
row-size=32B cardinality=150.00K
|
||||
====
|
||||
# IMPALA-6388: Verify that the order of the union operands does not impact the
|
||||
|
||||
@@ -494,3 +494,23 @@ on leftSide.id = rightSide.id;
|
||||
---- TYPES
|
||||
BIGINT,BIGINT
|
||||
====
|
||||
---- QUERY
|
||||
# IMPALA-1270: ensure semi join in subplan with distinct added is
|
||||
# executable.
|
||||
# NOTE: reference functional.alltypessmall because functional_parquet.alltypessmall
|
||||
# does not have stats computed.
|
||||
select a.id, e.key from complextypestbl a
|
||||
left semi join functional.alltypessmall c on (a.id = c.int_col)
|
||||
inner join a.nested_struct.g e
|
||||
where length(e.key) > 0
|
||||
---- RESULTS
|
||||
1,'foo'
|
||||
2,'g1'
|
||||
2,'g2'
|
||||
2,'g3'
|
||||
2,'g4'
|
||||
2,'g5'
|
||||
5,'foo'
|
||||
---- TYPES
|
||||
BIGINT,STRING
|
||||
====
|
||||
|
||||
@@ -1434,3 +1434,88 @@ from functional.alltypestiny
|
||||
---- TYPES
|
||||
INT, INT
|
||||
====
|
||||
---- QUERY
|
||||
# IMPALA-1270: test that distinct subquery is executable and returns correct results.
|
||||
select id from alltypestiny
|
||||
where int_col in (select int_col from alltypes where id % 2 = 0)
|
||||
---- RESULTS
|
||||
0
|
||||
2
|
||||
4
|
||||
6
|
||||
---- TYPES
|
||||
INT
|
||||
====
|
||||
---- QUERY
|
||||
# IMPALA-1270: test that distinct subquery with anti join is executable and
|
||||
# returns correct results.
|
||||
select id from alltypestiny
|
||||
where int_col not in (select int_col from alltypes where id % 2 = 0)
|
||||
---- RESULTS
|
||||
1
|
||||
3
|
||||
5
|
||||
7
|
||||
---- TYPES
|
||||
INT
|
||||
====
|
||||
---- QUERY
|
||||
# IMPALA-1270: test that subquery with no join predicates is executable and
|
||||
# returns correct results. A limit is added by the planner.
|
||||
select id from alltypestiny
|
||||
where exists (select int_col from alltypes where id % 2 = 0)
|
||||
---- RESULTS
|
||||
0
|
||||
1
|
||||
2
|
||||
3
|
||||
4
|
||||
5
|
||||
6
|
||||
7
|
||||
---- TYPES
|
||||
INT
|
||||
====
|
||||
---- QUERY
|
||||
# IMPALA-1270: test subquery with multiple join predicates with distinct
|
||||
# added returns correct results.
|
||||
select count(*) from alltypesagg t1
|
||||
where int_col in (
|
||||
select int_col from alltypes t2
|
||||
where t1.bool_col = t2.bool_col and id is not null);
|
||||
---- RESULTS
|
||||
90
|
||||
---- TYPES
|
||||
BIGINT
|
||||
====
|
||||
---- QUERY
|
||||
# IMPALA-1270: test subquery with aggregate function returns correct results.
|
||||
select id from alltypesagg t1
|
||||
where int_col in (
|
||||
select count(*)
|
||||
from alltypes t2
|
||||
group by int_col, tinyint_col)
|
||||
---- RESULTS
|
||||
730
|
||||
730
|
||||
1730
|
||||
1730
|
||||
2730
|
||||
2730
|
||||
3730
|
||||
3730
|
||||
4730
|
||||
4730
|
||||
5730
|
||||
5730
|
||||
6730
|
||||
6730
|
||||
7730
|
||||
7730
|
||||
8730
|
||||
8730
|
||||
9730
|
||||
9730
|
||||
---- TYPES
|
||||
INT
|
||||
====
|
||||
|
||||
Reference in New Issue
Block a user