From 782fe0c886668ada5b007bb16e7bb7a127e3e182 Mon Sep 17 00:00:00 2001 From: okumin Date: Tue, 17 Sep 2024 18:59:42 +0900 Subject: [PATCH 1/6] HIVE-24167: TPC-DS query 14 fails while generating plan for the filter --- .../org/apache/hadoop/hive/conf/HiveConf.java | 4 + data/conf/hive-site.xml | 5 + data/conf/iceberg/llap/hive-site.xml | 5 + data/conf/iceberg/tez/hive-site.xml | 5 + data/conf/llap/hive-site.xml | 5 + data/conf/perf/tpcds30tb/tez/hive-site.xml | 5 + data/conf/rlist/hive-site.xml | 5 + data/conf/tez/hive-site.xml | 5 + .../org/apache/hadoop/hive/ql/Context.java | 5 +- .../RuntimeStatsPersistenceCheckerHook.java | 4 + .../hadoop/hive/ql/parse/TezCompiler.java | 33 - .../hive/ql/plan/mapper/PlanMapper.java | 37 +- .../hive/ql/plan/mapper/StatsSources.java | 5 + .../hive/ql/reexec/ReOptimizePlugin.java | 8 + .../queries/clientpositive/perf/cbo_query14.q | 1 - .../queries/clientpositive/perf/query14.q | 1 - .../perf/tpcds30tb/tez/cbo_query14.q.out | 481 ++--- .../perf/tpcds30tb/tez/query14.q.out | 1823 +++++++++++------ 18 files changed, 1541 insertions(+), 896 deletions(-) diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java index 0d3f9e358ba3..3e5036f3aa36 100644 --- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -809,6 +809,10 @@ public static enum ConfVars { HIVE_IN_TEST_REPL("hive.in.repl.test", false, "internal usage only, true in replication test mode", true), HIVE_IN_TEST_IDE("hive.in.ide.test", false, "internal usage only, true if test running in ide", true), + HIVE_IN_TEST_PLANMAPPER_STRICT_VALIDATION("hive.in.test.planmapper.strict.validation", false, + "internal use only, whether to raise an error when unexpected links are found. We ignore equivalence mapping " + + "violation because it introduces only minor problems. But we want to strictly check it in qtest so that we " + + "can prevent further degradations"), HIVE_TESTING_SHORT_LOGS("hive.testing.short.logs", false, "internal usage only, used only in test mode. If set true, when requesting the " + "operation logs the short version (generated by LogDivertAppenderForTest) will be " + diff --git a/data/conf/hive-site.xml b/data/conf/hive-site.xml index cd716824bb9f..36819fcdcbb0 100644 --- a/data/conf/hive-site.xml +++ b/data/conf/hive-site.xml @@ -31,6 +31,11 @@ Internal marker for test. Used for masking env-dependent values + + hive.in.test.planmapper.strict.validation + true + + mapreduce.jobtracker.staging.root.dir diff --git a/data/conf/iceberg/llap/hive-site.xml b/data/conf/iceberg/llap/hive-site.xml index e6dedf966e78..98606eb26dc5 100644 --- a/data/conf/iceberg/llap/hive-site.xml +++ b/data/conf/iceberg/llap/hive-site.xml @@ -25,6 +25,11 @@ Internal marker for test. Used for masking env-dependent values + + hive.in.test.planmapper.strict.validation + true + + diff --git a/data/conf/iceberg/tez/hive-site.xml b/data/conf/iceberg/tez/hive-site.xml index 2fda519cad96..917881ccf861 100644 --- a/data/conf/iceberg/tez/hive-site.xml +++ b/data/conf/iceberg/tez/hive-site.xml @@ -25,6 +25,11 @@ Internal marker for test. Used for masking env-dependent values + + hive.in.test.planmapper.strict.validation + true + + diff --git a/data/conf/llap/hive-site.xml b/data/conf/llap/hive-site.xml index 2cf2882b6efa..291c3445ad2a 100644 --- a/data/conf/llap/hive-site.xml +++ b/data/conf/llap/hive-site.xml @@ -31,6 +31,11 @@ Internal marker for test. Used for masking env-dependent values + + hive.in.test.planmapper.strict.validation + true + + diff --git a/data/conf/perf/tpcds30tb/tez/hive-site.xml b/data/conf/perf/tpcds30tb/tez/hive-site.xml index a8d81e33c38e..ecd940572c1b 100644 --- a/data/conf/perf/tpcds30tb/tez/hive-site.xml +++ b/data/conf/perf/tpcds30tb/tez/hive-site.xml @@ -22,6 +22,11 @@ true Internal marker for test. Used for masking env-dependent values + + hive.in.test.planmapper.strict.validation + false + We can enable it once we resolve the problem of query14.q + hive.rpc.query.plan true diff --git a/data/conf/rlist/hive-site.xml b/data/conf/rlist/hive-site.xml index 9768b759fb31..492dd647fb64 100644 --- a/data/conf/rlist/hive-site.xml +++ b/data/conf/rlist/hive-site.xml @@ -30,6 +30,11 @@ Internal marker for test. Used for masking env-dependent values + + hive.in.test.planmapper.strict.validation + true + + diff --git a/data/conf/tez/hive-site.xml b/data/conf/tez/hive-site.xml index 7ee041667089..34b6ae159750 100644 --- a/data/conf/tez/hive-site.xml +++ b/data/conf/tez/hive-site.xml @@ -25,6 +25,11 @@ Internal marker for test. Used for masking env-dependent values + + hive.in.test.planmapper.strict.validation + true + + diff --git a/ql/src/java/org/apache/hadoop/hive/ql/Context.java b/ql/src/java/org/apache/hadoop/hive/ql/Context.java index 2e6df97c1521..742de91b97e2 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/Context.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/Context.java @@ -175,7 +175,7 @@ public class Context { private WmContext wmContext; private boolean isExplainPlan = false; - private PlanMapper planMapper = new PlanMapper(); + private PlanMapper planMapper; private StatsSource statsSource; private int executionIndex; @@ -423,6 +423,7 @@ private Context(Configuration conf, String executionId) { HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_MATERIALIZED_VIEW_ENABLE_AUTO_REWRITING_SQL) || HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_MATERIALIZED_VIEW_ENABLE_AUTO_REWRITING_SUBQUERY_SQL); scheduledQuery = false; + planMapper = new PlanMapper(conf); } protected Context(Context ctx) { @@ -470,6 +471,8 @@ protected Context(Context ctx) { this.opContext = new CompilationOpContext(); this.enableUnparse = ctx.enableUnparse; this.scheduledQuery = ctx.scheduledQuery; + // Don't inherit the original plan mapper + this.planMapper = new PlanMapper(ctx.conf); } public Map getFsScratchDirs() { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/hooks/RuntimeStatsPersistenceCheckerHook.java b/ql/src/java/org/apache/hadoop/hive/ql/hooks/RuntimeStatsPersistenceCheckerHook.java index cc754eebc4e2..28de59578454 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/hooks/RuntimeStatsPersistenceCheckerHook.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/hooks/RuntimeStatsPersistenceCheckerHook.java @@ -38,6 +38,10 @@ public class RuntimeStatsPersistenceCheckerHook implements ExecuteWithHookContex public void run(HookContext hookContext) throws Exception { PlanMapper pm = ((PrivateHookContext) hookContext).getContext().getPlanMapper(); + if (pm.isBroken()) { + LOG.warn("Skip checking signatures. The PlanMapper is broken"); + return; + } List sigs = pm.getAll(OpTreeSignature.class); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java index 6c17e9878893..77cf5353085f 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java @@ -26,7 +26,6 @@ import java.util.HashMap; import java.util.HashSet; import java.util.IdentityHashMap; -import java.util.Iterator; import java.util.LinkedHashMap; import java.util.LinkedHashSet; import java.util.LinkedList; @@ -100,7 +99,6 @@ import org.apache.hadoop.hive.ql.optimizer.SharedWorkOptimizer; import org.apache.hadoop.hive.ql.optimizer.SortedDynPartitionOptimizer; import org.apache.hadoop.hive.ql.optimizer.topnkey.TopNKeyProcessor; -import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveFilter; import org.apache.hadoop.hive.ql.optimizer.correlation.ReduceSinkDeDuplication; import org.apache.hadoop.hive.ql.optimizer.topnkey.TopNKeyPushdownProcessor; import org.apache.hadoop.hive.ql.optimizer.correlation.ReduceSinkJoinDeDuplication; @@ -117,7 +115,6 @@ import org.apache.hadoop.hive.ql.optimizer.physical.SerializeFilter; import org.apache.hadoop.hive.ql.optimizer.physical.StageIDsRearranger; import org.apache.hadoop.hive.ql.optimizer.physical.Vectorizer; -import org.apache.hadoop.hive.ql.optimizer.signature.OpTreeSignature; import org.apache.hadoop.hive.ql.optimizer.stats.annotation.AnnotateWithStatistics; import org.apache.hadoop.hive.ql.plan.AggregationDesc; import org.apache.hadoop.hive.ql.plan.AppMasterEventDesc; @@ -139,7 +136,6 @@ import org.apache.hadoop.hive.ql.plan.TezWork; import org.apache.hadoop.hive.ql.plan.mapper.AuxOpTreeSignature; import org.apache.hadoop.hive.ql.plan.mapper.PlanMapper; -import org.apache.hadoop.hive.ql.plan.mapper.PlanMapper.EquivGroup; import org.apache.hadoop.hive.ql.session.SessionState; import org.apache.hadoop.hive.ql.session.SessionState.LogHelper; import org.apache.hadoop.hive.ql.stats.OperatorStats; @@ -991,35 +987,6 @@ private void removeSemiJoinIfNoStats(OptimizeTezProcContext procCtx) ogw.startWalking(topNodes, null); } - private static class CollectAll implements SemanticNodeProcessor { - private PlanMapper planMapper; - - @Override - public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, Object... nodeOutputs) - throws SemanticException { - ParseContext pCtx = ((OptimizeTezProcContext) procCtx).parseContext; - planMapper = pCtx.getContext().getPlanMapper(); - FilterOperator fop = (FilterOperator) nd; - OpTreeSignature sig = planMapper.getSignatureOf(fop); - List ar = getGroups(planMapper, HiveFilter.class); - - - return nd; - } - - private List getGroups(PlanMapper planMapper2, Class class1) { - Iterator it = planMapper.iterateGroups(); - List ret = new ArrayList(); - while (it.hasNext()) { - EquivGroup g = it.next(); - if (g.getAll(class1).size() > 0) { - ret.add(g); - } - } - return ret; - } - } - private static class MarkRuntimeStatsAsIncorrect implements SemanticNodeProcessor { private PlanMapper planMapper; diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/PlanMapper.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/PlanMapper.java index 80c23ae9a3a9..f0d795df078e 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/PlanMapper.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/PlanMapper.java @@ -32,11 +32,17 @@ import java.util.Objects; import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; import org.apache.hadoop.hive.ql.exec.Operator; import org.apache.hadoop.hive.ql.optimizer.signature.OpTreeSignature; import org.apache.hadoop.hive.ql.optimizer.signature.OpTreeSignatureFactory; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Sets; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Enables to connect related objects to eachother. @@ -44,9 +50,16 @@ * Most importantly it aids to connect Operators to OperatorStats and probably RelNodes. */ public class PlanMapper { + private static final Logger LOG = LoggerFactory.getLogger(PlanMapper.class); - Set groups = new HashSet<>(); - private Map objectMap = new CompositeMap<>(OpTreeSignature.class, AuxOpTreeSignature.class); + private final Set groups = new HashSet<>(); + private final Map objectMap = new CompositeMap<>(OpTreeSignature.class, AuxOpTreeSignature.class); + private final boolean failsWithIllegalLink; + private final AtomicBoolean isBroken = new AtomicBoolean(false); + + public PlanMapper(Configuration conf) { + failsWithIllegalLink = HiveConf.getBoolVar(conf, ConfVars.HIVE_IN_TEST_PLANMAPPER_STRICT_VALIDATION); + } /** * Specialized class which can compare by identity or value; based on the key type. @@ -217,7 +230,11 @@ private void link(Object o1, Object o2, boolean mayMerge) { } if (mGroups.size() > 1) { if (!mayMerge) { - throw new RuntimeException("equivalence mapping violation"); + LOG.warn("Illegally linking {} and {}", o1, o2); + if (failsWithIllegalLink) { + throw new RuntimeException("equivalence mapping violation"); + } + isBroken.set(true); } EquivGroup newGrp = new EquivGroup(); newGrp.add(o1); @@ -248,6 +265,10 @@ private Object getKeyFor(Object o) { return o; } + public boolean isBroken() { + return isBroken.get(); + } + public List getAll(Class clazz) { List ret = new ArrayList<>(); for (EquivGroup g : groups) { @@ -256,13 +277,7 @@ public List getAll(Class clazz) { return ret; } - public void runMapper(GroupTransformer mapper) { - for (EquivGroup equivGroup : groups) { - mapper.map(equivGroup); - } - } - - public List lookupAll(Class clazz, Object key) { + private List lookupAll(Class clazz, Object key) { EquivGroup group = objectMap.get(key); if (group == null) { throw new NoSuchElementException(Objects.toString(key)); @@ -270,6 +285,7 @@ public List lookupAll(Class clazz, Object key) { return group.getAll(clazz); } + @VisibleForTesting public T lookup(Class clazz, Object key) { List all = lookupAll(clazz, key); if (all.size() != 1) { @@ -279,7 +295,6 @@ public T lookup(Class clazz, Object key) { return all.get(0); } - @VisibleForTesting public Iterator iterateGroups() { return groups.iterator(); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/StatsSources.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/StatsSources.java index fc265d0223f6..6d5eef26426c 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/StatsSources.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/StatsSources.java @@ -77,6 +77,11 @@ public static StatsSource getStatsSourceContaining(StatsSource currentStatsSourc private static ImmutableList extractStatsFromPlanMapper(PlanMapper pm) { Builder li = ImmutableList.builder(); + if (pm.isBroken()) { + LOG.warn("Don't generate any stats. This PlanMapper is broken"); + return li.build(); + } + Iterator it = pm.iterateGroups(); while (it.hasNext()) { EquivGroup e = it.next(); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/reexec/ReOptimizePlugin.java b/ql/src/java/org/apache/hadoop/hive/ql/reexec/ReOptimizePlugin.java index fd9b47ccba25..a0920da6c616 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/reexec/ReOptimizePlugin.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/reexec/ReOptimizePlugin.java @@ -102,6 +102,14 @@ public void prepareToReExecute() { @Override public boolean shouldReExecuteAfterCompile(int executionNum, PlanMapper oldPlanMapper, PlanMapper newPlanMapper) { + if (oldPlanMapper.isBroken() || newPlanMapper.isBroken()) { + LOG.warn( + "Giving up a re-execution. The old plan mapper is {}, and the new one is {}", + oldPlanMapper.isBroken() ? "broken" : "not broken", + newPlanMapper.isBroken() ? "broken" : "not broken"); + return false; + } + boolean planDidChange = !planEquals(oldPlanMapper, newPlanMapper); LOG.info("planDidChange: {}", planDidChange); return planDidChange; diff --git a/ql/src/test/queries/clientpositive/perf/cbo_query14.q b/ql/src/test/queries/clientpositive/perf/cbo_query14.q index 870ff909ad28..d7881a97c592 100644 --- a/ql/src/test/queries/clientpositive/perf/cbo_query14.q +++ b/ql/src/test/queries/clientpositive/perf/cbo_query14.q @@ -1,5 +1,4 @@ set hive.optimize.cte.suggester.class=org.apache.hadoop.hive.ql.optimizer.calcite.CommonTableExpressionPrintSuggester; ---! qt:disabled:HIVE-24167 set hive.mapred.mode=nonstrict; -- start query 1 in stream 0 using template query14.tpl and seed 1819994127 explain cbo diff --git a/ql/src/test/queries/clientpositive/perf/query14.q b/ql/src/test/queries/clientpositive/perf/query14.q index 8a0846443151..c12ecb56c4ad 100644 --- a/ql/src/test/queries/clientpositive/perf/query14.q +++ b/ql/src/test/queries/clientpositive/perf/query14.q @@ -1,4 +1,3 @@ ---! qt:disabled:HIVE-24167 set hive.mapred.mode=nonstrict; -- start query 1 in stream 0 using template query14.tpl and seed 1819994127 explain diff --git a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/cbo_query14.q.out b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/cbo_query14.q.out index c1e097652d87..b08d48b994da 100644 --- a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/cbo_query14.q.out +++ b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/cbo_query14.q.out @@ -1,6 +1,7 @@ -Warning: Map Join MAPJOIN[1133][bigTable=?] in task 'Reducer 2' is a cross product -Warning: Map Join MAPJOIN[1398][bigTable=?] in task 'Reducer 35' is a cross product -Warning: Map Join MAPJOIN[1492][bigTable=?] in task 'Reducer 42' is a cross product +Warning: Map Join MAPJOIN[1098][bigTable=?] in task 'Reducer 10' is a cross product +Warning: Map Join MAPJOIN[1147][bigTable=?] in task 'Reducer 15' is a cross product +Warning: Map Join MAPJOIN[1163][bigTable=?] in task 'Reducer 21' is a cross product +Warning: Map Join MAPJOIN[1199][bigTable=?] in task 'Reducer 30' is a cross product CBO PLAN: HiveSortLimit(sort0=[$0], sort1=[$1], sort2=[$2], sort3=[$3], dir0=[ASC], dir1=[ASC], dir2=[ASC], dir3=[ASC], fetch=[100]) HiveProject(channel=[$0], i_brand_id=[$1], i_class_id=[$2], i_category_id=[$3], $f4=[$4], $f5=[$5]) @@ -8,288 +9,228 @@ HiveSortLimit(sort0=[$0], sort1=[$1], sort2=[$2], sort3=[$3], dir0=[ASC], dir1=[ HiveProject(channel=[$0], i_brand_id=[$1], i_class_id=[$2], i_category_id=[$3], sales=[$4], number_sales=[$5]) HiveUnion(all=[true]) HiveProject(channel=[_UTF-16LE'store':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"], i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], sales=[$3], number_sales=[$4]) - HiveJoin(condition=[>($3, $5)], joinType=[inner], algorithm=[none], cost=[not available]) + HiveJoin(condition=[>($3, $6)], joinType=[inner], algorithm=[none], cost=[not available]) HiveProject($f0=[$0], $f1=[$1], $f2=[$2], $f3=[$3], $f4=[$4]) HiveFilter(condition=[IS NOT NULL($3)]) HiveAggregate(group=[{0, 1, 2}], agg#0=[sum($3)], agg#1=[count()]) - HiveProject($f0=[$13], $f1=[$14], $f2=[$15], $f3=[*(CAST($1):DECIMAL(10, 0), $2)]) - HiveJoin(condition=[=($0, $12)], joinType=[inner], algorithm=[none], cost=[not available]) - HiveJoin(condition=[AND(=($6, $9), =($7, $10), =($8, $11))], joinType=[inner], algorithm=[none], cost=[not available]) - HiveJoin(condition=[=($0, $5)], joinType=[inner], algorithm=[none], cost=[not available]) - HiveJoin(condition=[=($3, $4)], joinType=[inner], algorithm=[none], cost=[not available]) - HiveProject(ss_item_sk=[$1], ss_quantity=[$9], ss_list_price=[$11], ss_sold_date_sk=[$22]) - HiveFilter(condition=[IS NOT NULL($22)]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, store_sales]], table:alias=[store_sales]) - HiveProject(d_date_sk=[$0]) - HiveFilter(condition=[AND(=($6, 2000), =($8, 11))]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, date_dim]], table:alias=[date_dim]) - HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11]) - HiveFilter(condition=[AND(IS NOT NULL($11), IS NOT NULL($7), IS NOT NULL($9))]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, item]], table:alias=[item]) - HiveProject($f0=[$0], $f1=[$1], $f2=[$2]) - HiveFilter(condition=[=($3, 3)]) - HiveAggregate(group=[{0, 1, 2}], agg#0=[count($3)]) - HiveProject(brand_id=[$0], class_id=[$1], category_id=[$2], $f3=[$3]) - HiveUnion(all=[true]) + HiveProject($f0=[$8], $f1=[$9], $f2=[$10], $f3=[*(CAST($1):DECIMAL(10, 0), $2)]) + HiveJoin(condition=[=($0, $7)], joinType=[inner], algorithm=[none], cost=[not available]) + HiveSemiJoin(condition=[=($0, $7)], joinType=[semi]) + HiveJoin(condition=[=($3, $4)], joinType=[inner], algorithm=[none], cost=[not available]) + HiveProject(ss_item_sk=[$1], ss_quantity=[$9], ss_list_price=[$11], ss_sold_date_sk=[$22]) + HiveFilter(condition=[IS NOT NULL($22)]) + HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales]) + HiveProject(d_date_sk=[$0], d_year=[CAST(2000):INTEGER], d_moy=[CAST(11):INTEGER]) + HiveFilter(condition=[AND(=($6, 2000), =($8, 11))]) + HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim]) + HiveProject(ss_item_sk=[$0]) + HiveJoin(condition=[AND(=($1, $4), =($2, $5), =($3, $6))], joinType=[inner], algorithm=[none], cost=[not available]) + HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11]) + HiveFilter(condition=[AND(IS NOT NULL($11), IS NOT NULL($7), IS NOT NULL($9))]) + HiveTableScan(table=[[default, item]], table:alias=[item]) + HiveProject($f0=[$0], $f1=[$1], $f2=[$2]) + HiveFilter(condition=[=($3, 3)]) + HiveAggregate(group=[{0, 1, 2}], agg#0=[count($3)]) HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3]) - HiveAggregate(group=[{4, 5, 6}], agg#0=[count()]) - HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available]) - HiveJoin(condition=[=($1, $2)], joinType=[inner], algorithm=[none], cost=[not available]) - HiveProject(ss_item_sk=[$1], ss_sold_date_sk=[$22]) - HiveFilter(condition=[IS NOT NULL($22)]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, store_sales]], table:alias=[store_sales]) - HiveProject(d_date_sk=[$0]) - HiveFilter(condition=[BETWEEN(false, $6, 1999, 2001)]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, date_dim]], table:alias=[d1]) - HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11]) - HiveFilter(condition=[AND(IS NOT NULL($11), IS NOT NULL($7), IS NOT NULL($9))]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, item]], table:alias=[iss]) - HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3]) - HiveAggregate(group=[{4, 5, 6}], agg#0=[count()]) - HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available]) - HiveJoin(condition=[=($1, $2)], joinType=[inner], algorithm=[none], cost=[not available]) - HiveProject(cs_item_sk=[$14], cs_sold_date_sk=[$33]) - HiveFilter(condition=[IS NOT NULL($33)]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, catalog_sales]], table:alias=[catalog_sales]) - HiveProject(d_date_sk=[$0]) - HiveFilter(condition=[BETWEEN(false, $6, 1999, 2001)]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, date_dim]], table:alias=[d2]) - HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11]) - HiveFilter(condition=[AND(IS NOT NULL($11), IS NOT NULL($7), IS NOT NULL($9))]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, item]], table:alias=[ics]) - HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3]) - HiveAggregate(group=[{4, 5, 6}], agg#0=[count()]) - HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available]) - HiveJoin(condition=[=($1, $2)], joinType=[inner], algorithm=[none], cost=[not available]) - HiveProject(ws_item_sk=[$2], ws_sold_date_sk=[$33]) - HiveFilter(condition=[IS NOT NULL($33)]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, web_sales]], table:alias=[web_sales]) - HiveProject(d_date_sk=[$0]) - HiveFilter(condition=[BETWEEN(false, $6, 1999, 2001)]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, date_dim]], table:alias=[d3]) - HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11]) - HiveFilter(condition=[AND(IS NOT NULL($11), IS NOT NULL($7), IS NOT NULL($9))]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, item]], table:alias=[iws]) + HiveUnion(all=[true]) + HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3]) + HiveAggregate(group=[{4, 5, 6}], agg#0=[count()]) + HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available]) + HiveJoin(condition=[=($1, $2)], joinType=[inner], algorithm=[none], cost=[not available]) + HiveProject(ss_item_sk=[$1], ss_sold_date_sk=[$22]) + HiveFilter(condition=[IS NOT NULL($22)]) + HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales]) + HiveProject(d_date_sk=[$0]) + HiveFilter(condition=[BETWEEN(false, $6, 1999, 2001)]) + HiveTableScan(table=[[default, date_dim]], table:alias=[d1]) + HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11]) + HiveFilter(condition=[AND(IS NOT NULL($11), IS NOT NULL($7), IS NOT NULL($9))]) + HiveTableScan(table=[[default, item]], table:alias=[iss]) + HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3]) + HiveAggregate(group=[{4, 5, 6}], agg#0=[count()]) + HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available]) + HiveJoin(condition=[=($1, $2)], joinType=[inner], algorithm=[none], cost=[not available]) + HiveProject(cs_item_sk=[$14], cs_sold_date_sk=[$33]) + HiveFilter(condition=[IS NOT NULL($33)]) + HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales]) + HiveProject(d_date_sk=[$0]) + HiveFilter(condition=[BETWEEN(false, $6, 1999, 2001)]) + HiveTableScan(table=[[default, date_dim]], table:alias=[d2]) + HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11]) + HiveFilter(condition=[AND(IS NOT NULL($11), IS NOT NULL($7), IS NOT NULL($9))]) + HiveTableScan(table=[[default, item]], table:alias=[ics]) + HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3]) + HiveAggregate(group=[{4, 5, 6}], agg#0=[count()]) + HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available]) + HiveJoin(condition=[=($1, $2)], joinType=[inner], algorithm=[none], cost=[not available]) + HiveProject(ws_item_sk=[$2], ws_sold_date_sk=[$33]) + HiveFilter(condition=[IS NOT NULL($33)]) + HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales]) + HiveProject(d_date_sk=[$0]) + HiveFilter(condition=[BETWEEN(false, $6, 1999, 2001)]) + HiveTableScan(table=[[default, date_dim]], table:alias=[d3]) + HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11]) + HiveFilter(condition=[AND(IS NOT NULL($11), IS NOT NULL($7), IS NOT NULL($9))]) + HiveTableScan(table=[[default, item]], table:alias=[iws]) HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, item]], table:alias=[item]) - HiveProject($f0=[CAST(/($0, $1)):DECIMAL(22, 6)]) - HiveFilter(condition=[IS NOT NULL(CAST(/($0, $1)):DECIMAL(22, 6))]) - HiveProject($f0=[$0], $f1=[$1]) - HiveAggregate(group=[{}], agg#0=[sum($0)], agg#1=[count($0)]) - HiveProject($f0=[*(CAST($0):DECIMAL(10, 0), $1)]) - HiveUnion(all=[true]) - HiveProject(quantity=[$0], list_price=[$1]) - HiveJoin(condition=[=($2, $3)], joinType=[inner], algorithm=[none], cost=[not available]) - HiveProject(ss_quantity=[$9], ss_list_price=[$11], ss_sold_date_sk=[$22]) - HiveFilter(condition=[IS NOT NULL($22)]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, store_sales]], table:alias=[store_sales]) - HiveProject(d_date_sk=[$0]) - HiveFilter(condition=[BETWEEN(false, $6, 1999, 2001)]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, date_dim]], table:alias=[date_dim]) - HiveProject(quantity=[$0], list_price=[$1]) - HiveJoin(condition=[=($2, $3)], joinType=[inner], algorithm=[none], cost=[not available]) - HiveProject(cs_quantity=[$17], cs_list_price=[$19], cs_sold_date_sk=[$33]) - HiveFilter(condition=[IS NOT NULL($33)]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, catalog_sales]], table:alias=[catalog_sales]) - HiveProject(d_date_sk=[$0]) - HiveFilter(condition=[BETWEEN(false, $6, 1998, 2000)]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, date_dim]], table:alias=[date_dim]) - HiveProject(quantity=[$0], list_price=[$1]) - HiveJoin(condition=[=($2, $3)], joinType=[inner], algorithm=[none], cost=[not available]) - HiveProject(ws_quantity=[$17], ws_list_price=[$19], ws_sold_date_sk=[$33]) - HiveFilter(condition=[IS NOT NULL($33)]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, web_sales]], table:alias=[web_sales]) - HiveProject(d_date_sk=[$0]) - HiveFilter(condition=[BETWEEN(false, $6, 1998, 2000)]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, date_dim]], table:alias=[date_dim]) + HiveTableScan(table=[[default, item]], table:alias=[item]) + HiveJoin(condition=[true], joinType=[inner], algorithm=[none], cost=[not available]) + HiveProject(cnt=[$0]) + HiveFilter(condition=[sq_count_check($0)]) + HiveProject(cnt=[$0]) + HiveAggregate(group=[{}], cnt=[COUNT()]) + HiveTableScan(table=[[default, avg_sales]], table:alias=[avg_sales]) + HiveProject(average_sales=[$0]) + HiveFilter(condition=[IS NOT NULL($0)]) + HiveTableScan(table=[[default, avg_sales]], table:alias=[avg_sales]) HiveProject(channel=[_UTF-16LE'catalog':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"], i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], sales=[$3], number_sales=[$4]) - HiveJoin(condition=[>($3, $5)], joinType=[inner], algorithm=[none], cost=[not available]) + HiveJoin(condition=[>($3, $6)], joinType=[inner], algorithm=[none], cost=[not available]) HiveProject($f0=[$0], $f1=[$1], $f2=[$2], $f3=[$3], $f4=[$4]) HiveFilter(condition=[IS NOT NULL($3)]) HiveAggregate(group=[{0, 1, 2}], agg#0=[sum($3)], agg#1=[count()]) - HiveProject($f0=[$13], $f1=[$14], $f2=[$15], $f3=[*(CAST($1):DECIMAL(10, 0), $2)]) - HiveJoin(condition=[=($0, $12)], joinType=[inner], algorithm=[none], cost=[not available]) - HiveJoin(condition=[AND(=($6, $9), =($7, $10), =($8, $11))], joinType=[inner], algorithm=[none], cost=[not available]) - HiveJoin(condition=[=($0, $5)], joinType=[inner], algorithm=[none], cost=[not available]) - HiveJoin(condition=[=($3, $4)], joinType=[inner], algorithm=[none], cost=[not available]) - HiveProject(cs_item_sk=[$14], cs_quantity=[$17], cs_list_price=[$19], cs_sold_date_sk=[$33]) - HiveFilter(condition=[IS NOT NULL($33)]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, catalog_sales]], table:alias=[catalog_sales]) - HiveProject(d_date_sk=[$0]) - HiveFilter(condition=[AND(=($6, 2000), =($8, 11))]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, date_dim]], table:alias=[date_dim]) - HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11]) - HiveFilter(condition=[AND(IS NOT NULL($11), IS NOT NULL($7), IS NOT NULL($9))]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, item]], table:alias=[item]) - HiveProject($f0=[$0], $f1=[$1], $f2=[$2]) - HiveFilter(condition=[=($3, 3)]) - HiveAggregate(group=[{0, 1, 2}], agg#0=[count($3)]) - HiveProject(brand_id=[$0], class_id=[$1], category_id=[$2], $f3=[$3]) - HiveUnion(all=[true]) - HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3]) - HiveAggregate(group=[{4, 5, 6}], agg#0=[count()]) - HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available]) - HiveJoin(condition=[=($1, $2)], joinType=[inner], algorithm=[none], cost=[not available]) - HiveProject(ss_item_sk=[$1], ss_sold_date_sk=[$22]) - HiveFilter(condition=[IS NOT NULL($22)]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, store_sales]], table:alias=[store_sales]) - HiveProject(d_date_sk=[$0]) - HiveFilter(condition=[BETWEEN(false, $6, 1999, 2001)]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, date_dim]], table:alias=[d1]) - HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11]) - HiveFilter(condition=[AND(IS NOT NULL($11), IS NOT NULL($7), IS NOT NULL($9))]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, item]], table:alias=[iss]) + HiveProject($f0=[$8], $f1=[$9], $f2=[$10], $f3=[*(CAST($1):DECIMAL(10, 0), $2)]) + HiveJoin(condition=[=($0, $7)], joinType=[inner], algorithm=[none], cost=[not available]) + HiveSemiJoin(condition=[=($0, $7)], joinType=[semi]) + HiveJoin(condition=[=($3, $4)], joinType=[inner], algorithm=[none], cost=[not available]) + HiveProject(cs_item_sk=[$14], cs_quantity=[$17], cs_list_price=[$19], cs_sold_date_sk=[$33]) + HiveFilter(condition=[IS NOT NULL($33)]) + HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales]) + HiveProject(d_date_sk=[$0], d_year=[CAST(2000):INTEGER], d_moy=[CAST(11):INTEGER]) + HiveFilter(condition=[AND(=($6, 2000), =($8, 11))]) + HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim]) + HiveProject(ss_item_sk=[$0]) + HiveJoin(condition=[AND(=($1, $4), =($2, $5), =($3, $6))], joinType=[inner], algorithm=[none], cost=[not available]) + HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11]) + HiveFilter(condition=[AND(IS NOT NULL($11), IS NOT NULL($7), IS NOT NULL($9))]) + HiveTableScan(table=[[default, item]], table:alias=[item]) + HiveProject($f0=[$0], $f1=[$1], $f2=[$2]) + HiveFilter(condition=[=($3, 3)]) + HiveAggregate(group=[{0, 1, 2}], agg#0=[count($3)]) HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3]) - HiveAggregate(group=[{4, 5, 6}], agg#0=[count()]) - HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available]) - HiveJoin(condition=[=($1, $2)], joinType=[inner], algorithm=[none], cost=[not available]) - HiveProject(cs_item_sk=[$14], cs_sold_date_sk=[$33]) - HiveFilter(condition=[IS NOT NULL($33)]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, catalog_sales]], table:alias=[catalog_sales]) - HiveProject(d_date_sk=[$0]) - HiveFilter(condition=[BETWEEN(false, $6, 1999, 2001)]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, date_dim]], table:alias=[d2]) - HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11]) - HiveFilter(condition=[AND(IS NOT NULL($11), IS NOT NULL($7), IS NOT NULL($9))]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, item]], table:alias=[ics]) - HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3]) - HiveAggregate(group=[{4, 5, 6}], agg#0=[count()]) - HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available]) - HiveJoin(condition=[=($1, $2)], joinType=[inner], algorithm=[none], cost=[not available]) - HiveProject(ws_item_sk=[$2], ws_sold_date_sk=[$33]) - HiveFilter(condition=[IS NOT NULL($33)]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, web_sales]], table:alias=[web_sales]) - HiveProject(d_date_sk=[$0]) - HiveFilter(condition=[BETWEEN(false, $6, 1999, 2001)]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, date_dim]], table:alias=[d3]) - HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11]) - HiveFilter(condition=[AND(IS NOT NULL($11), IS NOT NULL($7), IS NOT NULL($9))]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, item]], table:alias=[iws]) + HiveUnion(all=[true]) + HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3]) + HiveAggregate(group=[{4, 5, 6}], agg#0=[count()]) + HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available]) + HiveJoin(condition=[=($1, $2)], joinType=[inner], algorithm=[none], cost=[not available]) + HiveProject(ss_item_sk=[$1], ss_sold_date_sk=[$22]) + HiveFilter(condition=[IS NOT NULL($22)]) + HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales]) + HiveProject(d_date_sk=[$0]) + HiveFilter(condition=[BETWEEN(false, $6, 1999, 2001)]) + HiveTableScan(table=[[default, date_dim]], table:alias=[d1]) + HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11]) + HiveFilter(condition=[AND(IS NOT NULL($11), IS NOT NULL($7), IS NOT NULL($9))]) + HiveTableScan(table=[[default, item]], table:alias=[iss]) + HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3]) + HiveAggregate(group=[{4, 5, 6}], agg#0=[count()]) + HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available]) + HiveJoin(condition=[=($1, $2)], joinType=[inner], algorithm=[none], cost=[not available]) + HiveProject(cs_item_sk=[$14], cs_sold_date_sk=[$33]) + HiveFilter(condition=[IS NOT NULL($33)]) + HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales]) + HiveProject(d_date_sk=[$0]) + HiveFilter(condition=[BETWEEN(false, $6, 1999, 2001)]) + HiveTableScan(table=[[default, date_dim]], table:alias=[d2]) + HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11]) + HiveFilter(condition=[AND(IS NOT NULL($11), IS NOT NULL($7), IS NOT NULL($9))]) + HiveTableScan(table=[[default, item]], table:alias=[ics]) + HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3]) + HiveAggregate(group=[{4, 5, 6}], agg#0=[count()]) + HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available]) + HiveJoin(condition=[=($1, $2)], joinType=[inner], algorithm=[none], cost=[not available]) + HiveProject(ws_item_sk=[$2], ws_sold_date_sk=[$33]) + HiveFilter(condition=[IS NOT NULL($33)]) + HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales]) + HiveProject(d_date_sk=[$0]) + HiveFilter(condition=[BETWEEN(false, $6, 1999, 2001)]) + HiveTableScan(table=[[default, date_dim]], table:alias=[d3]) + HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11]) + HiveFilter(condition=[AND(IS NOT NULL($11), IS NOT NULL($7), IS NOT NULL($9))]) + HiveTableScan(table=[[default, item]], table:alias=[iws]) HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, item]], table:alias=[item]) - HiveProject($f0=[CAST(/($0, $1)):DECIMAL(22, 6)]) - HiveFilter(condition=[IS NOT NULL(CAST(/($0, $1)):DECIMAL(22, 6))]) - HiveProject($f0=[$0], $f1=[$1]) - HiveAggregate(group=[{}], agg#0=[sum($0)], agg#1=[count($0)]) - HiveProject($f0=[*(CAST($0):DECIMAL(10, 0), $1)]) - HiveUnion(all=[true]) - HiveProject(quantity=[$0], list_price=[$1]) - HiveJoin(condition=[=($2, $3)], joinType=[inner], algorithm=[none], cost=[not available]) - HiveProject(ss_quantity=[$9], ss_list_price=[$11], ss_sold_date_sk=[$22]) - HiveFilter(condition=[IS NOT NULL($22)]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, store_sales]], table:alias=[store_sales]) - HiveProject(d_date_sk=[$0]) - HiveFilter(condition=[BETWEEN(false, $6, 1999, 2001)]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, date_dim]], table:alias=[date_dim]) - HiveProject(quantity=[$0], list_price=[$1]) - HiveJoin(condition=[=($2, $3)], joinType=[inner], algorithm=[none], cost=[not available]) - HiveProject(cs_quantity=[$17], cs_list_price=[$19], cs_sold_date_sk=[$33]) - HiveFilter(condition=[IS NOT NULL($33)]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, catalog_sales]], table:alias=[catalog_sales]) - HiveProject(d_date_sk=[$0]) - HiveFilter(condition=[BETWEEN(false, $6, 1998, 2000)]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, date_dim]], table:alias=[date_dim]) - HiveProject(quantity=[$0], list_price=[$1]) - HiveJoin(condition=[=($2, $3)], joinType=[inner], algorithm=[none], cost=[not available]) - HiveProject(ws_quantity=[$17], ws_list_price=[$19], ws_sold_date_sk=[$33]) - HiveFilter(condition=[IS NOT NULL($33)]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, web_sales]], table:alias=[web_sales]) - HiveProject(d_date_sk=[$0]) - HiveFilter(condition=[BETWEEN(false, $6, 1998, 2000)]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, date_dim]], table:alias=[date_dim]) + HiveTableScan(table=[[default, item]], table:alias=[item]) + HiveJoin(condition=[true], joinType=[inner], algorithm=[none], cost=[not available]) + HiveProject(cnt=[$0]) + HiveFilter(condition=[sq_count_check($0)]) + HiveProject(cnt=[$0]) + HiveAggregate(group=[{}], cnt=[COUNT()]) + HiveTableScan(table=[[default, avg_sales]], table:alias=[avg_sales]) + HiveProject(average_sales=[$0]) + HiveFilter(condition=[IS NOT NULL($0)]) + HiveTableScan(table=[[default, avg_sales]], table:alias=[avg_sales]) HiveProject(channel=[_UTF-16LE'web':VARCHAR(2147483647) CHARACTER SET "UTF-16LE"], i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], sales=[$3], number_sales=[$4]) - HiveJoin(condition=[>($3, $5)], joinType=[inner], algorithm=[none], cost=[not available]) + HiveJoin(condition=[>($3, $6)], joinType=[inner], algorithm=[none], cost=[not available]) HiveProject($f0=[$0], $f1=[$1], $f2=[$2], $f3=[$3], $f4=[$4]) HiveFilter(condition=[IS NOT NULL($3)]) HiveAggregate(group=[{0, 1, 2}], agg#0=[sum($3)], agg#1=[count()]) - HiveProject($f0=[$13], $f1=[$14], $f2=[$15], $f3=[*(CAST($1):DECIMAL(10, 0), $2)]) - HiveJoin(condition=[=($0, $12)], joinType=[inner], algorithm=[none], cost=[not available]) - HiveJoin(condition=[AND(=($6, $9), =($7, $10), =($8, $11))], joinType=[inner], algorithm=[none], cost=[not available]) - HiveJoin(condition=[=($0, $5)], joinType=[inner], algorithm=[none], cost=[not available]) - HiveJoin(condition=[=($3, $4)], joinType=[inner], algorithm=[none], cost=[not available]) - HiveProject(ws_item_sk=[$2], ws_quantity=[$17], ws_list_price=[$19], ws_sold_date_sk=[$33]) - HiveFilter(condition=[IS NOT NULL($33)]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, web_sales]], table:alias=[web_sales]) - HiveProject(d_date_sk=[$0]) - HiveFilter(condition=[AND(=($6, 2000), =($8, 11))]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, date_dim]], table:alias=[date_dim]) - HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11]) - HiveFilter(condition=[AND(IS NOT NULL($11), IS NOT NULL($7), IS NOT NULL($9))]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, item]], table:alias=[item]) - HiveProject($f0=[$0], $f1=[$1], $f2=[$2]) - HiveFilter(condition=[=($3, 3)]) - HiveAggregate(group=[{0, 1, 2}], agg#0=[count($3)]) - HiveProject(brand_id=[$0], class_id=[$1], category_id=[$2], $f3=[$3]) - HiveUnion(all=[true]) - HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3]) - HiveAggregate(group=[{4, 5, 6}], agg#0=[count()]) - HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available]) - HiveJoin(condition=[=($1, $2)], joinType=[inner], algorithm=[none], cost=[not available]) - HiveProject(ss_item_sk=[$1], ss_sold_date_sk=[$22]) - HiveFilter(condition=[IS NOT NULL($22)]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, store_sales]], table:alias=[store_sales]) - HiveProject(d_date_sk=[$0]) - HiveFilter(condition=[BETWEEN(false, $6, 1999, 2001)]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, date_dim]], table:alias=[d1]) - HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11]) - HiveFilter(condition=[AND(IS NOT NULL($11), IS NOT NULL($7), IS NOT NULL($9))]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, item]], table:alias=[iss]) - HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3]) - HiveAggregate(group=[{4, 5, 6}], agg#0=[count()]) - HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available]) - HiveJoin(condition=[=($1, $2)], joinType=[inner], algorithm=[none], cost=[not available]) - HiveProject(cs_item_sk=[$14], cs_sold_date_sk=[$33]) - HiveFilter(condition=[IS NOT NULL($33)]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, catalog_sales]], table:alias=[catalog_sales]) - HiveProject(d_date_sk=[$0]) - HiveFilter(condition=[BETWEEN(false, $6, 1999, 2001)]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, date_dim]], table:alias=[d2]) - HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11]) - HiveFilter(condition=[AND(IS NOT NULL($11), IS NOT NULL($7), IS NOT NULL($9))]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, item]], table:alias=[ics]) + HiveProject($f0=[$8], $f1=[$9], $f2=[$10], $f3=[*(CAST($1):DECIMAL(10, 0), $2)]) + HiveJoin(condition=[=($0, $7)], joinType=[inner], algorithm=[none], cost=[not available]) + HiveSemiJoin(condition=[=($0, $7)], joinType=[semi]) + HiveJoin(condition=[=($3, $4)], joinType=[inner], algorithm=[none], cost=[not available]) + HiveProject(ws_item_sk=[$2], ws_quantity=[$17], ws_list_price=[$19], ws_sold_date_sk=[$33]) + HiveFilter(condition=[IS NOT NULL($33)]) + HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales]) + HiveProject(d_date_sk=[$0], d_year=[CAST(2000):INTEGER], d_moy=[CAST(11):INTEGER]) + HiveFilter(condition=[AND(=($6, 2000), =($8, 11))]) + HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim]) + HiveProject(ss_item_sk=[$0]) + HiveJoin(condition=[AND(=($1, $4), =($2, $5), =($3, $6))], joinType=[inner], algorithm=[none], cost=[not available]) + HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11]) + HiveFilter(condition=[AND(IS NOT NULL($11), IS NOT NULL($7), IS NOT NULL($9))]) + HiveTableScan(table=[[default, item]], table:alias=[item]) + HiveProject($f0=[$0], $f1=[$1], $f2=[$2]) + HiveFilter(condition=[=($3, 3)]) + HiveAggregate(group=[{0, 1, 2}], agg#0=[count($3)]) HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3]) - HiveAggregate(group=[{4, 5, 6}], agg#0=[count()]) - HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available]) - HiveJoin(condition=[=($1, $2)], joinType=[inner], algorithm=[none], cost=[not available]) - HiveProject(ws_item_sk=[$2], ws_sold_date_sk=[$33]) - HiveFilter(condition=[IS NOT NULL($33)]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, web_sales]], table:alias=[web_sales]) - HiveProject(d_date_sk=[$0]) - HiveFilter(condition=[BETWEEN(false, $6, 1999, 2001)]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, date_dim]], table:alias=[d3]) - HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11]) - HiveFilter(condition=[AND(IS NOT NULL($11), IS NOT NULL($7), IS NOT NULL($9))]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, item]], table:alias=[iws]) + HiveUnion(all=[true]) + HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3]) + HiveAggregate(group=[{4, 5, 6}], agg#0=[count()]) + HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available]) + HiveJoin(condition=[=($1, $2)], joinType=[inner], algorithm=[none], cost=[not available]) + HiveProject(ss_item_sk=[$1], ss_sold_date_sk=[$22]) + HiveFilter(condition=[IS NOT NULL($22)]) + HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales]) + HiveProject(d_date_sk=[$0]) + HiveFilter(condition=[BETWEEN(false, $6, 1999, 2001)]) + HiveTableScan(table=[[default, date_dim]], table:alias=[d1]) + HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11]) + HiveFilter(condition=[AND(IS NOT NULL($11), IS NOT NULL($7), IS NOT NULL($9))]) + HiveTableScan(table=[[default, item]], table:alias=[iss]) + HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3]) + HiveAggregate(group=[{4, 5, 6}], agg#0=[count()]) + HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available]) + HiveJoin(condition=[=($1, $2)], joinType=[inner], algorithm=[none], cost=[not available]) + HiveProject(cs_item_sk=[$14], cs_sold_date_sk=[$33]) + HiveFilter(condition=[IS NOT NULL($33)]) + HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales]) + HiveProject(d_date_sk=[$0]) + HiveFilter(condition=[BETWEEN(false, $6, 1999, 2001)]) + HiveTableScan(table=[[default, date_dim]], table:alias=[d2]) + HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11]) + HiveFilter(condition=[AND(IS NOT NULL($11), IS NOT NULL($7), IS NOT NULL($9))]) + HiveTableScan(table=[[default, item]], table:alias=[ics]) + HiveProject(i_brand_id=[$0], i_class_id=[$1], i_category_id=[$2], $f3=[$3]) + HiveAggregate(group=[{4, 5, 6}], agg#0=[count()]) + HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available]) + HiveJoin(condition=[=($1, $2)], joinType=[inner], algorithm=[none], cost=[not available]) + HiveProject(ws_item_sk=[$2], ws_sold_date_sk=[$33]) + HiveFilter(condition=[IS NOT NULL($33)]) + HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales]) + HiveProject(d_date_sk=[$0]) + HiveFilter(condition=[BETWEEN(false, $6, 1999, 2001)]) + HiveTableScan(table=[[default, date_dim]], table:alias=[d3]) + HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11]) + HiveFilter(condition=[AND(IS NOT NULL($11), IS NOT NULL($7), IS NOT NULL($9))]) + HiveTableScan(table=[[default, item]], table:alias=[iws]) HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, item]], table:alias=[item]) - HiveProject($f0=[CAST(/($0, $1)):DECIMAL(22, 6)]) - HiveFilter(condition=[IS NOT NULL(CAST(/($0, $1)):DECIMAL(22, 6))]) - HiveProject($f0=[$0], $f1=[$1]) - HiveAggregate(group=[{}], agg#0=[sum($0)], agg#1=[count($0)]) - HiveProject($f0=[*(CAST($0):DECIMAL(10, 0), $1)]) - HiveUnion(all=[true]) - HiveProject(quantity=[$0], list_price=[$1]) - HiveJoin(condition=[=($2, $3)], joinType=[inner], algorithm=[none], cost=[not available]) - HiveProject(ss_quantity=[$9], ss_list_price=[$11], ss_sold_date_sk=[$22]) - HiveFilter(condition=[IS NOT NULL($22)]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, store_sales]], table:alias=[store_sales]) - HiveProject(d_date_sk=[$0]) - HiveFilter(condition=[BETWEEN(false, $6, 1999, 2001)]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, date_dim]], table:alias=[date_dim]) - HiveProject(quantity=[$0], list_price=[$1]) - HiveJoin(condition=[=($2, $3)], joinType=[inner], algorithm=[none], cost=[not available]) - HiveProject(cs_quantity=[$17], cs_list_price=[$19], cs_sold_date_sk=[$33]) - HiveFilter(condition=[IS NOT NULL($33)]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, catalog_sales]], table:alias=[catalog_sales]) - HiveProject(d_date_sk=[$0]) - HiveFilter(condition=[BETWEEN(false, $6, 1998, 2000)]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, date_dim]], table:alias=[date_dim]) - HiveProject(quantity=[$0], list_price=[$1]) - HiveJoin(condition=[=($2, $3)], joinType=[inner], algorithm=[none], cost=[not available]) - HiveProject(ws_quantity=[$17], ws_list_price=[$19], ws_sold_date_sk=[$33]) - HiveFilter(condition=[IS NOT NULL($33)]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, web_sales]], table:alias=[web_sales]) - HiveProject(d_date_sk=[$0]) - HiveFilter(condition=[BETWEEN(false, $6, 1998, 2000)]) - HiveTableScan(table=[[tpcds_bin_partitioned_orc_30000, date_dim]], table:alias=[date_dim]) + HiveTableScan(table=[[default, item]], table:alias=[item]) + HiveJoin(condition=[true], joinType=[inner], algorithm=[none], cost=[not available]) + HiveProject(cnt=[$0]) + HiveFilter(condition=[sq_count_check($0)]) + HiveProject(cnt=[$0]) + HiveAggregate(group=[{}], cnt=[COUNT()]) + HiveTableScan(table=[[default, avg_sales]], table:alias=[avg_sales]) + HiveProject(average_sales=[$0]) + HiveFilter(condition=[IS NOT NULL($0)]) + HiveTableScan(table=[[default, avg_sales]], table:alias=[avg_sales]) diff --git a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query14.q.out b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query14.q.out index bec076e30e28..84bd82249940 100644 --- a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query14.q.out +++ b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/query14.q.out @@ -1,582 +1,1247 @@ -Warning: Map Join MAPJOIN[1133][bigTable=?] in task 'Reducer 2' is a cross product -Warning: Map Join MAPJOIN[1398][bigTable=?] in task 'Reducer 35' is a cross product -Warning: Map Join MAPJOIN[1492][bigTable=?] in task 'Reducer 42' is a cross product -Plan optimized by CBO. +Warning: Map Join MAPJOIN[1098][bigTable=?] in task 'Reducer 10' is a cross product +Warning: Map Join MAPJOIN[1147][bigTable=?] in task 'Reducer 15' is a cross product +Warning: Map Join MAPJOIN[1163][bigTable=?] in task 'Reducer 21' is a cross product +Warning: Map Join MAPJOIN[1199][bigTable=?] in task 'Reducer 30' is a cross product +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-4 depends on stages: Stage-2, Stage-0 + Stage-0 depends on stages: Stage-1 + Stage-3 depends on stages: Stage-4 -Vertex dependency in root stage -Map 1 <- Map 26 (BROADCAST_EDGE), Map 6 (BROADCAST_EDGE), Map 7 (BROADCAST_EDGE), Reducer 11 (BROADCAST_EDGE) -Map 18 <- Map 6 (BROADCAST_EDGE), Map 7 (BROADCAST_EDGE) -Map 22 <- Map 6 (BROADCAST_EDGE), Map 7 (BROADCAST_EDGE) -Map 27 <- Map 6 (BROADCAST_EDGE), Union 28 (CONTAINS) -Map 30 <- Map 31 (BROADCAST_EDGE), Union 28 (CONTAINS) -Map 32 <- Map 33 (BROADCAST_EDGE), Union 28 (CONTAINS) -Map 34 <- Map 26 (BROADCAST_EDGE), Map 6 (BROADCAST_EDGE), Map 7 (BROADCAST_EDGE), Reducer 14 (BROADCAST_EDGE) -Map 36 <- Map 6 (BROADCAST_EDGE), Union 37 (CONTAINS) -Map 39 <- Map 31 (BROADCAST_EDGE), Union 37 (CONTAINS) -Map 40 <- Map 33 (BROADCAST_EDGE), Union 37 (CONTAINS) -Map 41 <- Map 26 (BROADCAST_EDGE), Map 6 (BROADCAST_EDGE), Map 7 (BROADCAST_EDGE), Reducer 17 (BROADCAST_EDGE) -Map 43 <- Map 6 (BROADCAST_EDGE), Union 44 (CONTAINS) -Map 46 <- Map 31 (BROADCAST_EDGE), Union 44 (CONTAINS) -Map 47 <- Map 33 (BROADCAST_EDGE), Union 44 (CONTAINS) -Map 8 <- Map 6 (BROADCAST_EDGE), Map 7 (BROADCAST_EDGE) -Reducer 11 <- Union 10 (SIMPLE_EDGE) -Reducer 12 <- Map 8 (SIMPLE_EDGE), Union 13 (CONTAINS) -Reducer 14 <- Union 13 (SIMPLE_EDGE) -Reducer 15 <- Map 8 (SIMPLE_EDGE), Union 16 (CONTAINS) -Reducer 17 <- Union 16 (SIMPLE_EDGE) -Reducer 19 <- Map 18 (SIMPLE_EDGE), Union 10 (CONTAINS) -Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 29 (BROADCAST_EDGE), Union 3 (CONTAINS) -Reducer 20 <- Map 18 (SIMPLE_EDGE), Union 13 (CONTAINS) -Reducer 21 <- Map 18 (SIMPLE_EDGE), Union 16 (CONTAINS) -Reducer 23 <- Map 22 (SIMPLE_EDGE), Union 10 (CONTAINS) -Reducer 24 <- Map 22 (SIMPLE_EDGE), Union 13 (CONTAINS) -Reducer 25 <- Map 22 (SIMPLE_EDGE), Union 16 (CONTAINS) -Reducer 29 <- Union 28 (CUSTOM_SIMPLE_EDGE) -Reducer 35 <- Map 34 (SIMPLE_EDGE), Reducer 38 (BROADCAST_EDGE), Union 3 (CONTAINS) -Reducer 38 <- Union 37 (CUSTOM_SIMPLE_EDGE) -Reducer 4 <- Union 3 (SIMPLE_EDGE) -Reducer 42 <- Map 41 (SIMPLE_EDGE), Reducer 45 (BROADCAST_EDGE), Union 3 (CONTAINS) -Reducer 45 <- Union 44 (CUSTOM_SIMPLE_EDGE) -Reducer 5 <- Reducer 4 (SIMPLE_EDGE) -Reducer 9 <- Map 8 (SIMPLE_EDGE), Union 10 (CONTAINS) +STAGE PLANS: + Stage: Stage-1 + Tez +#### A masked pattern was here #### + Edges: + Map 1 <- Map 4 (BROADCAST_EDGE), Union 2 (CONTAINS) + Map 7 <- Reducer 5 (BROADCAST_EDGE), Union 2 (CONTAINS) + Map 8 <- Reducer 6 (BROADCAST_EDGE), Union 2 (CONTAINS) + Reducer 3 <- Union 2 (CUSTOM_SIMPLE_EDGE) + Reducer 5 <- Map 4 (SIMPLE_EDGE) + Reducer 6 <- Map 4 (SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: store_sales + filterExpr: ss_sold_date_sk is not null (type: boolean) + Statistics: Num rows: 82510879939 Data size: 10005709976020 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: ss_quantity (type: int), ss_list_price (type: decimal(7,2)), ss_sold_date_sk (type: bigint) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 82510879939 Data size: 10005709976020 Basic stats: COMPLETE Column stats: COMPLETE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col2 (type: bigint) + 1 _col0 (type: bigint) + outputColumnNames: _col0, _col1 + input vertices: + 1 Map 4 + Statistics: Num rows: 49749852010 Data size: 5545343696744 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: (CAST( _col0 AS decimal(10,0)) * _col1) (type: decimal(18,2)) + outputColumnNames: _col0 + Statistics: Num rows: 88516906238 Data size: 10029566262388 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: sum(_col0), count(_col0) + minReductionHashAggr: 0.99 + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 120 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 120 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: decimal(28,2)), _col1 (type: bigint) + Execution mode: vectorized, llap + LLAP IO: may be used (ACID table) + Map 4 + Map Operator Tree: + TableScan + alias: date_dim + filterExpr: (d_year BETWEEN 1999 AND 2001 or d_year BETWEEN 1998 AND 2000) (type: boolean) + Statistics: Num rows: 73049 Data size: 876588 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: d_year BETWEEN 1999 AND 2001 (type: boolean) + Statistics: Num rows: 1101 Data size: 13212 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: d_date_sk (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1101 Data size: 8808 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: bigint) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: bigint) + Statistics: Num rows: 1101 Data size: 8808 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1101 Data size: 8808 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: bigint) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1101 Data size: 8808 Basic stats: COMPLETE Column stats: COMPLETE + Dynamic Partitioning Event Operator + Target column: ss_sold_date_sk (bigint) + Target Input: store_sales + Partition key expr: ss_sold_date_sk + Statistics: Num rows: 1101 Data size: 8808 Basic stats: COMPLETE Column stats: COMPLETE + Target Vertex: Map 1 + Filter Operator + predicate: d_year BETWEEN 1998 AND 2000 (type: boolean) + Statistics: Num rows: 1101 Data size: 13212 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: d_date_sk (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1101 Data size: 8808 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: bigint) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: bigint) + Statistics: Num rows: 1101 Data size: 8808 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1101 Data size: 8808 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: bigint) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1101 Data size: 8808 Basic stats: COMPLETE Column stats: COMPLETE + Dynamic Partitioning Event Operator + Target column: cs_sold_date_sk (bigint) + Target Input: catalog_sales + Partition key expr: cs_sold_date_sk + Statistics: Num rows: 1101 Data size: 8808 Basic stats: COMPLETE Column stats: COMPLETE + Target Vertex: Map 7 + Dynamic Partitioning Event Operator + Target column: ws_sold_date_sk (bigint) + Target Input: web_sales + Partition key expr: ws_sold_date_sk + Statistics: Num rows: 1101 Data size: 8808 Basic stats: COMPLETE Column stats: COMPLETE + Target Vertex: Map 8 + Reduce Output Operator + key expressions: _col0 (type: bigint) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: bigint) + Statistics: Num rows: 1101 Data size: 8808 Basic stats: COMPLETE Column stats: COMPLETE + Execution mode: vectorized, llap + LLAP IO: may be used (ACID table) + Map 7 + Map Operator Tree: + TableScan + alias: catalog_sales + filterExpr: cs_sold_date_sk is not null (type: boolean) + Statistics: Num rows: 43005109025 Data size: 5320191433036 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: cs_quantity (type: int), cs_list_price (type: decimal(7,2)), cs_sold_date_sk (type: bigint) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 43005109025 Data size: 5320191433036 Basic stats: COMPLETE Column stats: COMPLETE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col2 (type: bigint) + 1 _col0 (type: bigint) + outputColumnNames: _col0, _col1 + input vertices: + 1 Reducer 5 + Statistics: Num rows: 25746588712 Data size: 2974162204528 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: (CAST( _col0 AS decimal(10,0)) * _col1) (type: decimal(18,2)) + outputColumnNames: _col0 + Statistics: Num rows: 88516906238 Data size: 10029566262388 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: sum(_col0), count(_col0) + minReductionHashAggr: 0.99 + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 120 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 120 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: decimal(28,2)), _col1 (type: bigint) + Execution mode: vectorized, llap + LLAP IO: may be used (ACID table) + Map 8 + Map Operator Tree: + TableScan + alias: web_sales + filterExpr: ws_sold_date_sk is not null (type: boolean) + Statistics: Num rows: 21594638446 Data size: 2677421528564 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: ws_quantity (type: int), ws_list_price (type: decimal(7,2)), ws_sold_date_sk (type: bigint) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 21594638446 Data size: 2677421528564 Basic stats: COMPLETE Column stats: COMPLETE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col2 (type: bigint) + 1 _col0 (type: bigint) + outputColumnNames: _col0, _col1 + input vertices: + 1 Reducer 6 + Statistics: Num rows: 13020465516 Data size: 1510060361116 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: (CAST( _col0 AS decimal(10,0)) * _col1) (type: decimal(18,2)) + outputColumnNames: _col0 + Statistics: Num rows: 88516906238 Data size: 10029566262388 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: sum(_col0), count(_col0) + minReductionHashAggr: 0.99 + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 120 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 120 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: decimal(28,2)), _col1 (type: bigint) + Execution mode: vectorized, llap + LLAP IO: may be used (ACID table) + Reducer 3 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0), count(VALUE._col1) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 120 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: CAST( (_col0 / _col1) AS decimal(22,6)) (type: decimal(22,6)) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat + output format: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat + serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde + name: default.avg_sales + Reducer 5 + Execution mode: vectorized, llap + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: bigint) + outputColumnNames: _col0 + Reduce Output Operator + key expressions: _col0 (type: bigint) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: bigint) + Statistics: Num rows: 1101 Data size: 8808 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 6 + Execution mode: vectorized, llap + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: bigint) + outputColumnNames: _col0 + Reduce Output Operator + key expressions: _col0 (type: bigint) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: bigint) + Statistics: Num rows: 1101 Data size: 8808 Basic stats: COMPLETE Column stats: COMPLETE + Union 2 + Vertex: Union 2 -Stage-0 - Fetch Operator - limit:100 - Stage-1 - Reducer 5 vectorized, llap - File Output Operator [FS_1649] - Limit [LIM_1648] (rows=100 width=223) - Number of rows:100 - Select Operator [SEL_1647] (rows=398760 width=222) - Output:["_col0","_col1","_col2","_col3","_col4","_col5"] - <-Reducer 4 [SIMPLE_EDGE] vectorized, llap - SHUFFLE [RS_1646] - Select Operator [SEL_1645] (rows=398760 width=222) - Output:["_col0","_col1","_col2","_col3","_col4","_col5"] - Group By Operator [GBY_1644] (rows=398760 width=230) - Output:["_col0","_col1","_col2","_col3","_col5","_col6"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4 - <-Union 3 [SIMPLE_EDGE] - <-Reducer 2 [CONTAINS] vectorized, llap - Reduce Output Operator [RS_1643] - PartitionCols:_col0, _col1, _col2, _col3, _col4 - Group By Operator [GBY_1642] (rows=398760 width=230) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"],aggregations:["sum(_col4)","sum(_col5)"],keys:_col0, _col1, _col2, _col3, 0L - Top N Key Operator [TNK_1641] (rows=159504 width=221) - keys:_col0, _col1, _col2, _col3,top n:100 - Select Operator [SEL_1640] (rows=60027 width=220) - Output:["_col0","_col1","_col2","_col3","_col4","_col5"] - Filter Operator [FIL_1639] (rows=60027 width=243) - predicate:(_col3 > _col5) - Map Join Operator [MAPJOIN_1638] (rows=180081 width=243) - Conds:(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col5"] - <-Reducer 29 [BROADCAST_EDGE] vectorized, llap - BROADCAST [RS_1635] - Select Operator [SEL_1634] (rows=1 width=112) - Output:["_col0"] - Filter Operator [FIL_1633] (rows=1 width=120) - predicate:CAST( (_col0 / _col1) AS decimal(22,6)) is not null - Group By Operator [GBY_1632] (rows=1 width=120) - Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)"] - <-Union 28 [CUSTOM_SIMPLE_EDGE] - <-Map 27 [CONTAINS] vectorized, llap - Reduce Output Operator [RS_1702] - Group By Operator [GBY_1701] (rows=1 width=120) - Output:["_col0","_col1"],aggregations:["sum(_col0)","count(_col0)"] - Select Operator [SEL_1700] (rows=88516906238 width=113) - Output:["_col0"] - Map Join Operator [MAPJOIN_1699] (rows=49749852010 width=111) - Conds:SEL_1698._col2=RS_1584._col0(Inner),Output:["_col0","_col1"] - <-Map 6 [BROADCAST_EDGE] vectorized, llap - BROADCAST [RS_1584] - PartitionCols:_col0 - Select Operator [SEL_1571] (rows=1101 width=8) - Output:["_col0"] - Filter Operator [FIL_1569] (rows=1101 width=12) - predicate:d_year BETWEEN 1999 AND 2001 - TableScan [TS_3] (rows=73049 width=16) - tpcds_bin_partitioned_orc_30000@date_dim,date_dim, ACID table,Tbl:COMPLETE,Col:COMPLETE,Output:["d_date_sk","d_year","d_moy"] - <-Select Operator [SEL_1698] (rows=82510879939 width=121) - Output:["_col0","_col1","_col2"] - TableScan [TS_1311] (rows=82510879939 width=121) - Output:["ss_quantity","ss_list_price"] - <-Map 30 [CONTAINS] vectorized, llap - Reduce Output Operator [RS_1721] - Group By Operator [GBY_1720] (rows=1 width=120) - Output:["_col0","_col1"],aggregations:["sum(_col0)","count(_col0)"] - Select Operator [SEL_1719] (rows=88516906238 width=113) - Output:["_col0"] - Map Join Operator [MAPJOIN_1718] (rows=25746588712 width=115) - Conds:SEL_1717._col2=RS_1705._col0(Inner),Output:["_col0","_col1"] - <-Map 31 [BROADCAST_EDGE] vectorized, llap - BROADCAST [RS_1705] - PartitionCols:_col0 - Select Operator [SEL_1704] (rows=1101 width=8) - Output:["_col0"] - Filter Operator [FIL_1703] (rows=1101 width=12) - predicate:d_year BETWEEN 1998 AND 2000 - TableScan [TS_111] (rows=73049 width=12) - tpcds_bin_partitioned_orc_30000@date_dim,date_dim, ACID table,Tbl:COMPLETE,Col:COMPLETE,Output:["d_date_sk","d_year"] - <-Select Operator [SEL_1717] (rows=43005109025 width=123) - Output:["_col0","_col1","_col2"] - TableScan [TS_1334] (rows=43005109025 width=123) - Output:["cs_quantity","cs_list_price"] - <-Map 32 [CONTAINS] vectorized, llap - Reduce Output Operator [RS_1740] - Group By Operator [GBY_1739] (rows=1 width=120) - Output:["_col0","_col1"],aggregations:["sum(_col0)","count(_col0)"] - Select Operator [SEL_1738] (rows=88516906238 width=113) - Output:["_col0"] - Map Join Operator [MAPJOIN_1737] (rows=13020465516 width=115) - Conds:SEL_1736._col2=RS_1724._col0(Inner),Output:["_col0","_col1"] - <-Map 33 [BROADCAST_EDGE] vectorized, llap - BROADCAST [RS_1724] - PartitionCols:_col0 - Select Operator [SEL_1723] (rows=1101 width=8) - Output:["_col0"] - Filter Operator [FIL_1722] (rows=1101 width=12) - predicate:d_year BETWEEN 1998 AND 2000 - TableScan [TS_122] (rows=73049 width=12) - tpcds_bin_partitioned_orc_30000@date_dim,date_dim, ACID table,Tbl:COMPLETE,Col:COMPLETE,Output:["d_date_sk","d_year"] - <-Select Operator [SEL_1736] (rows=21594638446 width=123) - Output:["_col0","_col1","_col2"] - TableScan [TS_1357] (rows=21594638446 width=123) - Output:["ws_quantity","ws_list_price"] - <-Filter Operator [FIL_1637] (rows=180081 width=131) - predicate:_col3 is not null - Group By Operator [GBY_1636] (rows=180081 width=131) - Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)"],keys:KEY._col0, KEY._col1, KEY._col2 - <-Map 1 [SIMPLE_EDGE] vectorized, llap - SHUFFLE [RS_1631] - PartitionCols:_col0, _col1, _col2 - Group By Operator [GBY_1630] (rows=180081 width=131) - Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["sum(_col3)","count()"],keys:_col0, _col1, _col2 - Select Operator [SEL_1629] (rows=1445421 width=11) - Output:["_col0","_col1","_col2","_col3"] - Map Join Operator [MAPJOIN_1628] (rows=1445421 width=11) - Conds:MAPJOIN_1627._col0=RS_1621._col0(Inner),Output:["_col1","_col2","_col13","_col14","_col15"] - <-Map 26 [BROADCAST_EDGE] vectorized, llap - BROADCAST [RS_1621] - PartitionCols:_col0 - Select Operator [SEL_1620] (rows=462000 width=19) - Output:["_col0","_col1","_col2","_col3"] - TableScan [TS_77] (rows=462000 width=19) - tpcds_bin_partitioned_orc_30000@item,item, ACID table,Tbl:COMPLETE,Col:COMPLETE,Output:["i_item_sk","i_brand_id","i_class_id","i_category_id"] - <-Map Join Operator [MAPJOIN_1627] (rows=1445421 width=8) - Conds:MAPJOIN_1626._col6, _col7, _col8=RS_1619._col0, _col1, _col2(Inner),Output:["_col0","_col1","_col2"] - <-Reducer 11 [BROADCAST_EDGE] vectorized, llap - BROADCAST [RS_1619] - PartitionCols:_col0, _col1, _col2 - Select Operator [SEL_1618] (rows=1 width=12) - Output:["_col0","_col1","_col2"] - Filter Operator [FIL_1617] (rows=1 width=20) - predicate:(_col3 = 3L) - Group By Operator [GBY_1616] (rows=153920 width=20) - Output:["_col0","_col1","_col2","_col3"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2 - <-Union 10 [SIMPLE_EDGE] - <-Reducer 19 [CONTAINS] vectorized, llap - Reduce Output Operator [RS_1675] - PartitionCols:_col0, _col1, _col2 - Group By Operator [GBY_1674] (rows=153920 width=20) - Output:["_col0","_col1","_col2","_col3"],aggregations:["count(_col3)"],keys:_col0, _col1, _col2 - Group By Operator [GBY_1673] (rows=180081 width=20) - Output:["_col0","_col1","_col2","_col3"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2 - <-Map 18 [SIMPLE_EDGE] vectorized, llap - SHUFFLE [RS_1670] - PartitionCols:_col0, _col1, _col2 - Group By Operator [GBY_1669] (rows=215917119 width=19) - Output:["_col0","_col1","_col2","_col3"],aggregations:["count()"],keys:_col4, _col5, _col6 - Map Join Operator [MAPJOIN_1668] (rows=25557780268 width=11) - Conds:MAPJOIN_1667._col0=RS_1612._col0(Inner),Output:["_col4","_col5","_col6"] - <-Map 7 [BROADCAST_EDGE] vectorized, llap - BROADCAST [RS_1612] - PartitionCols:_col0 - Select Operator [SEL_1609] (rows=458612 width=19) - Output:["_col0","_col1","_col2","_col3"] - Filter Operator [FIL_1608] (rows=458612 width=19) - predicate:(i_category_id is not null and i_brand_id is not null and i_class_id is not null) - TableScan [TS_6] (rows=462000 width=19) - tpcds_bin_partitioned_orc_30000@item,item, ACID table,Tbl:COMPLETE,Col:COMPLETE,Output:["i_item_sk","i_brand_id","i_class_id","i_category_id"] - <-Map Join Operator [MAPJOIN_1667] (rows=25746588712 width=8) - Conds:SEL_1666._col1=RS_1580._col0(Inner),Output:["_col0"] - <-Map 6 [BROADCAST_EDGE] vectorized, llap - BROADCAST [RS_1580] - PartitionCols:_col0 - Please refer to the previous Select Operator [SEL_1571] - <-Select Operator [SEL_1666] (rows=43005109025 width=16) - Output:["_col0","_col1"] - TableScan [TS_29] (rows=43005109025 width=16) - tpcds_bin_partitioned_orc_30000@catalog_sales,catalog_sales, ACID table,Tbl:COMPLETE,Col:COMPLETE,Output:["cs_item_sk"] - <-Reducer 23 [CONTAINS] vectorized, llap - Reduce Output Operator [RS_1691] - PartitionCols:_col0, _col1, _col2 - Group By Operator [GBY_1690] (rows=153920 width=20) - Output:["_col0","_col1","_col2","_col3"],aggregations:["count(_col3)"],keys:_col0, _col1, _col2 - Group By Operator [GBY_1689] (rows=180081 width=20) - Output:["_col0","_col1","_col2","_col3"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2 - <-Map 22 [SIMPLE_EDGE] vectorized, llap - SHUFFLE [RS_1686] - PartitionCols:_col0, _col1, _col2 - Group By Operator [GBY_1685] (rows=109129086 width=19) - Output:["_col0","_col1","_col2","_col3"],aggregations:["count()"],keys:_col4, _col5, _col6 - Map Join Operator [MAPJOIN_1684] (rows=12924982039 width=11) - Conds:MAPJOIN_1683._col0=RS_1613._col0(Inner),Output:["_col4","_col5","_col6"] - <-Map 7 [BROADCAST_EDGE] vectorized, llap - BROADCAST [RS_1613] - PartitionCols:_col0 - Please refer to the previous Select Operator [SEL_1609] - <-Map Join Operator [MAPJOIN_1683] (rows=13020465516 width=8) - Conds:SEL_1682._col1=RS_1582._col0(Inner),Output:["_col0"] - <-Map 6 [BROADCAST_EDGE] vectorized, llap - BROADCAST [RS_1582] - PartitionCols:_col0 - Please refer to the previous Select Operator [SEL_1571] - <-Select Operator [SEL_1682] (rows=21594638446 width=16) - Output:["_col0","_col1"] - TableScan [TS_50] (rows=21594638446 width=16) - tpcds_bin_partitioned_orc_30000@web_sales,web_sales, ACID table,Tbl:COMPLETE,Col:COMPLETE,Output:["ws_item_sk"] - <-Reducer 9 [CONTAINS] vectorized, llap - Reduce Output Operator [RS_1659] - PartitionCols:_col0, _col1, _col2 - Group By Operator [GBY_1658] (rows=153920 width=20) - Output:["_col0","_col1","_col2","_col3"],aggregations:["count(_col3)"],keys:_col0, _col1, _col2 - Group By Operator [GBY_1657] (rows=180081 width=20) - Output:["_col0","_col1","_col2","_col3"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2 - <-Map 8 [SIMPLE_EDGE] vectorized, llap - SHUFFLE [RS_1654] - PartitionCols:_col0, _col1, _col2 - Group By Operator [GBY_1653] (rows=416887515 width=19) - Output:["_col0","_col1","_col2","_col3"],aggregations:["count()"],keys:_col4, _col5, _col6 - Map Join Operator [MAPJOIN_1652] (rows=49385019517 width=11) - Conds:MAPJOIN_1651._col0=RS_1611._col0(Inner),Output:["_col4","_col5","_col6"] - <-Map 7 [BROADCAST_EDGE] vectorized, llap - BROADCAST [RS_1611] - PartitionCols:_col0 - Please refer to the previous Select Operator [SEL_1609] - <-Map Join Operator [MAPJOIN_1651] (rows=49749852010 width=8) - Conds:SEL_1650._col1=RS_1578._col0(Inner),Output:["_col0"] - <-Map 6 [BROADCAST_EDGE] vectorized, llap - BROADCAST [RS_1578] - PartitionCols:_col0 - Please refer to the previous Select Operator [SEL_1571] - <-Select Operator [SEL_1650] (rows=82510879939 width=16) - Output:["_col0","_col1"] - TableScan [TS_9] (rows=82510879939 width=16) - tpcds_bin_partitioned_orc_30000@store_sales,store_sales, ACID table,Tbl:COMPLETE,Col:COMPLETE,Output:["ss_item_sk"] - <-Map Join Operator [MAPJOIN_1626] (rows=1390495544 width=19) - Conds:MAPJOIN_1625._col0=RS_1610._col0(Inner),Output:["_col0","_col1","_col2","_col6","_col7","_col8"] - <-Map 7 [BROADCAST_EDGE] vectorized, llap - BROADCAST [RS_1610] - PartitionCols:_col0 - Please refer to the previous Select Operator [SEL_1609] - <-Map Join Operator [MAPJOIN_1625] (rows=1400767848 width=8) - Conds:SEL_1624._col3=RS_1572._col0(Inner),Output:["_col0","_col1","_col2"] - <-Map 6 [BROADCAST_EDGE] vectorized, llap - BROADCAST [RS_1572] - PartitionCols:_col0 - Select Operator [SEL_1570] (rows=31 width=8) - Output:["_col0"] - Filter Operator [FIL_1568] (rows=31 width=16) - predicate:((d_year = 2000) and (d_moy = 11)) - Please refer to the previous TableScan [TS_3] - <-Select Operator [SEL_1624] (rows=82510879939 width=129) - Output:["_col0","_col1","_col2","_col3"] - TableScan [TS_0] (rows=82510879939 width=129) - tpcds_bin_partitioned_orc_30000@store_sales,store_sales, ACID table,Tbl:COMPLETE,Col:COMPLETE,Output:["ss_item_sk","ss_quantity","ss_list_price"] - <-Reducer 35 [CONTAINS] vectorized, llap - Reduce Output Operator [RS_1764] - PartitionCols:_col0, _col1, _col2, _col3, _col4 - Group By Operator [GBY_1763] (rows=398760 width=230) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"],aggregations:["sum(_col4)","sum(_col5)"],keys:_col0, _col1, _col2, _col3, 0L - Top N Key Operator [TNK_1762] (rows=159504 width=221) - keys:_col0, _col1, _col2, _col3,top n:100 - Select Operator [SEL_1761] (rows=60027 width=222) - Output:["_col0","_col1","_col2","_col3","_col4","_col5"] - Filter Operator [FIL_1760] (rows=60027 width=243) - predicate:(_col3 > _col5) - Map Join Operator [MAPJOIN_1759] (rows=180081 width=243) - Conds:(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col5"] - <-Reducer 38 [BROADCAST_EDGE] vectorized, llap - BROADCAST [RS_1756] - Select Operator [SEL_1755] (rows=1 width=112) - Output:["_col0"] - Filter Operator [FIL_1754] (rows=1 width=120) - predicate:CAST( (_col0 / _col1) AS decimal(22,6)) is not null - Group By Operator [GBY_1753] (rows=1 width=120) - Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)"] - <-Union 37 [CUSTOM_SIMPLE_EDGE] - <-Map 36 [CONTAINS] vectorized, llap - Reduce Output Operator [RS_1769] - Group By Operator [GBY_1768] (rows=1 width=120) - Output:["_col0","_col1"],aggregations:["sum(_col0)","count(_col0)"] - Select Operator [SEL_1767] (rows=88516906238 width=113) - Output:["_col0"] - Map Join Operator [MAPJOIN_1766] (rows=49749852010 width=111) - Conds:SEL_1765._col2=RS_1586._col0(Inner),Output:["_col0","_col1"] - <-Map 6 [BROADCAST_EDGE] vectorized, llap - BROADCAST [RS_1586] - PartitionCols:_col0 - Please refer to the previous Select Operator [SEL_1571] - <-Select Operator [SEL_1765] (rows=82510879939 width=121) - Output:["_col0","_col1","_col2"] - TableScan [TS_1405] (rows=82510879939 width=121) - Output:["ss_quantity","ss_list_price"] - <-Map 39 [CONTAINS] vectorized, llap - Reduce Output Operator [RS_1774] - Group By Operator [GBY_1773] (rows=1 width=120) - Output:["_col0","_col1"],aggregations:["sum(_col0)","count(_col0)"] - Select Operator [SEL_1772] (rows=88516906238 width=113) - Output:["_col0"] - Map Join Operator [MAPJOIN_1771] (rows=25746588712 width=115) - Conds:SEL_1770._col2=RS_1707._col0(Inner),Output:["_col0","_col1"] - <-Map 31 [BROADCAST_EDGE] vectorized, llap - BROADCAST [RS_1707] - PartitionCols:_col0 - Please refer to the previous Select Operator [SEL_1704] - <-Select Operator [SEL_1770] (rows=43005109025 width=123) - Output:["_col0","_col1","_col2"] - TableScan [TS_1428] (rows=43005109025 width=123) - Output:["cs_quantity","cs_list_price"] - <-Map 40 [CONTAINS] vectorized, llap - Reduce Output Operator [RS_1779] - Group By Operator [GBY_1778] (rows=1 width=120) - Output:["_col0","_col1"],aggregations:["sum(_col0)","count(_col0)"] - Select Operator [SEL_1777] (rows=88516906238 width=113) - Output:["_col0"] - Map Join Operator [MAPJOIN_1776] (rows=13020465516 width=115) - Conds:SEL_1775._col2=RS_1726._col0(Inner),Output:["_col0","_col1"] - <-Map 33 [BROADCAST_EDGE] vectorized, llap - BROADCAST [RS_1726] - PartitionCols:_col0 - Please refer to the previous Select Operator [SEL_1723] - <-Select Operator [SEL_1775] (rows=21594638446 width=123) - Output:["_col0","_col1","_col2"] - TableScan [TS_1451] (rows=21594638446 width=123) - Output:["ws_quantity","ws_list_price"] - <-Filter Operator [FIL_1758] (rows=180081 width=131) - predicate:_col3 is not null - Group By Operator [GBY_1757] (rows=180081 width=131) - Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)"],keys:KEY._col0, KEY._col1, KEY._col2 - <-Map 34 [SIMPLE_EDGE] vectorized, llap - SHUFFLE [RS_1752] - PartitionCols:_col0, _col1, _col2 - Group By Operator [GBY_1751] (rows=180081 width=131) - Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["sum(_col3)","count()"],keys:_col0, _col1, _col2 - Select Operator [SEL_1750] (rows=748035 width=11) - Output:["_col0","_col1","_col2","_col3"] - Map Join Operator [MAPJOIN_1749] (rows=748035 width=11) - Conds:MAPJOIN_1748._col0=RS_1622._col0(Inner),Output:["_col1","_col2","_col13","_col14","_col15"] - <-Map 26 [BROADCAST_EDGE] vectorized, llap - BROADCAST [RS_1622] - PartitionCols:_col0 - Please refer to the previous Select Operator [SEL_1620] - <-Map Join Operator [MAPJOIN_1748] (rows=748035 width=8) - Conds:MAPJOIN_1747._col6, _col7, _col8=RS_1744._col0, _col1, _col2(Inner),Output:["_col0","_col1","_col2"] - <-Reducer 14 [BROADCAST_EDGE] vectorized, llap - BROADCAST [RS_1744] - PartitionCols:_col0, _col1, _col2 - Select Operator [SEL_1743] (rows=1 width=12) - Output:["_col0","_col1","_col2"] - Filter Operator [FIL_1742] (rows=1 width=20) - predicate:(_col3 = 3L) - Group By Operator [GBY_1741] (rows=153920 width=20) - Output:["_col0","_col1","_col2","_col3"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2 - <-Union 13 [SIMPLE_EDGE] - <-Reducer 12 [CONTAINS] vectorized, llap - Reduce Output Operator [RS_1662] - PartitionCols:_col0, _col1, _col2 - Group By Operator [GBY_1661] (rows=153920 width=20) - Output:["_col0","_col1","_col2","_col3"],aggregations:["count(_col3)"],keys:_col0, _col1, _col2 - Group By Operator [GBY_1660] (rows=180081 width=20) - Output:["_col0","_col1","_col2","_col3"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2 - <-Map 8 [SIMPLE_EDGE] vectorized, llap - SHUFFLE [RS_1655] - PartitionCols:_col0, _col1, _col2 - Please refer to the previous Group By Operator [GBY_1653] - <-Reducer 20 [CONTAINS] vectorized, llap - Reduce Output Operator [RS_1678] - PartitionCols:_col0, _col1, _col2 - Group By Operator [GBY_1677] (rows=153920 width=20) - Output:["_col0","_col1","_col2","_col3"],aggregations:["count(_col3)"],keys:_col0, _col1, _col2 - Group By Operator [GBY_1676] (rows=180081 width=20) - Output:["_col0","_col1","_col2","_col3"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2 - <-Map 18 [SIMPLE_EDGE] vectorized, llap - SHUFFLE [RS_1671] - PartitionCols:_col0, _col1, _col2 - Please refer to the previous Group By Operator [GBY_1669] - <-Reducer 24 [CONTAINS] vectorized, llap - Reduce Output Operator [RS_1694] - PartitionCols:_col0, _col1, _col2 - Group By Operator [GBY_1693] (rows=153920 width=20) - Output:["_col0","_col1","_col2","_col3"],aggregations:["count(_col3)"],keys:_col0, _col1, _col2 - Group By Operator [GBY_1692] (rows=180081 width=20) - Output:["_col0","_col1","_col2","_col3"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2 - <-Map 22 [SIMPLE_EDGE] vectorized, llap - SHUFFLE [RS_1687] - PartitionCols:_col0, _col1, _col2 - Please refer to the previous Group By Operator [GBY_1685] - <-Map Join Operator [MAPJOIN_1747] (rows=719610520 width=118) - Conds:MAPJOIN_1746._col0=RS_1614._col0(Inner),Output:["_col0","_col1","_col2","_col6","_col7","_col8"] - <-Map 7 [BROADCAST_EDGE] vectorized, llap - BROADCAST [RS_1614] - PartitionCols:_col0 - Please refer to the previous Select Operator [SEL_1609] - <-Map Join Operator [MAPJOIN_1746] (rows=724926652 width=106) - Conds:SEL_1745._col3=RS_1574._col0(Inner),Output:["_col0","_col1","_col2"] - <-Map 6 [BROADCAST_EDGE] vectorized, llap - BROADCAST [RS_1574] - PartitionCols:_col0 - Please refer to the previous Select Operator [SEL_1570] - <-Select Operator [SEL_1745] (rows=43005109025 width=131) - Output:["_col0","_col1","_col2","_col3"] - TableScan [TS_142] (rows=43005109025 width=131) - tpcds_bin_partitioned_orc_30000@catalog_sales,catalog_sales, ACID table,Tbl:COMPLETE,Col:COMPLETE,Output:["cs_item_sk","cs_quantity","cs_list_price"] - <-Reducer 42 [CONTAINS] vectorized, llap - Reduce Output Operator [RS_1803] - PartitionCols:_col0, _col1, _col2, _col3, _col4 - Group By Operator [GBY_1802] (rows=398760 width=230) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"],aggregations:["sum(_col4)","sum(_col5)"],keys:_col0, _col1, _col2, _col3, 0L - Top N Key Operator [TNK_1801] (rows=159504 width=221) - keys:_col0, _col1, _col2, _col3,top n:100 - Select Operator [SEL_1800] (rows=39450 width=218) - Output:["_col0","_col1","_col2","_col3","_col4","_col5"] - Filter Operator [FIL_1799] (rows=39450 width=243) - predicate:(_col3 > _col5) - Map Join Operator [MAPJOIN_1798] (rows=118350 width=243) - Conds:(Inner),Output:["_col0","_col1","_col2","_col3","_col4","_col5"] - <-Reducer 45 [BROADCAST_EDGE] vectorized, llap - BROADCAST [RS_1795] - Select Operator [SEL_1794] (rows=1 width=112) - Output:["_col0"] - Filter Operator [FIL_1793] (rows=1 width=120) - predicate:CAST( (_col0 / _col1) AS decimal(22,6)) is not null - Group By Operator [GBY_1792] (rows=1 width=120) - Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)"] - <-Union 44 [CUSTOM_SIMPLE_EDGE] - <-Map 43 [CONTAINS] vectorized, llap - Reduce Output Operator [RS_1808] - Group By Operator [GBY_1807] (rows=1 width=120) - Output:["_col0","_col1"],aggregations:["sum(_col0)","count(_col0)"] - Select Operator [SEL_1806] (rows=88516906238 width=113) - Output:["_col0"] - Map Join Operator [MAPJOIN_1805] (rows=49749852010 width=111) - Conds:SEL_1804._col2=RS_1588._col0(Inner),Output:["_col0","_col1"] - <-Map 6 [BROADCAST_EDGE] vectorized, llap - BROADCAST [RS_1588] - PartitionCols:_col0 - Please refer to the previous Select Operator [SEL_1571] - <-Select Operator [SEL_1804] (rows=82510879939 width=121) - Output:["_col0","_col1","_col2"] - TableScan [TS_1499] (rows=82510879939 width=121) - Output:["ss_quantity","ss_list_price"] - <-Map 46 [CONTAINS] vectorized, llap - Reduce Output Operator [RS_1813] - Group By Operator [GBY_1812] (rows=1 width=120) - Output:["_col0","_col1"],aggregations:["sum(_col0)","count(_col0)"] - Select Operator [SEL_1811] (rows=88516906238 width=113) - Output:["_col0"] - Map Join Operator [MAPJOIN_1810] (rows=25746588712 width=115) - Conds:SEL_1809._col2=RS_1709._col0(Inner),Output:["_col0","_col1"] - <-Map 31 [BROADCAST_EDGE] vectorized, llap - BROADCAST [RS_1709] - PartitionCols:_col0 - Please refer to the previous Select Operator [SEL_1704] - <-Select Operator [SEL_1809] (rows=43005109025 width=123) - Output:["_col0","_col1","_col2"] - TableScan [TS_1522] (rows=43005109025 width=123) - Output:["cs_quantity","cs_list_price"] - <-Map 47 [CONTAINS] vectorized, llap - Reduce Output Operator [RS_1818] - Group By Operator [GBY_1817] (rows=1 width=120) - Output:["_col0","_col1"],aggregations:["sum(_col0)","count(_col0)"] - Select Operator [SEL_1816] (rows=88516906238 width=113) - Output:["_col0"] - Map Join Operator [MAPJOIN_1815] (rows=13020465516 width=115) - Conds:SEL_1814._col2=RS_1728._col0(Inner),Output:["_col0","_col1"] - <-Map 33 [BROADCAST_EDGE] vectorized, llap - BROADCAST [RS_1728] - PartitionCols:_col0 - Please refer to the previous Select Operator [SEL_1723] - <-Select Operator [SEL_1814] (rows=21594638446 width=123) - Output:["_col0","_col1","_col2"] - TableScan [TS_1545] (rows=21594638446 width=123) - Output:["ws_quantity","ws_list_price"] - <-Filter Operator [FIL_1797] (rows=118350 width=131) - predicate:_col3 is not null - Group By Operator [GBY_1796] (rows=118350 width=131) - Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["sum(VALUE._col0)","count(VALUE._col1)"],keys:KEY._col0, KEY._col1, KEY._col2 - <-Map 41 [SIMPLE_EDGE] vectorized, llap - SHUFFLE [RS_1791] - PartitionCols:_col0, _col1, _col2 - Group By Operator [GBY_1790] (rows=118350 width=131) - Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["sum(_col3)","count()"],keys:_col0, _col1, _col2 - Select Operator [SEL_1789] (rows=378293 width=11) - Output:["_col0","_col1","_col2","_col3"] - Map Join Operator [MAPJOIN_1788] (rows=378293 width=11) - Conds:MAPJOIN_1787._col0=RS_1623._col0(Inner),Output:["_col1","_col2","_col13","_col14","_col15"] - <-Map 26 [BROADCAST_EDGE] vectorized, llap - BROADCAST [RS_1623] - PartitionCols:_col0 - Please refer to the previous Select Operator [SEL_1620] - <-Map Join Operator [MAPJOIN_1787] (rows=378293 width=8) - Conds:MAPJOIN_1786._col6, _col7, _col8=RS_1783._col0, _col1, _col2(Inner),Output:["_col0","_col1","_col2"] - <-Reducer 17 [BROADCAST_EDGE] vectorized, llap - BROADCAST [RS_1783] - PartitionCols:_col0, _col1, _col2 - Select Operator [SEL_1782] (rows=1 width=12) - Output:["_col0","_col1","_col2"] - Filter Operator [FIL_1781] (rows=1 width=20) - predicate:(_col3 = 3L) - Group By Operator [GBY_1780] (rows=153920 width=20) - Output:["_col0","_col1","_col2","_col3"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2 - <-Union 16 [SIMPLE_EDGE] - <-Reducer 15 [CONTAINS] vectorized, llap - Reduce Output Operator [RS_1665] - PartitionCols:_col0, _col1, _col2 - Group By Operator [GBY_1664] (rows=153920 width=20) - Output:["_col0","_col1","_col2","_col3"],aggregations:["count(_col3)"],keys:_col0, _col1, _col2 - Group By Operator [GBY_1663] (rows=180081 width=20) - Output:["_col0","_col1","_col2","_col3"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2 - <-Map 8 [SIMPLE_EDGE] vectorized, llap - SHUFFLE [RS_1656] - PartitionCols:_col0, _col1, _col2 - Please refer to the previous Group By Operator [GBY_1653] - <-Reducer 21 [CONTAINS] vectorized, llap - Reduce Output Operator [RS_1681] - PartitionCols:_col0, _col1, _col2 - Group By Operator [GBY_1680] (rows=153920 width=20) - Output:["_col0","_col1","_col2","_col3"],aggregations:["count(_col3)"],keys:_col0, _col1, _col2 - Group By Operator [GBY_1679] (rows=180081 width=20) - Output:["_col0","_col1","_col2","_col3"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2 - <-Map 18 [SIMPLE_EDGE] vectorized, llap - SHUFFLE [RS_1672] - PartitionCols:_col0, _col1, _col2 - Please refer to the previous Group By Operator [GBY_1669] - <-Reducer 25 [CONTAINS] vectorized, llap - Reduce Output Operator [RS_1697] - PartitionCols:_col0, _col1, _col2 - Group By Operator [GBY_1696] (rows=153920 width=20) - Output:["_col0","_col1","_col2","_col3"],aggregations:["count(_col3)"],keys:_col0, _col1, _col2 - Group By Operator [GBY_1695] (rows=180081 width=20) - Output:["_col0","_col1","_col2","_col3"],aggregations:["count(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2 - <-Map 22 [SIMPLE_EDGE] vectorized, llap - SHUFFLE [RS_1688] - PartitionCols:_col0, _col1, _col2 - Please refer to the previous Group By Operator [GBY_1685] - <-Map Join Operator [MAPJOIN_1786] (rows=363918657 width=135) - Conds:MAPJOIN_1785._col0=RS_1615._col0(Inner),Output:["_col0","_col1","_col2","_col6","_col7","_col8"] - <-Map 7 [BROADCAST_EDGE] vectorized, llap - BROADCAST [RS_1615] - PartitionCols:_col0 - Please refer to the previous Select Operator [SEL_1609] - <-Map Join Operator [MAPJOIN_1785] (rows=366607110 width=123) - Conds:SEL_1784._col3=RS_1576._col0(Inner),Output:["_col0","_col1","_col2"] - <-Map 6 [BROADCAST_EDGE] vectorized, llap - BROADCAST [RS_1576] - PartitionCols:_col0 - Please refer to the previous Select Operator [SEL_1570] - <-Select Operator [SEL_1784] (rows=21594638446 width=131) - Output:["_col0","_col1","_col2","_col3"] - TableScan [TS_285] (rows=21594638446 width=131) - tpcds_bin_partitioned_orc_30000@web_sales,web_sales, ACID table,Tbl:COMPLETE,Col:COMPLETE,Output:["ws_item_sk","ws_quantity","ws_list_price"] + Stage: Stage-2 + Dependency Collection + + Stage: Stage-4 + Tez +#### A masked pattern was here #### + Edges: + Map 14 <- Map 19 (BROADCAST_EDGE), Map 26 (BROADCAST_EDGE), Reducer 25 (BROADCAST_EDGE) + Map 20 <- Map 19 (BROADCAST_EDGE), Map 26 (BROADCAST_EDGE), Reducer 25 (BROADCAST_EDGE) + Map 22 <- Map 19 (BROADCAST_EDGE), Map 26 (BROADCAST_EDGE) + Map 27 <- Map 19 (BROADCAST_EDGE), Map 26 (BROADCAST_EDGE) + Map 29 <- Map 19 (BROADCAST_EDGE), Map 26 (BROADCAST_EDGE), Reducer 25 (BROADCAST_EDGE) + Map 31 <- Map 19 (BROADCAST_EDGE), Map 26 (BROADCAST_EDGE) + Reducer 10 <- Map 9 (CUSTOM_SIMPLE_EDGE), Reducer 13 (BROADCAST_EDGE) + Reducer 11 <- Reducer 10 (CUSTOM_SIMPLE_EDGE) + Reducer 12 <- Reducer 10 (CUSTOM_SIMPLE_EDGE) + Reducer 13 <- Map 9 (CUSTOM_SIMPLE_EDGE) + Reducer 15 <- Map 14 (SIMPLE_EDGE), Reducer 11 (BROADCAST_EDGE), Union 16 (CONTAINS) + Reducer 17 <- Union 16 (SIMPLE_EDGE) + Reducer 18 <- Reducer 17 (SIMPLE_EDGE) + Reducer 21 <- Map 20 (SIMPLE_EDGE), Reducer 12 (BROADCAST_EDGE), Union 16 (CONTAINS) + Reducer 23 <- Map 22 (SIMPLE_EDGE), Union 24 (CONTAINS) + Reducer 25 <- Map 26 (BROADCAST_EDGE), Union 24 (SIMPLE_EDGE) + Reducer 28 <- Map 27 (SIMPLE_EDGE), Union 24 (CONTAINS) + Reducer 30 <- Map 29 (SIMPLE_EDGE), Reducer 10 (BROADCAST_EDGE), Union 16 (CONTAINS) + Reducer 32 <- Map 31 (SIMPLE_EDGE), Union 24 (CONTAINS) +#### A masked pattern was here #### + Vertices: + Map 14 + Map Operator Tree: + TableScan + alias: store_sales + Statistics: Num rows: 82510879939 Data size: 10665797015532 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: ss_item_sk (type: bigint), ss_quantity (type: int), ss_list_price (type: decimal(7,2)), ss_sold_date_sk (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 82510879939 Data size: 10665797015532 Basic stats: COMPLETE Column stats: COMPLETE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col3 (type: bigint) + 1 _col0 (type: bigint) + outputColumnNames: _col0, _col1, _col2 + input vertices: + 1 Map 19 + Statistics: Num rows: 1400767848 Data size: 11206142900 Basic stats: COMPLETE Column stats: COMPLETE + Map Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col0 (type: bigint) + 1 _col0 (type: bigint) + outputColumnNames: _col0, _col1, _col2 + input vertices: + 1 Reducer 25 + Statistics: Num rows: 1400767848 Data size: 11206142900 Basic stats: COMPLETE Column stats: COMPLETE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: bigint) + 1 _col0 (type: bigint) + outputColumnNames: _col1, _col2, _col8, _col9, _col10 + input vertices: + 1 Map 26 + Statistics: Num rows: 1400767848 Data size: 16809200716 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col8 (type: int), _col9 (type: int), _col10 (type: int), (CAST( _col1 AS decimal(10,0)) * _col2) (type: decimal(18,2)) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 1400767848 Data size: 16809200716 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: sum(_col3), count() + keys: _col0 (type: int), _col1 (type: int), _col2 (type: int) + minReductionHashAggr: 0.99 + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + Statistics: Num rows: 11885346 Data size: 1568865568 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int), _col2 (type: int) + null sort order: zzz + sort order: +++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int), _col2 (type: int) + Statistics: Num rows: 11885346 Data size: 1568865568 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col3 (type: decimal(28,2)), _col4 (type: bigint) + Execution mode: vectorized, llap + LLAP IO: may be used (ACID table) + Map 19 + Map Operator Tree: + TableScan + alias: date_dim + filterExpr: (((d_year = 2000) and (d_moy = 11)) or d_year BETWEEN 1999 AND 2001) (type: boolean) + Statistics: Num rows: 73049 Data size: 1168784 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: ((d_year = 2000) and (d_moy = 11)) (type: boolean) + Statistics: Num rows: 31 Data size: 496 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: d_date_sk (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 31 Data size: 248 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: bigint) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: bigint) + Statistics: Num rows: 31 Data size: 248 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 31 Data size: 248 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: bigint) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 31 Data size: 248 Basic stats: COMPLETE Column stats: COMPLETE + Dynamic Partitioning Event Operator + Target column: ss_sold_date_sk (bigint) + Target Input: store_sales + Partition key expr: ss_sold_date_sk + Statistics: Num rows: 31 Data size: 248 Basic stats: COMPLETE Column stats: COMPLETE + Target Vertex: Map 14 + Reduce Output Operator + key expressions: _col0 (type: bigint) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: bigint) + Statistics: Num rows: 31 Data size: 248 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 31 Data size: 248 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: bigint) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 31 Data size: 248 Basic stats: COMPLETE Column stats: COMPLETE + Dynamic Partitioning Event Operator + Target column: ws_sold_date_sk (bigint) + Target Input: web_sales + Partition key expr: ws_sold_date_sk + Statistics: Num rows: 31 Data size: 248 Basic stats: COMPLETE Column stats: COMPLETE + Target Vertex: Map 29 + Reduce Output Operator + key expressions: _col0 (type: bigint) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: bigint) + Statistics: Num rows: 31 Data size: 248 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 31 Data size: 248 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: bigint) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 31 Data size: 248 Basic stats: COMPLETE Column stats: COMPLETE + Dynamic Partitioning Event Operator + Target column: cs_sold_date_sk (bigint) + Target Input: catalog_sales + Partition key expr: cs_sold_date_sk + Statistics: Num rows: 31 Data size: 248 Basic stats: COMPLETE Column stats: COMPLETE + Target Vertex: Map 20 + Filter Operator + predicate: d_year BETWEEN 1999 AND 2001 (type: boolean) + Statistics: Num rows: 1101 Data size: 13212 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: d_date_sk (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1101 Data size: 8808 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: bigint) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: bigint) + Statistics: Num rows: 1101 Data size: 8808 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1101 Data size: 8808 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: bigint) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1101 Data size: 8808 Basic stats: COMPLETE Column stats: COMPLETE + Dynamic Partitioning Event Operator + Target column: cs_sold_date_sk (bigint) + Target Input: catalog_sales + Partition key expr: cs_sold_date_sk + Statistics: Num rows: 1101 Data size: 8808 Basic stats: COMPLETE Column stats: COMPLETE + Target Vertex: Map 27 + Reduce Output Operator + key expressions: _col0 (type: bigint) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: bigint) + Statistics: Num rows: 1101 Data size: 8808 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1101 Data size: 8808 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: bigint) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1101 Data size: 8808 Basic stats: COMPLETE Column stats: COMPLETE + Dynamic Partitioning Event Operator + Target column: ws_sold_date_sk (bigint) + Target Input: web_sales + Partition key expr: ws_sold_date_sk + Statistics: Num rows: 1101 Data size: 8808 Basic stats: COMPLETE Column stats: COMPLETE + Target Vertex: Map 31 + Reduce Output Operator + key expressions: _col0 (type: bigint) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: bigint) + Statistics: Num rows: 1101 Data size: 8808 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: bigint) + outputColumnNames: _col0 + Statistics: Num rows: 1101 Data size: 8808 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: bigint) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1101 Data size: 8808 Basic stats: COMPLETE Column stats: COMPLETE + Dynamic Partitioning Event Operator + Target column: ss_sold_date_sk (bigint) + Target Input: store_sales + Partition key expr: ss_sold_date_sk + Statistics: Num rows: 1101 Data size: 8808 Basic stats: COMPLETE Column stats: COMPLETE + Target Vertex: Map 22 + Execution mode: vectorized, llap + LLAP IO: may be used (ACID table) + Map 20 + Map Operator Tree: + TableScan + alias: catalog_sales + Statistics: Num rows: 43005109025 Data size: 5664232305236 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: cs_item_sk (type: bigint), cs_quantity (type: int), cs_list_price (type: decimal(7,2)), cs_sold_date_sk (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 43005109025 Data size: 5664232305236 Basic stats: COMPLETE Column stats: COMPLETE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col3 (type: bigint) + 1 _col0 (type: bigint) + outputColumnNames: _col0, _col1, _col2 + input vertices: + 1 Map 19 + Statistics: Num rows: 724926652 Data size: 77448818784 Basic stats: COMPLETE Column stats: COMPLETE + Map Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col0 (type: bigint) + 1 _col0 (type: bigint) + outputColumnNames: _col0, _col1, _col2 + input vertices: + 1 Reducer 25 + Statistics: Num rows: 724926652 Data size: 77448818784 Basic stats: COMPLETE Column stats: COMPLETE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: bigint) + 1 _col0 (type: bigint) + outputColumnNames: _col1, _col2, _col8, _col9, _col10 + input vertices: + 1 Map 26 + Statistics: Num rows: 724926652 Data size: 80348511816 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col8 (type: int), _col9 (type: int), _col10 (type: int), (CAST( _col1 AS decimal(10,0)) * _col2) (type: decimal(18,2)) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 724926652 Data size: 80348511816 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: sum(_col3), count() + keys: _col0 (type: int), _col1 (type: int), _col2 (type: int) + minReductionHashAggr: 0.99 + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + Statistics: Num rows: 56545434 Data size: 7463996240 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int), _col2 (type: int) + null sort order: zzz + sort order: +++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int), _col2 (type: int) + Statistics: Num rows: 56545434 Data size: 7463996240 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col3 (type: decimal(28,2)), _col4 (type: bigint) + Execution mode: vectorized, llap + LLAP IO: may be used (ACID table) + Map 22 + Map Operator Tree: + TableScan + alias: store_sales + Statistics: Num rows: 82510879939 Data size: 1320174079024 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: ss_item_sk (type: bigint), ss_sold_date_sk (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 82510879939 Data size: 1320174079024 Basic stats: COMPLETE Column stats: COMPLETE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col1 (type: bigint) + 1 _col0 (type: bigint) + outputColumnNames: _col0 + input vertices: + 1 Map 19 + Statistics: Num rows: 49749852010 Data size: 397998816080 Basic stats: COMPLETE Column stats: COMPLETE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: bigint) + 1 _col0 (type: bigint) + outputColumnNames: _col4, _col5, _col6 + input vertices: + 1 Map 26 + Statistics: Num rows: 49385019517 Data size: 592620220724 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count() + keys: _col4 (type: int), _col5 (type: int), _col6 (type: int) + minReductionHashAggr: 0.99 + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 416887515 Data size: 8337750196 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int), _col2 (type: int) + null sort order: zzz + sort order: +++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int), _col2 (type: int) + Statistics: Num rows: 416887515 Data size: 8337750196 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col3 (type: bigint) + Execution mode: vectorized, llap + LLAP IO: may be used (ACID table) + Map 26 + Map Operator Tree: + TableScan + alias: iss + Statistics: Num rows: 462000 Data size: 9226424 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (i_category_id is not null and i_brand_id is not null and i_class_id is not null) (type: boolean) + Statistics: Num rows: 458612 Data size: 9158760 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: i_item_sk (type: bigint), i_brand_id (type: int), i_class_id (type: int), i_category_id (type: int) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 458612 Data size: 9158760 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: bigint) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: bigint) + Statistics: Num rows: 458612 Data size: 9158760 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: int) + Reduce Output Operator + key expressions: _col1 (type: int), _col2 (type: int), _col3 (type: int) + null sort order: zzz + sort order: +++ + Map-reduce partition columns: _col1 (type: int), _col2 (type: int), _col3 (type: int) + Statistics: Num rows: 458612 Data size: 9158760 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Reduce Output Operator + key expressions: _col0 (type: bigint) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: bigint) + Statistics: Num rows: 458612 Data size: 9158760 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: int) + Reduce Output Operator + key expressions: _col0 (type: bigint) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: bigint) + Statistics: Num rows: 458612 Data size: 9158760 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: int) + Select Operator + expressions: i_item_sk (type: bigint), i_brand_id (type: int), i_class_id (type: int), i_category_id (type: int) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 462000 Data size: 9226424 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: bigint) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: bigint) + Statistics: Num rows: 462000 Data size: 9226424 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: int) + Reduce Output Operator + key expressions: _col0 (type: bigint) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: bigint) + Statistics: Num rows: 462000 Data size: 9226424 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: int) + Reduce Output Operator + key expressions: _col0 (type: bigint) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: bigint) + Statistics: Num rows: 462000 Data size: 9226424 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: int) + Execution mode: vectorized, llap + LLAP IO: may be used (ACID table) + Map 27 + Map Operator Tree: + TableScan + alias: catalog_sales + Statistics: Num rows: 43005109025 Data size: 688081744400 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: cs_item_sk (type: bigint), cs_sold_date_sk (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 43005109025 Data size: 688081744400 Basic stats: COMPLETE Column stats: COMPLETE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col1 (type: bigint) + 1 _col0 (type: bigint) + outputColumnNames: _col0 + input vertices: + 1 Map 19 + Statistics: Num rows: 25746588712 Data size: 205972709696 Basic stats: COMPLETE Column stats: COMPLETE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: bigint) + 1 _col0 (type: bigint) + outputColumnNames: _col4, _col5, _col6 + input vertices: + 1 Map 26 + Statistics: Num rows: 25557780268 Data size: 306693349736 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count() + keys: _col4 (type: int), _col5 (type: int), _col6 (type: int) + minReductionHashAggr: 0.99 + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 215917119 Data size: 4318342276 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int), _col2 (type: int) + null sort order: zzz + sort order: +++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int), _col2 (type: int) + Statistics: Num rows: 215917119 Data size: 4318342276 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col3 (type: bigint) + Execution mode: vectorized, llap + LLAP IO: may be used (ACID table) + Map 29 + Map Operator Tree: + TableScan + alias: web_sales + Statistics: Num rows: 21594638446 Data size: 2850178636132 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: ws_item_sk (type: bigint), ws_quantity (type: int), ws_list_price (type: decimal(7,2)), ws_sold_date_sk (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 21594638446 Data size: 2850178636132 Basic stats: COMPLETE Column stats: COMPLETE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col3 (type: bigint) + 1 _col0 (type: bigint) + outputColumnNames: _col0, _col1, _col2 + input vertices: + 1 Map 19 + Statistics: Num rows: 366607110 Data size: 45145642900 Basic stats: COMPLETE Column stats: COMPLETE + Map Join Operator + condition map: + Left Semi Join 0 to 1 + keys: + 0 _col0 (type: bigint) + 1 _col0 (type: bigint) + outputColumnNames: _col0, _col1, _col2 + input vertices: + 1 Reducer 25 + Statistics: Num rows: 366607110 Data size: 45145642900 Basic stats: COMPLETE Column stats: COMPLETE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: bigint) + 1 _col0 (type: bigint) + outputColumnNames: _col1, _col2, _col8, _col9, _col10 + input vertices: + 1 Map 26 + Statistics: Num rows: 366607110 Data size: 46612057764 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col8 (type: int), _col9 (type: int), _col10 (type: int), (CAST( _col1 AS decimal(10,0)) * _col2) (type: decimal(18,2)) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 366607110 Data size: 46612057764 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: sum(_col3), count() + keys: _col0 (type: int), _col1 (type: int), _col2 (type: int) + minReductionHashAggr: 0.99 + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + Statistics: Num rows: 32954823 Data size: 4350035428 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int), _col2 (type: int) + null sort order: zzz + sort order: +++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int), _col2 (type: int) + Statistics: Num rows: 32954823 Data size: 4350035428 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col3 (type: decimal(28,2)), _col4 (type: bigint) + Execution mode: vectorized, llap + LLAP IO: may be used (ACID table) + Map 31 + Map Operator Tree: + TableScan + alias: web_sales + Statistics: Num rows: 21594638446 Data size: 345514215136 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: ws_item_sk (type: bigint), ws_sold_date_sk (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 21594638446 Data size: 345514215136 Basic stats: COMPLETE Column stats: COMPLETE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col1 (type: bigint) + 1 _col0 (type: bigint) + outputColumnNames: _col0 + input vertices: + 1 Map 19 + Statistics: Num rows: 13020465516 Data size: 104163724128 Basic stats: COMPLETE Column stats: COMPLETE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: bigint) + 1 _col0 (type: bigint) + outputColumnNames: _col4, _col5, _col6 + input vertices: + 1 Map 26 + Statistics: Num rows: 12924982039 Data size: 155099770988 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count() + keys: _col4 (type: int), _col5 (type: int), _col6 (type: int) + minReductionHashAggr: 0.99 + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 109129086 Data size: 2182581616 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int), _col2 (type: int) + null sort order: zzz + sort order: +++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int), _col2 (type: int) + Statistics: Num rows: 109129086 Data size: 2182581616 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col3 (type: bigint) + Execution mode: vectorized, llap + LLAP IO: may be used (ACID table) + Map 9 + Map Operator Tree: + TableScan + alias: avg_sales + Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count() + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) + Filter Operator + predicate: average_sales is not null (type: boolean) + Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: average_sales (type: decimal(22,6)) + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: decimal(22,6)) + Execution mode: vectorized, llap + LLAP IO: all inputs + Reducer 10 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + mode: mergepartial + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: sq_count_check(_col0) (type: boolean) + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col1 + input vertices: + 1 Reducer 13 + Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: decimal(22,6)) + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: decimal(22,6)) + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: decimal(22,6)) + Reducer 11 + Execution mode: vectorized, llap + Reduce Operator Tree: + Select Operator + expressions: VALUE._col1 (type: decimal(22,6)) + outputColumnNames: _col1 + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: decimal(22,6)) + Reducer 12 + Execution mode: vectorized, llap + Reduce Operator Tree: + Select Operator + expressions: VALUE._col1 (type: decimal(22,6)) + outputColumnNames: _col1 + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col1 (type: decimal(22,6)) + Reducer 13 + Execution mode: vectorized, llap + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: decimal(22,6)) + outputColumnNames: _col0 + Reduce Output Operator + null sort order: + sort order: + Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: decimal(22,6)) + Reducer 15 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0), count(VALUE._col1) + keys: KEY._col0 (type: int), KEY._col1 (type: int), KEY._col2 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + Statistics: Num rows: 180081 Data size: 23770692 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: _col3 is not null (type: boolean) + Statistics: Num rows: 180081 Data size: 23770692 Basic stats: COMPLETE Column stats: COMPLETE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col1, _col2, _col3, _col4, _col5, _col6 + input vertices: + 0 Reducer 11 + Statistics: Num rows: 180081 Data size: 43939764 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (_col5 > _col1) (type: boolean) + Statistics: Num rows: 60027 Data size: 14646588 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: 'store' (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: int), _col5 (type: decimal(28,2)), _col6 (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 60027 Data size: 13265967 Basic stats: COMPLETE Column stats: COMPLETE + Top N Key Operator + sort order: ++++ + keys: _col0 (type: string), _col1 (type: int), _col2 (type: int), _col3 (type: int) + null sort order: zzzz + Statistics: Num rows: 180081 Data size: 39797901 Basic stats: COMPLETE Column stats: COMPLETE + top n: 100 + Group By Operator + aggregations: sum(_col4), sum(_col5) + keys: _col0 (type: string), _col1 (type: int), _col2 (type: int), _col3 (type: int), 0L (type: bigint) + grouping sets: 0, 1, 3, 7, 15 + minReductionHashAggr: 0.962006 + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 450202 Data size: 103996662 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: bigint) + null sort order: zzzzz + sort order: +++++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: bigint) + Statistics: Num rows: 450202 Data size: 103996662 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col5 (type: decimal(38,2)), _col6 (type: bigint) + Reducer 17 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0), sum(VALUE._col1) + keys: KEY._col0 (type: string), KEY._col1 (type: int), KEY._col2 (type: int), KEY._col3 (type: int), KEY._col4 (type: bigint) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2, _col3, _col5, _col6 + Statistics: Num rows: 450202 Data size: 103996662 Basic stats: COMPLETE Column stats: COMPLETE + pruneGroupingSetId: true + Select Operator + expressions: _col0 (type: string), _col1 (type: int), _col2 (type: int), _col3 (type: int), _col5 (type: decimal(38,2)), _col6 (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 450202 Data size: 100395046 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: int), _col2 (type: int), _col3 (type: int) + null sort order: zzzz + sort order: ++++ + Statistics: Num rows: 450202 Data size: 100395046 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col4 (type: decimal(38,2)), _col5 (type: bigint) + Reducer 18 + Execution mode: vectorized, llap + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: int), KEY.reducesinkkey2 (type: int), KEY.reducesinkkey3 (type: int), VALUE._col0 (type: decimal(38,2)), VALUE._col1 (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 450202 Data size: 100395046 Basic stats: COMPLETE Column stats: COMPLETE + Limit + Number of rows: 100 + Statistics: Num rows: 100 Data size: 22300 Basic stats: COMPLETE Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 100 Data size: 22300 Basic stats: COMPLETE Column stats: COMPLETE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + Reducer 21 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0), count(VALUE._col1) + keys: KEY._col0 (type: int), KEY._col1 (type: int), KEY._col2 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + Statistics: Num rows: 180081 Data size: 23770692 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: _col3 is not null (type: boolean) + Statistics: Num rows: 180081 Data size: 23770692 Basic stats: COMPLETE Column stats: COMPLETE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col1, _col2, _col3, _col4, _col5, _col6 + input vertices: + 0 Reducer 12 + Statistics: Num rows: 180081 Data size: 43939764 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (_col5 > _col1) (type: boolean) + Statistics: Num rows: 60027 Data size: 14646588 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: 'catalog' (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: int), _col5 (type: decimal(28,2)), _col6 (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 60027 Data size: 13386021 Basic stats: COMPLETE Column stats: COMPLETE + Top N Key Operator + sort order: ++++ + keys: _col0 (type: string), _col1 (type: int), _col2 (type: int), _col3 (type: int) + null sort order: zzzz + Statistics: Num rows: 180081 Data size: 39797901 Basic stats: COMPLETE Column stats: COMPLETE + top n: 100 + Group By Operator + aggregations: sum(_col4), sum(_col5) + keys: _col0 (type: string), _col1 (type: int), _col2 (type: int), _col3 (type: int), 0L (type: bigint) + grouping sets: 0, 1, 3, 7, 15 + minReductionHashAggr: 0.962006 + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 450202 Data size: 103996662 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: bigint) + null sort order: zzzzz + sort order: +++++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: bigint) + Statistics: Num rows: 450202 Data size: 103996662 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col5 (type: decimal(38,2)), _col6 (type: bigint) + Reducer 23 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: int), KEY._col1 (type: int), KEY._col2 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 180081 Data size: 3601620 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count(_col3) + keys: _col0 (type: int), _col1 (type: int), _col2 (type: int) + minReductionHashAggr: 0.9873353 + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 153920 Data size: 3078400 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int), _col2 (type: int) + null sort order: zzz + sort order: +++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int), _col2 (type: int) + Statistics: Num rows: 153920 Data size: 3078400 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col3 (type: bigint) + Reducer 25 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: int), KEY._col1 (type: int), KEY._col2 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 153920 Data size: 3078400 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (_col3 = 3L) (type: boolean) + Statistics: Num rows: 1 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: _col0 (type: int), _col1 (type: int), _col2 (type: int) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col1 (type: int), _col2 (type: int), _col3 (type: int) + 1 _col0 (type: int), _col1 (type: int), _col2 (type: int) + outputColumnNames: _col0 + input vertices: + 0 Map 26 + Statistics: Num rows: 476 Data size: 3808 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + keys: _col0 (type: bigint) + minReductionHashAggr: 0.4 + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 477 Data size: 3816 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: bigint) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: bigint) + Statistics: Num rows: 477 Data size: 3816 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: bigint) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: bigint) + Statistics: Num rows: 477 Data size: 3816 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: bigint) + null sort order: z + sort order: + + Map-reduce partition columns: _col0 (type: bigint) + Statistics: Num rows: 477 Data size: 3816 Basic stats: COMPLETE Column stats: COMPLETE + Reducer 28 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: int), KEY._col1 (type: int), KEY._col2 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 180081 Data size: 3601620 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count(_col3) + keys: _col0 (type: int), _col1 (type: int), _col2 (type: int) + minReductionHashAggr: 0.9873353 + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 153920 Data size: 3078400 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int), _col2 (type: int) + null sort order: zzz + sort order: +++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int), _col2 (type: int) + Statistics: Num rows: 153920 Data size: 3078400 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col3 (type: bigint) + Reducer 30 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: sum(VALUE._col0), count(VALUE._col1) + keys: KEY._col0 (type: int), KEY._col1 (type: int), KEY._col2 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + Statistics: Num rows: 180081 Data size: 23770692 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: _col3 is not null (type: boolean) + Statistics: Num rows: 180081 Data size: 23770692 Basic stats: COMPLETE Column stats: COMPLETE + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 + 1 + outputColumnNames: _col1, _col2, _col3, _col4, _col5, _col6 + input vertices: + 0 Reducer 10 + Statistics: Num rows: 180081 Data size: 43939764 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: (_col5 > _col1) (type: boolean) + Statistics: Num rows: 60027 Data size: 14646588 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + expressions: 'web' (type: string), _col2 (type: int), _col3 (type: int), _col4 (type: int), _col5 (type: decimal(28,2)), _col6 (type: bigint) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 60027 Data size: 13145913 Basic stats: COMPLETE Column stats: COMPLETE + Top N Key Operator + sort order: ++++ + keys: _col0 (type: string), _col1 (type: int), _col2 (type: int), _col3 (type: int) + null sort order: zzzz + Statistics: Num rows: 180081 Data size: 39797901 Basic stats: COMPLETE Column stats: COMPLETE + top n: 100 + Group By Operator + aggregations: sum(_col4), sum(_col5) + keys: _col0 (type: string), _col1 (type: int), _col2 (type: int), _col3 (type: int), 0L (type: bigint) + grouping sets: 0, 1, 3, 7, 15 + minReductionHashAggr: 0.962006 + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 + Statistics: Num rows: 450202 Data size: 103996662 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: bigint) + null sort order: zzzzz + sort order: +++++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: bigint) + Statistics: Num rows: 450202 Data size: 103996662 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col5 (type: decimal(38,2)), _col6 (type: bigint) + Reducer 32 + Execution mode: vectorized, llap + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: int), KEY._col1 (type: int), KEY._col2 (type: int) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 180081 Data size: 3601620 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count(_col3) + keys: _col0 (type: int), _col1 (type: int), _col2 (type: int) + minReductionHashAggr: 0.9873353 + mode: hash + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 153920 Data size: 3078400 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int), _col2 (type: int) + null sort order: zzz + sort order: +++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int), _col2 (type: int) + Statistics: Num rows: 153920 Data size: 3078400 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col3 (type: bigint) + Union 16 + Vertex: Union 16 + Union 24 + Vertex: Union 24 + + Stage: Stage-0 + Move Operator + files: + hdfs directory: true +#### A masked pattern was here #### + + Stage: Stage-3 + Fetch Operator + limit: 100 + Processor Tree: + ListSink From 3840b81da36f25f493d63beef54c9f3740a7f874 Mon Sep 17 00:00:00 2001 From: okumin Date: Sat, 19 Oct 2024 11:48:25 +0900 Subject: [PATCH 2/6] Remove hive.in.test.planmapper.strict.validation --- .../java/org/apache/hadoop/hive/conf/HiveConf.java | 4 ---- data/conf/hive-site.xml | 5 ----- data/conf/iceberg/llap/hive-site.xml | 5 ----- data/conf/iceberg/tez/hive-site.xml | 5 ----- data/conf/llap/hive-site.xml | 5 ----- data/conf/perf/tpcds30tb/tez/hive-site.xml | 5 ----- data/conf/rlist/hive-site.xml | 5 ----- data/conf/tez/hive-site.xml | 5 ----- ql/src/java/org/apache/hadoop/hive/ql/Context.java | 5 +---- .../apache/hadoop/hive/ql/plan/mapper/PlanMapper.java | 11 ----------- 10 files changed, 1 insertion(+), 54 deletions(-) diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java index 3e5036f3aa36..0d3f9e358ba3 100644 --- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -809,10 +809,6 @@ public static enum ConfVars { HIVE_IN_TEST_REPL("hive.in.repl.test", false, "internal usage only, true in replication test mode", true), HIVE_IN_TEST_IDE("hive.in.ide.test", false, "internal usage only, true if test running in ide", true), - HIVE_IN_TEST_PLANMAPPER_STRICT_VALIDATION("hive.in.test.planmapper.strict.validation", false, - "internal use only, whether to raise an error when unexpected links are found. We ignore equivalence mapping " - + "violation because it introduces only minor problems. But we want to strictly check it in qtest so that we " - + "can prevent further degradations"), HIVE_TESTING_SHORT_LOGS("hive.testing.short.logs", false, "internal usage only, used only in test mode. If set true, when requesting the " + "operation logs the short version (generated by LogDivertAppenderForTest) will be " + diff --git a/data/conf/hive-site.xml b/data/conf/hive-site.xml index 36819fcdcbb0..cd716824bb9f 100644 --- a/data/conf/hive-site.xml +++ b/data/conf/hive-site.xml @@ -31,11 +31,6 @@ Internal marker for test. Used for masking env-dependent values - - hive.in.test.planmapper.strict.validation - true - - mapreduce.jobtracker.staging.root.dir diff --git a/data/conf/iceberg/llap/hive-site.xml b/data/conf/iceberg/llap/hive-site.xml index 98606eb26dc5..e6dedf966e78 100644 --- a/data/conf/iceberg/llap/hive-site.xml +++ b/data/conf/iceberg/llap/hive-site.xml @@ -25,11 +25,6 @@ Internal marker for test. Used for masking env-dependent values - - hive.in.test.planmapper.strict.validation - true - - diff --git a/data/conf/iceberg/tez/hive-site.xml b/data/conf/iceberg/tez/hive-site.xml index 917881ccf861..2fda519cad96 100644 --- a/data/conf/iceberg/tez/hive-site.xml +++ b/data/conf/iceberg/tez/hive-site.xml @@ -25,11 +25,6 @@ Internal marker for test. Used for masking env-dependent values - - hive.in.test.planmapper.strict.validation - true - - diff --git a/data/conf/llap/hive-site.xml b/data/conf/llap/hive-site.xml index 291c3445ad2a..2cf2882b6efa 100644 --- a/data/conf/llap/hive-site.xml +++ b/data/conf/llap/hive-site.xml @@ -31,11 +31,6 @@ Internal marker for test. Used for masking env-dependent values - - hive.in.test.planmapper.strict.validation - true - - diff --git a/data/conf/perf/tpcds30tb/tez/hive-site.xml b/data/conf/perf/tpcds30tb/tez/hive-site.xml index ecd940572c1b..a8d81e33c38e 100644 --- a/data/conf/perf/tpcds30tb/tez/hive-site.xml +++ b/data/conf/perf/tpcds30tb/tez/hive-site.xml @@ -22,11 +22,6 @@ true Internal marker for test. Used for masking env-dependent values - - hive.in.test.planmapper.strict.validation - false - We can enable it once we resolve the problem of query14.q - hive.rpc.query.plan true diff --git a/data/conf/rlist/hive-site.xml b/data/conf/rlist/hive-site.xml index 492dd647fb64..9768b759fb31 100644 --- a/data/conf/rlist/hive-site.xml +++ b/data/conf/rlist/hive-site.xml @@ -30,11 +30,6 @@ Internal marker for test. Used for masking env-dependent values - - hive.in.test.planmapper.strict.validation - true - - diff --git a/data/conf/tez/hive-site.xml b/data/conf/tez/hive-site.xml index 34b6ae159750..7ee041667089 100644 --- a/data/conf/tez/hive-site.xml +++ b/data/conf/tez/hive-site.xml @@ -25,11 +25,6 @@ Internal marker for test. Used for masking env-dependent values - - hive.in.test.planmapper.strict.validation - true - - diff --git a/ql/src/java/org/apache/hadoop/hive/ql/Context.java b/ql/src/java/org/apache/hadoop/hive/ql/Context.java index 742de91b97e2..2e6df97c1521 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/Context.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/Context.java @@ -175,7 +175,7 @@ public class Context { private WmContext wmContext; private boolean isExplainPlan = false; - private PlanMapper planMapper; + private PlanMapper planMapper = new PlanMapper(); private StatsSource statsSource; private int executionIndex; @@ -423,7 +423,6 @@ private Context(Configuration conf, String executionId) { HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_MATERIALIZED_VIEW_ENABLE_AUTO_REWRITING_SQL) || HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_MATERIALIZED_VIEW_ENABLE_AUTO_REWRITING_SUBQUERY_SQL); scheduledQuery = false; - planMapper = new PlanMapper(conf); } protected Context(Context ctx) { @@ -471,8 +470,6 @@ protected Context(Context ctx) { this.opContext = new CompilationOpContext(); this.enableUnparse = ctx.enableUnparse; this.scheduledQuery = ctx.scheduledQuery; - // Don't inherit the original plan mapper - this.planMapper = new PlanMapper(ctx.conf); } public Map getFsScratchDirs() { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/PlanMapper.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/PlanMapper.java index f0d795df078e..ef232d2fcfac 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/PlanMapper.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/PlanMapper.java @@ -33,9 +33,6 @@ import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.conf.HiveConf.ConfVars; import org.apache.hadoop.hive.ql.exec.Operator; import org.apache.hadoop.hive.ql.optimizer.signature.OpTreeSignature; import org.apache.hadoop.hive.ql.optimizer.signature.OpTreeSignatureFactory; @@ -54,13 +51,8 @@ public class PlanMapper { private final Set groups = new HashSet<>(); private final Map objectMap = new CompositeMap<>(OpTreeSignature.class, AuxOpTreeSignature.class); - private final boolean failsWithIllegalLink; private final AtomicBoolean isBroken = new AtomicBoolean(false); - public PlanMapper(Configuration conf) { - failsWithIllegalLink = HiveConf.getBoolVar(conf, ConfVars.HIVE_IN_TEST_PLANMAPPER_STRICT_VALIDATION); - } - /** * Specialized class which can compare by identity or value; based on the key type. */ @@ -231,9 +223,6 @@ private void link(Object o1, Object o2, boolean mayMerge) { if (mGroups.size() > 1) { if (!mayMerge) { LOG.warn("Illegally linking {} and {}", o1, o2); - if (failsWithIllegalLink) { - throw new RuntimeException("equivalence mapping violation"); - } isBroken.set(true); } EquivGroup newGrp = new EquivGroup(); From 6fda20a1db1b140d786a4f5271d3f0299ed700be Mon Sep 17 00:00:00 2001 From: okumin Date: Sat, 19 Oct 2024 11:50:36 +0900 Subject: [PATCH 3/6] Use a primitive boolean for a flag --- .../org/apache/hadoop/hive/ql/plan/mapper/PlanMapper.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/PlanMapper.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/PlanMapper.java index ef232d2fcfac..141d8cd4c1d5 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/PlanMapper.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/PlanMapper.java @@ -32,7 +32,6 @@ import java.util.Objects; import java.util.Set; -import java.util.concurrent.atomic.AtomicBoolean; import org.apache.hadoop.hive.ql.exec.Operator; import org.apache.hadoop.hive.ql.optimizer.signature.OpTreeSignature; import org.apache.hadoop.hive.ql.optimizer.signature.OpTreeSignatureFactory; @@ -51,7 +50,7 @@ public class PlanMapper { private final Set groups = new HashSet<>(); private final Map objectMap = new CompositeMap<>(OpTreeSignature.class, AuxOpTreeSignature.class); - private final AtomicBoolean isBroken = new AtomicBoolean(false); + private boolean isBroken = false; /** * Specialized class which can compare by identity or value; based on the key type. @@ -223,7 +222,7 @@ private void link(Object o1, Object o2, boolean mayMerge) { if (mGroups.size() > 1) { if (!mayMerge) { LOG.warn("Illegally linking {} and {}", o1, o2); - isBroken.set(true); + isBroken = true; } EquivGroup newGrp = new EquivGroup(); newGrp.add(o1); @@ -255,7 +254,7 @@ private Object getKeyFor(Object o) { } public boolean isBroken() { - return isBroken.get(); + return isBroken; } public List getAll(Class clazz) { From c8651f849559e1a4b32cb1b5945400d1369dff12 Mon Sep 17 00:00:00 2001 From: okumin Date: Sat, 19 Oct 2024 12:14:30 +0900 Subject: [PATCH 4/6] Add a defensive last resort --- .../apache/hadoop/hive/ql/plan/mapper/PlanMapper.java | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/PlanMapper.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/PlanMapper.java index 141d8cd4c1d5..4fd55b480b64 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/PlanMapper.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/PlanMapper.java @@ -258,6 +258,10 @@ public boolean isBroken() { } public List getAll(Class clazz) { + if (isBroken) { + LOG.warn("PlanMapper#getAll is no longer valid. Please use PlanMapper#isBroken to handle the state correctly"); + return Collections.emptyList(); + } List ret = new ArrayList<>(); for (EquivGroup g : groups) { ret.addAll(g.getAll(clazz)); @@ -284,6 +288,11 @@ public T lookup(Class clazz, Object key) { } public Iterator iterateGroups() { + if (isBroken) { + LOG.warn("PlanMapper#iterateGroups is no longer valid. " + + "Please use PlanMapper#isBroken to handle the state correctly"); + return Collections.emptyIterator(); + } return groups.iterator(); } From fbd94d41b5d75aeb9f69640b9730722563fd7b18 Mon Sep 17 00:00:00 2001 From: okumin Date: Sat, 19 Oct 2024 13:14:17 +0900 Subject: [PATCH 5/6] Update cbo_query14.q.out --- .../perf/tpcds30tb/tez/cbo_query14.q.out | 66 +++++++++++++++++++ 1 file changed, 66 insertions(+) diff --git a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/cbo_query14.q.out b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/cbo_query14.q.out index b08d48b994da..7a5c961c5b60 100644 --- a/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/cbo_query14.q.out +++ b/ql/src/test/results/clientpositive/perf/tpcds30tb/tez/cbo_query14.q.out @@ -1,3 +1,69 @@ +CTE Suggestion: +HiveProject(d_date_sk=[$0]) + HiveFilter(condition=[BETWEEN(false, $6, 1998, 2000)]) + HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim]) + +CTE Suggestion: +HiveJoin(condition=[true], joinType=[inner], algorithm=[none], cost=[not available]) + HiveFilter(condition=[sq_count_check($0)]) + HiveAggregate(group=[{}], cnt=[COUNT()]) + HiveTableScan(table=[[default, avg_sales]], table:alias=[avg_sales]) + HiveProject(average_sales=[$0]) + HiveFilter(condition=[IS NOT NULL($0)]) + HiveTableScan(table=[[default, avg_sales]], table:alias=[avg_sales]) + +CTE Suggestion: +HiveProject(d_date_sk=[$0], d_year=[CAST(2000):INTEGER], d_moy=[CAST(11):INTEGER]) + HiveFilter(condition=[AND(=($6, 2000), =($8, 11))]) + HiveTableScan(table=[[default, date_dim]], table:alias=[date_dim]) + +CTE Suggestion: +HiveProject(ss_item_sk=[$0]) + HiveJoin(condition=[AND(=($1, $4), =($2, $5), =($3, $6))], joinType=[inner], algorithm=[none], cost=[not available]) + HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11]) + HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11))]) + HiveTableScan(table=[[default, item]], table:alias=[item]) + HiveProject($f0=[$0], $f1=[$1], $f2=[$2]) + HiveFilter(condition=[=($3, 3)]) + HiveAggregate(group=[{0, 1, 2}], agg#0=[count($3)]) + HiveUnion(all=[true]) + HiveAggregate(group=[{4, 5, 6}], agg#0=[count()]) + HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available]) + HiveJoin(condition=[=($1, $2)], joinType=[inner], algorithm=[none], cost=[not available]) + HiveProject(ss_item_sk=[$1], ss_sold_date_sk=[$22]) + HiveFilter(condition=[IS NOT NULL($22)]) + HiveTableScan(table=[[default, store_sales]], table:alias=[store_sales]) + HiveProject(d_date_sk=[$0]) + HiveFilter(condition=[BETWEEN(false, $6, 1999, 2001)]) + HiveTableScan(table=[[default, date_dim]], table:alias=[d1]) + HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11]) + HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11))]) + HiveTableScan(table=[[default, item]], table:alias=[iss]) + HiveAggregate(group=[{4, 5, 6}], agg#0=[count()]) + HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available]) + HiveJoin(condition=[=($1, $2)], joinType=[inner], algorithm=[none], cost=[not available]) + HiveProject(cs_item_sk=[$14], cs_sold_date_sk=[$33]) + HiveFilter(condition=[IS NOT NULL($33)]) + HiveTableScan(table=[[default, catalog_sales]], table:alias=[catalog_sales]) + HiveProject(d_date_sk=[$0]) + HiveFilter(condition=[BETWEEN(false, $6, 1999, 2001)]) + HiveTableScan(table=[[default, date_dim]], table:alias=[d2]) + HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11]) + HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11))]) + HiveTableScan(table=[[default, item]], table:alias=[ics]) + HiveAggregate(group=[{4, 5, 6}], agg#0=[count()]) + HiveJoin(condition=[=($0, $3)], joinType=[inner], algorithm=[none], cost=[not available]) + HiveJoin(condition=[=($1, $2)], joinType=[inner], algorithm=[none], cost=[not available]) + HiveProject(ws_item_sk=[$2], ws_sold_date_sk=[$33]) + HiveFilter(condition=[IS NOT NULL($33)]) + HiveTableScan(table=[[default, web_sales]], table:alias=[web_sales]) + HiveProject(d_date_sk=[$0]) + HiveFilter(condition=[BETWEEN(false, $6, 1999, 2001)]) + HiveTableScan(table=[[default, date_dim]], table:alias=[d3]) + HiveProject(i_item_sk=[$0], i_brand_id=[$7], i_class_id=[$9], i_category_id=[$11]) + HiveFilter(condition=[AND(IS NOT NULL($7), IS NOT NULL($9), IS NOT NULL($11))]) + HiveTableScan(table=[[default, item]], table:alias=[iws]) + Warning: Map Join MAPJOIN[1098][bigTable=?] in task 'Reducer 10' is a cross product Warning: Map Join MAPJOIN[1147][bigTable=?] in task 'Reducer 15' is a cross product Warning: Map Join MAPJOIN[1163][bigTable=?] in task 'Reducer 21' is a cross product From 2ca6f972cdf31e529411cea0615b15efae4e5176 Mon Sep 17 00:00:00 2001 From: okumin Date: Mon, 28 Oct 2024 20:01:08 +0900 Subject: [PATCH 6/6] Attach IncorrectRuntimeStatsMarker on finding an illegal link --- .../RuntimeStatsPersistenceCheckerHook.java | 4 ---- .../hive/ql/plan/mapper/PlanMapper.java | 22 +++++-------------- .../hive/ql/plan/mapper/StatsSources.java | 5 ----- .../hive/ql/reexec/ReOptimizePlugin.java | 8 ------- 4 files changed, 5 insertions(+), 34 deletions(-) diff --git a/ql/src/java/org/apache/hadoop/hive/ql/hooks/RuntimeStatsPersistenceCheckerHook.java b/ql/src/java/org/apache/hadoop/hive/ql/hooks/RuntimeStatsPersistenceCheckerHook.java index 28de59578454..cc754eebc4e2 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/hooks/RuntimeStatsPersistenceCheckerHook.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/hooks/RuntimeStatsPersistenceCheckerHook.java @@ -38,10 +38,6 @@ public class RuntimeStatsPersistenceCheckerHook implements ExecuteWithHookContex public void run(HookContext hookContext) throws Exception { PlanMapper pm = ((PrivateHookContext) hookContext).getContext().getPlanMapper(); - if (pm.isBroken()) { - LOG.warn("Skip checking signatures. The PlanMapper is broken"); - return; - } List sigs = pm.getAll(OpTreeSignature.class); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/PlanMapper.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/PlanMapper.java index 4fd55b480b64..52a2cf2a7437 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/PlanMapper.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/PlanMapper.java @@ -37,6 +37,7 @@ import org.apache.hadoop.hive.ql.optimizer.signature.OpTreeSignatureFactory; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Sets; +import org.apache.hadoop.hive.ql.stats.OperatorStats; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -48,9 +49,8 @@ public class PlanMapper { private static final Logger LOG = LoggerFactory.getLogger(PlanMapper.class); - private final Set groups = new HashSet<>(); - private final Map objectMap = new CompositeMap<>(OpTreeSignature.class, AuxOpTreeSignature.class); - private boolean isBroken = false; + Set groups = new HashSet<>(); + private Map objectMap = new CompositeMap<>(OpTreeSignature.class, AuxOpTreeSignature.class); /** * Specialized class which can compare by identity or value; based on the key type. @@ -222,7 +222,8 @@ private void link(Object o1, Object o2, boolean mayMerge) { if (mGroups.size() > 1) { if (!mayMerge) { LOG.warn("Illegally linking {} and {}", o1, o2); - isBroken = true; + mGroups.forEach(g -> g.add(new OperatorStats.IncorrectRuntimeStatsMarker())); + return; } EquivGroup newGrp = new EquivGroup(); newGrp.add(o1); @@ -253,15 +254,7 @@ private Object getKeyFor(Object o) { return o; } - public boolean isBroken() { - return isBroken; - } - public List getAll(Class clazz) { - if (isBroken) { - LOG.warn("PlanMapper#getAll is no longer valid. Please use PlanMapper#isBroken to handle the state correctly"); - return Collections.emptyList(); - } List ret = new ArrayList<>(); for (EquivGroup g : groups) { ret.addAll(g.getAll(clazz)); @@ -288,11 +281,6 @@ public T lookup(Class clazz, Object key) { } public Iterator iterateGroups() { - if (isBroken) { - LOG.warn("PlanMapper#iterateGroups is no longer valid. " - + "Please use PlanMapper#isBroken to handle the state correctly"); - return Collections.emptyIterator(); - } return groups.iterator(); } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/StatsSources.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/StatsSources.java index 6d5eef26426c..fc265d0223f6 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/StatsSources.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/mapper/StatsSources.java @@ -77,11 +77,6 @@ public static StatsSource getStatsSourceContaining(StatsSource currentStatsSourc private static ImmutableList extractStatsFromPlanMapper(PlanMapper pm) { Builder li = ImmutableList.builder(); - if (pm.isBroken()) { - LOG.warn("Don't generate any stats. This PlanMapper is broken"); - return li.build(); - } - Iterator it = pm.iterateGroups(); while (it.hasNext()) { EquivGroup e = it.next(); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/reexec/ReOptimizePlugin.java b/ql/src/java/org/apache/hadoop/hive/ql/reexec/ReOptimizePlugin.java index a0920da6c616..fd9b47ccba25 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/reexec/ReOptimizePlugin.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/reexec/ReOptimizePlugin.java @@ -102,14 +102,6 @@ public void prepareToReExecute() { @Override public boolean shouldReExecuteAfterCompile(int executionNum, PlanMapper oldPlanMapper, PlanMapper newPlanMapper) { - if (oldPlanMapper.isBroken() || newPlanMapper.isBroken()) { - LOG.warn( - "Giving up a re-execution. The old plan mapper is {}, and the new one is {}", - oldPlanMapper.isBroken() ? "broken" : "not broken", - newPlanMapper.isBroken() ? "broken" : "not broken"); - return false; - } - boolean planDidChange = !planEquals(oldPlanMapper, newPlanMapper); LOG.info("planDidChange: {}", planDidChange); return planDidChange;