diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ppd/OpProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/ppd/OpProcFactory.java index c6894b3..ffaf5cc 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/ppd/OpProcFactory.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/ppd/OpProcFactory.java @@ -149,17 +149,17 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, } } - + public static class PTFPPD extends ScriptPPD { - + /* * For WindowingTableFunction if: - * a. there is a Rank/DenseRank function: if there are unpushedPred of the form + * a. there is a Rank/DenseRank function: if there are unpushedPred of the form * rnkValue < Constant; then use the smallest Constant val as the 'rankLimit' * on the WindowingTablFn. - * b. If there are no Wdw Fns with an End Boundary past the current row, the + * b. If there are no Wdw Fns with an End Boundary past the current row, the * condition can be pushed down as a limit pushdown(mapGroupBy=true) - * + * * (non-Javadoc) * @see org.apache.hadoop.hive.ql.ppd.OpProcFactory.ScriptPPD#process(org.apache.hadoop.hive.ql.lib.Node, java.util.Stack, org.apache.hadoop.hive.ql.lib.NodeProcessorCtx, java.lang.Object[]) */ @@ -170,30 +170,30 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, + ((Operator) nd).getIdentifier() + ")"); OpWalkerInfo owi = (OpWalkerInfo) procCtx; PTFOperator ptfOp = (PTFOperator) nd; - + pushRankLimit(ptfOp, owi); return super.process(nd, stack, procCtx, nodeOutputs); } - + private void pushRankLimit(PTFOperator ptfOp, OpWalkerInfo owi) throws SemanticException { PTFDesc conf = ptfOp.getConf(); - + if ( !conf.forWindowing() ) { return; } - + float threshold = owi.getParseContext().getConf().getFloatVar(HiveConf.ConfVars.HIVELIMITPUSHDOWNMEMORYUSAGE); if (threshold <= 0 || threshold >= 1) { return; } - + WindowTableFunctionDef wTFn = (WindowTableFunctionDef) conf.getFuncDef(); List rFnIdxs = rankingFunctions(wTFn); - + if ( rFnIdxs.size() == 0 ) { return; } - + ExprWalkerInfo childInfo = getChildWalkerInfo(ptfOp, owi); if (childInfo == null) { @@ -207,7 +207,7 @@ private void pushRankLimit(PTFOperator ptfOp, OpWalkerInfo owi) throws SemanticE preds = ExprNodeDescUtils.split(pred, preds); } } - + int rLimit = -1; int fnIdx = -1; for(ExprNodeDesc pred : preds) { @@ -219,7 +219,7 @@ private void pushRankLimit(PTFOperator ptfOp, OpWalkerInfo owi) throws SemanticE } } } - + if ( rLimit != -1 ) { wTFn.setRankLimit(rLimit); wTFn.setRankLimitFunction(fnIdx); @@ -228,68 +228,68 @@ private void pushRankLimit(PTFOperator ptfOp, OpWalkerInfo owi) throws SemanticE } } } - + private List rankingFunctions(WindowTableFunctionDef wTFn) { List rFns = new ArrayList(); for(int i=0; i < wTFn.getWindowFunctions().size(); i++ ) { WindowFunctionDef wFnDef = wTFn.getWindowFunctions().get(i); - if ( (wFnDef.getWFnEval() instanceof GenericUDAFRankEvaluator) || + if ( (wFnDef.getWFnEval() instanceof GenericUDAFRankEvaluator) || (wFnDef.getWFnEval() instanceof GenericUDAFDenseRankEvaluator ) ) { rFns.add(i); } } return rFns; } - + /* * For a predicate check if it is a candidate for pushing down as limit optimization. * The expression must be of the form rankFn <|<= constant. */ private int[] getLimit(WindowTableFunctionDef wTFn, List rFnIdxs, ExprNodeDesc expr) { - + if ( !(expr instanceof ExprNodeGenericFuncDesc) ) { return null; } - + ExprNodeGenericFuncDesc fExpr = (ExprNodeGenericFuncDesc) expr; - - if ( !(fExpr.getGenericUDF() instanceof GenericUDFOPLessThan) && + + if ( !(fExpr.getGenericUDF() instanceof GenericUDFOPLessThan) && !(fExpr.getGenericUDF() instanceof GenericUDFOPEqualOrLessThan) ) { return null; } - + if ( !(fExpr.getChildren().get(0) instanceof ExprNodeColumnDesc) ) { return null; } - + if ( !(fExpr.getChildren().get(1) instanceof ExprNodeConstantDesc) ) { return null; } - + ExprNodeConstantDesc constantExpr = (ExprNodeConstantDesc) fExpr.getChildren().get(1) ; - + if ( constantExpr.getTypeInfo() != TypeInfoFactory.intTypeInfo ) { return null; } - + int limit = (Integer) constantExpr.getValue(); if ( fExpr.getGenericUDF() instanceof GenericUDFOPEqualOrLessThan ) { limit = limit + 1; } String colName = ((ExprNodeColumnDesc)fExpr.getChildren().get(0)).getColumn(); - + for(int i=0; i < rFnIdxs.size(); i++ ) { String fAlias = wTFn.getWindowFunctions().get(i).getAlias(); if ( fAlias.equals(colName)) { return new int[] {limit,i}; } } - + return null; } - + /* - * Limit can be pushed down to Map-side if all Window Functions need access + * Limit can be pushed down to Map-side if all Window Functions need access * to rows before the current row. This is true for: * 1. Rank, DenseRank and Lead Fns. (the window doesn't matter for lead fn). * 2. If the Window for the function is Row based and the End Boundary doesn't @@ -298,8 +298,8 @@ private void pushRankLimit(PTFOperator ptfOp, OpWalkerInfo owi) throws SemanticE private boolean canPushLimitToReduceSink(WindowTableFunctionDef wTFn) { for(WindowFunctionDef wFnDef : wTFn.getWindowFunctions() ) { - if ( (wFnDef.getWFnEval() instanceof GenericUDAFRankEvaluator) || - (wFnDef.getWFnEval() instanceof GenericUDAFDenseRankEvaluator ) || + if ( (wFnDef.getWFnEval() instanceof GenericUDAFRankEvaluator) || + (wFnDef.getWFnEval() instanceof GenericUDAFDenseRankEvaluator ) || (wFnDef.getWFnEval() instanceof GenericUDAFLeadEvaluator ) ) { continue; } @@ -314,18 +314,18 @@ private boolean canPushLimitToReduceSink(WindowTableFunctionDef wTFn) { } return true; } - + private void pushRankLimitToRedSink(PTFOperator ptfOp, HiveConf conf, int rLimit) throws SemanticException { - + Operator parent = ptfOp.getParentOperators().get(0); Operator gP = parent == null ? null : parent.getParentOperators().get(0); - + if ( gP == null || !(gP instanceof ReduceSinkOperator )) { return; } - + float threshold = conf.getFloatVar(HiveConf.ConfVars.HIVELIMITPUSHDOWNMEMORYUSAGE); - + ReduceSinkOperator rSink = (ReduceSinkOperator) gP; ReduceSinkDesc rDesc = rSink.getConf(); rDesc.setTopN(rLimit); @@ -543,7 +543,7 @@ protected Object handlePredicates(Node nd, ExprWalkerInfo prunePreds, OpWalkerIn private void applyFilterTransitivity(JoinOperator nd, OpWalkerInfo owi) throws SemanticException { ExprWalkerInfo prunePreds = - owi.getPrunedPreds((Operator) nd); + owi.getPrunedPreds(nd); if (prunePreds != null) { // We want to use the row resolvers of the parents of the join op // because the rowresolver refers to the output columns of an operator @@ -579,9 +579,6 @@ private void applyFilterTransitivity(JoinOperator nd, OpWalkerInfo owi) int numColumns = eqExpressions.size(); int numEqualities = eqExpressions.get(0).size(); - // joins[i] is the join between table i and i+1 in the JoinOperator - JoinCondDesc[] joins = (nd).getConf().getConds(); - // oldFilters contains the filters to be pushed down Map> oldFilters = prunePreds.getFinalCandidates(); @@ -632,10 +629,32 @@ private void applyFilterTransitivity(JoinOperator nd, OpWalkerInfo owi) } } } + // Push where false filter transitively + Map> candidates = prunePreds.getNonFinalCandidates(); + List exprs; + // where false is not associated with any alias in candidates + if (null != candidates && candidates.get(null) != null && ((exprs = candidates.get(null)) != null)) { + Iterator itr = exprs.iterator(); + while (itr.hasNext()) { + ExprNodeDesc expr = itr.next(); + if (expr instanceof ExprNodeConstantDesc && Boolean.FALSE.equals(((ExprNodeConstantDesc)expr).getValue())) { + // push this 'where false' expr to all aliases + for (String alias : aliasToRR.keySet()) { + List pushedFilters = newFilters.get(alias); + if (null == pushedFilters) { + newFilters.put(alias, new ArrayList()); + } + newFilters.get(alias).add(expr); + } + // this filter is pushed, we can remove it from non-final candidates. + itr.remove(); + } + } + } for (Entry> aliasToFilters : newFilters.entrySet()){ - owi.getPrunedPreds((Operator) nd) + owi.getPrunedPreds(nd) .addPushDowns(aliasToFilters.getKey(), aliasToFilters.getValue()); } } diff --git a/ql/src/test/queries/clientpositive/optimize_nullscan.q b/ql/src/test/queries/clientpositive/optimize_nullscan.q index 61a71a2..f3b896b 100644 --- a/ql/src/test/queries/clientpositive/optimize_nullscan.q +++ b/ql/src/test/queries/clientpositive/optimize_nullscan.q @@ -23,3 +23,7 @@ select * from (select key from src where false) a left outer join (select value explain extended select * from (select key from src union all select src.key from src left outer join srcpart on src.key = srcpart.key) a where false; select * from (select key from src union all select src.key from src left outer join srcpart on src.key = srcpart.key) a where false; + +explain extended +select * from src s1, src s2 where false and s1.value = s2.value; +select * from src s1, src s2 where false and s1.value = s2.value; diff --git a/ql/src/test/results/clientpositive/optimize_nullscan.q.out b/ql/src/test/results/clientpositive/optimize_nullscan.q.out index fd77ad2..173cb7b 100644 --- a/ql/src/test/results/clientpositive/optimize_nullscan.q.out +++ b/ql/src/test/results/clientpositive/optimize_nullscan.q.out @@ -1772,34 +1772,27 @@ STAGE PLANS: Map Reduce Map Operator Tree: TableScan - alias: srcpart - Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE - GatherStats: false - Reduce Output Operator - key expressions: key (type: string) - sort order: + - Map-reduce partition columns: key (type: string) - Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE - tag: 1 - auto parallelism: false - TableScan alias: src Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE GatherStats: false - Reduce Output Operator - key expressions: key (type: string) - sort order: + - Map-reduce partition columns: key (type: string) - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - tag: 0 - auto parallelism: false + Filter Operator + isSamplingPred: false + predicate: false (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: key (type: string) + sort order: + + Map-reduce partition columns: key (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + tag: 0 + auto parallelism: false Path -> Alias: -#### A masked pattern was here #### + -mr-10004default.src{} [null-subquery2:a-subquery2:src] Path -> Partition: -#### A masked pattern was here #### + -mr-10004default.src{} Partition base file name: src - input format: org.apache.hadoop.mapred.TextInputFormat + input format: org.apache.hadoop.hive.ql.io.OneNullRowInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat properties: COLUMN_STATS_ACCURATE true @@ -1814,10 +1807,10 @@ STAGE PLANS: rawDataSize 5312 serialization.ddl struct src { string key, string value} serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + serialization.lib org.apache.hadoop.hive.serde2.NullStructSerDe totalSize 5812 #### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + serde: org.apache.hadoop.hive.serde2.NullStructSerDe input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -1840,196 +1833,8 @@ STAGE PLANS: serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe name: default.src name: default.src -#### A masked pattern was here #### - Partition - base file name: hr=11 - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - hr 11 - properties: - COLUMN_STATS_ACCURATE true - bucket_count -1 - columns key,value - columns.comments defaultdefault - columns.types string:string -#### A masked pattern was here #### - name default.srcpart - numFiles 1 - numRows 500 - partition_columns ds/hr - partition_columns.types string:string - rawDataSize 5312 - serialization.ddl struct srcpart { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 5812 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - bucket_count -1 - columns key,value - columns.comments defaultdefault - columns.types string:string -#### A masked pattern was here #### - name default.srcpart - partition_columns ds/hr - partition_columns.types string:string - serialization.ddl struct srcpart { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.srcpart - name: default.srcpart -#### A masked pattern was here #### - Partition - base file name: hr=12 - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - hr 12 - properties: - COLUMN_STATS_ACCURATE true - bucket_count -1 - columns key,value - columns.comments defaultdefault - columns.types string:string -#### A masked pattern was here #### - name default.srcpart - numFiles 1 - numRows 500 - partition_columns ds/hr - partition_columns.types string:string - rawDataSize 5312 - serialization.ddl struct srcpart { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 5812 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - bucket_count -1 - columns key,value - columns.comments defaultdefault - columns.types string:string -#### A masked pattern was here #### - name default.srcpart - partition_columns ds/hr - partition_columns.types string:string - serialization.ddl struct srcpart { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.srcpart - name: default.srcpart -#### A masked pattern was here #### - Partition - base file name: hr=11 - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-09 - hr 11 - properties: - COLUMN_STATS_ACCURATE true - bucket_count -1 - columns key,value - columns.comments defaultdefault - columns.types string:string -#### A masked pattern was here #### - name default.srcpart - numFiles 1 - numRows 500 - partition_columns ds/hr - partition_columns.types string:string - rawDataSize 5312 - serialization.ddl struct srcpart { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 5812 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - bucket_count -1 - columns key,value - columns.comments defaultdefault - columns.types string:string -#### A masked pattern was here #### - name default.srcpart - partition_columns ds/hr - partition_columns.types string:string - serialization.ddl struct srcpart { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.srcpart - name: default.srcpart -#### A masked pattern was here #### - Partition - base file name: hr=12 - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-09 - hr 12 - properties: - COLUMN_STATS_ACCURATE true - bucket_count -1 - columns key,value - columns.comments defaultdefault - columns.types string:string -#### A masked pattern was here #### - name default.srcpart - numFiles 1 - numRows 500 - partition_columns ds/hr - partition_columns.types string:string - rawDataSize 5312 - serialization.ddl struct srcpart { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 5812 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - bucket_count -1 - columns key,value - columns.comments defaultdefault - columns.types string:string -#### A masked pattern was here #### - name default.srcpart - partition_columns ds/hr - partition_columns.types string:string - serialization.ddl struct srcpart { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.srcpart - name: default.srcpart Truncated Path -> Alias: - /src [null-subquery2:a-subquery2:src] - /srcpart/ds=2008-04-08/hr=11 [null-subquery2:a-subquery2:srcpart] - /srcpart/ds=2008-04-08/hr=12 [null-subquery2:a-subquery2:srcpart] - /srcpart/ds=2008-04-09/hr=11 [null-subquery2:a-subquery2:srcpart] - /srcpart/ds=2008-04-09/hr=12 [null-subquery2:a-subquery2:srcpart] + -mr-10004default.src{} [null-subquery2:a-subquery2:src] Needs Tagging: true Reduce Operator Tree: Join Operator @@ -2039,32 +1844,28 @@ STAGE PLANS: 0 {KEY.reducesinkkey0} 1 outputColumnNames: _col0 - Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE - Filter Operator - isSamplingPred: false - predicate: false (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE - Select Operator - expressions: _col0 (type: string) - outputColumnNames: _col0 - Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE - File Output Operator - compressed: false - GlobalTableId: 0 + File Output Operator + compressed: false + GlobalTableId: 0 #### A masked pattern was here #### - NumFilesPerFileSink: 1 - table: - input format: org.apache.hadoop.mapred.SequenceFileInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat - properties: - columns _col0 - columns.types string - escape.delim \ - serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe - serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe - TotalFiles: 1 - GatherStats: false - MultiFileSpray: false + NumFilesPerFileSink: 1 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + properties: + columns _col0 + columns.types string + escape.delim \ + serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe + serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false Stage: Stage-2 Map Reduce @@ -2219,17 +2020,192 @@ PREHOOK: query: select * from (select key from src union all select src.key from PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Input: default@srcpart -PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 -PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 -PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 -PREHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 #### A masked pattern was here #### POSTHOOK: query: select * from (select key from src union all select src.key from src left outer join srcpart on src.key = srcpart.key) a where false POSTHOOK: type: QUERY POSTHOOK: Input: default@src POSTHOOK: Input: default@srcpart -POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 -POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 -POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=11 -POSTHOOK: Input: default@srcpart@ds=2008-04-09/hr=12 +#### A masked pattern was here #### +PREHOOK: query: explain extended +select * from src s1, src s2 where false and s1.value = s2.value +PREHOOK: type: QUERY +POSTHOOK: query: explain extended +select * from src s1, src s2 where false and s1.value = s2.value +POSTHOOK: type: QUERY +ABSTRACT SYNTAX TREE: + +TOK_QUERY + TOK_FROM + TOK_JOIN + TOK_TABREF + TOK_TABNAME + src + s1 + TOK_TABREF + TOK_TABNAME + src + s2 + TOK_INSERT + TOK_DESTINATION + TOK_DIR + TOK_TMP_FILE + TOK_SELECT + TOK_SELEXPR + TOK_ALLCOLREF + TOK_WHERE + and + false + = + . + TOK_TABLE_OR_COL + s1 + value + . + TOK_TABLE_OR_COL + s2 + value + + +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: s2 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: false (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: value (type: string) + sort order: + + Map-reduce partition columns: value (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + tag: 1 + value expressions: key (type: string) + auto parallelism: false + TableScan + alias: s1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: false (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Reduce Output Operator + key expressions: value (type: string) + sort order: + + Map-reduce partition columns: value (type: string) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + tag: 0 + value expressions: key (type: string) + auto parallelism: false + Path -> Alias: + -mr-10003default.src{} [s2, s1] + Path -> Partition: + -mr-10003default.src{} + Partition + base file name: src + input format: org.apache.hadoop.hive.ql.io.OneNullRowInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + COLUMN_STATS_ACCURATE true + bucket_count -1 + columns key,value + columns.comments defaultdefault + columns.types string:string +#### A masked pattern was here #### + name default.src + numFiles 1 + numRows 500 + rawDataSize 5312 + serialization.ddl struct src { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.NullStructSerDe + totalSize 5812 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.NullStructSerDe + + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + COLUMN_STATS_ACCURATE true + bucket_count -1 + columns key,value + columns.comments defaultdefault + columns.types string:string +#### A masked pattern was here #### + name default.src + numFiles 1 + numRows 500 + rawDataSize 5312 + serialization.ddl struct src { string key, string value} + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + totalSize 5812 +#### A masked pattern was here #### + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.src + name: default.src + Truncated Path -> Alias: + -mr-10003default.src{} [s2, s1] + Needs Tagging: true + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {VALUE._col0} {KEY.reducesinkkey0} + 1 {VALUE._col0} {KEY.reducesinkkey0} + outputColumnNames: _col0, _col1, _col5, _col6 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + isSamplingPred: false + predicate: false (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: string), _col5 (type: string), _col6 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + columns _col0,_col1,_col2,_col3 + columns.types string:string:string:string + escape.delim \ + hive.serialization.extend.nesting.levels true + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select * from src s1, src s2 where false and s1.value = s2.value +PREHOOK: type: QUERY +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: select * from src s1, src s2 where false and s1.value = s2.value +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src #### A masked pattern was here ####