diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java index 4364f28..0aba5da 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java @@ -94,6 +94,7 @@ import org.apache.hadoop.hive.ql.hooks.WriteEntity; import org.apache.hadoop.hive.ql.io.AcidOutputFormat; import org.apache.hadoop.hive.ql.io.AcidUtils; +import org.apache.hadoop.hive.ql.io.AcidUtils.Operation; import org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; import org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat; import org.apache.hadoop.hive.ql.io.HiveOutputFormat; @@ -229,12 +230,12 @@ private HashMap opToPartPruner; private HashMap opToPartList; private HashMap> topOps; - private HashMap> topSelOps; + private final HashMap> topSelOps; private LinkedHashMap, OpParseContext> opParseCtx; private List loadTableWork; private List loadFileWork; - private Map joinContext; - private Map smbMapJoinContext; + private final Map joinContext; + private final Map smbMapJoinContext; private final HashMap topToTable; private final Map fsopToTable; private final List reduceSinkOperatorsAddedByEnforceBucketingSorting; @@ -5980,8 +5981,12 @@ private Operator genBucketingSortingDest(String dest, Operator input, QB qb, maxReducers = numBuckets; } - input = genReduceSinkPlanForSortingBucketing(dest_tab, input, - sortCols, sortOrders, partnCols, maxReducers); + StringBuilder order = new StringBuilder(); + for (int sortOrder : sortOrders) { + order.append(sortOrder == BaseSemanticAnalyzer.HIVE_COLUMN_ORDER_ASC ? '+' : '-'); + } + input = genReduceSinkPlan(input, partnCols, sortCols, order.toString(), maxReducers, + (isAcidTable(dest_tab) ? getAcidType() : AcidUtils.Operation.NOT_ACID)); ctx.setMultiFileSpray(multiFileSpray); ctx.setNumFiles(numFiles); ctx.setPartnCols(partnColsNoConvert); @@ -6453,7 +6458,7 @@ private Operator genFileSinkPlan(String dest, QB qb, Operator input) fileSinkDesc.setWriteType(wt); acidFileSinks.add(fileSinkDesc); } - + fileSinkDesc.setTemporary(destTableIsTemporary); /* Set List Bucketing context. */ @@ -6935,7 +6940,6 @@ private Operator genLimitMapRedPlan(String dest, QB qb, Operator input, private ArrayList getSortCols(String dest, QB qb, Table tab, TableDesc table_desc, Operator input, boolean convert) throws SemanticException { - RowResolver inputRR = opParseCtx.get(input).getRowResolver(); List tabSortCols = tab.getSortCols(); List tabCols = tab.getCols(); @@ -6945,7 +6949,6 @@ private Operator genLimitMapRedPlan(String dest, QB qb, Operator input, int pos = 0; for (FieldSchema tabCol : tabCols) { if (sortCol.getCol().equals(tabCol.getName())) { - ColumnInfo colInfo = inputRR.getColumnInfos().get(pos); posns.add(pos); break; } @@ -6958,7 +6961,6 @@ private Operator genLimitMapRedPlan(String dest, QB qb, Operator input, private ArrayList getSortOrders(String dest, QB qb, Table tab, Operator input) throws SemanticException { - RowResolver inputRR = opParseCtx.get(input).getRowResolver(); List tabSortCols = tab.getSortCols(); List tabCols = tab.getCols(); @@ -6974,74 +6976,11 @@ private Operator genLimitMapRedPlan(String dest, QB qb, Operator input, return orders; } - @SuppressWarnings("nls") - private Operator genReduceSinkPlanForSortingBucketing(Table tab, Operator input, - ArrayList sortCols, - List sortOrders, - ArrayList partitionCols, - int numReducers) - throws SemanticException { - RowResolver inputRR = opParseCtx.get(input).getRowResolver(); - - // For the generation of the values expression just get the inputs - // signature and generate field expressions for those - Map colExprMap = new HashMap(); - ArrayList valueCols = new ArrayList(); - ArrayList outputColumns = new ArrayList(); - int i = 0; - for (ColumnInfo colInfo : inputRR.getColumnInfos()) { - String internalName = getColumnInternalName(i++); - outputColumns.add(internalName); - valueCols.add(new ExprNodeColumnDesc(colInfo)); - colExprMap.put(internalName, valueCols - .get(valueCols.size() - 1)); - } - - StringBuilder order = new StringBuilder(); - for (int sortOrder : sortOrders) { - order.append(sortOrder == BaseSemanticAnalyzer.HIVE_COLUMN_ORDER_ASC ? '+' : '-'); - } - - AcidUtils.Operation acidOp = (isAcidTable(tab) ? getAcidType() : AcidUtils.Operation.NOT_ACID); - - Operator interim = putOpInsertMap(OperatorFactory.getAndMakeChild(PlanUtils - .getReduceSinkDesc(sortCols, valueCols, outputColumns, false, -1, - partitionCols, order.toString(), numReducers, acidOp), - new RowSchema(inputRR.getColumnInfos()), input), inputRR); - interim.setColumnExprMap(colExprMap); - reduceSinkOperatorsAddedByEnforceBucketingSorting.add((ReduceSinkOperator) interim); - - // Add the extract operator to get the value fields - RowResolver out_rwsch = new RowResolver(); - RowResolver interim_rwsch = inputRR; - Integer pos = Integer.valueOf(0); - for (ColumnInfo colInfo : interim_rwsch.getColumnInfos()) { - String[] info = interim_rwsch.reverseLookup(colInfo.getInternalName()); - out_rwsch.put(info[0], info[1], new ColumnInfo( - getColumnInternalName(pos), colInfo.getType(), info[0], - colInfo.getIsVirtualCol(), colInfo.isHiddenVirtualCol())); - pos = Integer.valueOf(pos.intValue() + 1); - } - - Operator output = putOpInsertMap(OperatorFactory.getAndMakeChild( - new ExtractDesc(new ExprNodeColumnDesc(TypeInfoFactory.stringTypeInfo, - Utilities.ReduceField.VALUE.toString(), "", false)), new RowSchema( - out_rwsch.getColumnInfos()), interim), out_rwsch); - - if (LOG.isDebugEnabled()) { - LOG.debug("Created ReduceSink Plan for table: " + tab.getTableName() + - " row schema: " + out_rwsch.toString()); - } - - return output; - - } - private Operator genReduceSinkPlan(String dest, QB qb, Operator input, int numReducers) throws SemanticException { - + RowResolver inputRR = opParseCtx.get(input).getRowResolver(); - + // First generate the expression for the partition and sort keys // The cluster by clause / distribute by clause has the aliases for // partition function @@ -7099,16 +7038,16 @@ private Operator genReduceSinkPlan(String dest, QB qb, Operator input, sortCols.add(exprNode); } } - return genReduceSinkPlan(input, partCols, sortCols, order.toString(), numReducers); + return genReduceSinkPlan(input, partCols, sortCols, order.toString(), numReducers, Operation.NOT_ACID); } - + @SuppressWarnings("nls") private Operator genReduceSinkPlan(Operator input, - ArrayList partitionCols, ArrayList sortCols, - String sortOrder, int numReducers) throws SemanticException { + ArrayList partitionCols, ArrayList sortCols, + String sortOrder, int numReducers, AcidUtils.Operation acidOp) throws SemanticException { RowResolver inputRR = opParseCtx.get(input).getRowResolver(); - + Operator dummy = Operator.createDummy(); dummy.setParentOperators(Arrays.asList(input)); @@ -7171,9 +7110,8 @@ private Operator genReduceSinkPlan(Operator input, dummy.setParentOperators(null); - // TODO Not 100% sure NOT_ACID is always right here. ReduceSinkDesc rsdesc = PlanUtils.getReduceSinkDesc(sortCols, valueCols, outputColumns, - false, -1, partitionCols, sortOrder, numReducers, AcidUtils.Operation.NOT_ACID); + false, -1, partitionCols, sortOrder, numReducers, acidOp); Operator interim = putOpInsertMap(OperatorFactory.getAndMakeChild(rsdesc, new RowSchema(rsRR.getColumnInfos()), input), rsRR); @@ -11899,13 +11837,8 @@ Operator genPTFPlan(PTFInvocationSpec ptfQSpec, Operator input) throws SemanticE void buildPTFReduceSinkDetails(PartitionedTableFunctionDef tabDef, RowResolver inputRR, ArrayList partCols, - ArrayList valueCols, ArrayList orderCols, - Map colExprMap, - List outputColumnNames, - StringBuilder orderString, - RowResolver rsOpRR, - RowResolver extractRR) throws SemanticException { + StringBuilder orderString) throws SemanticException { List partColList = tabDef.getPartition().getExpressions(); @@ -11933,68 +11866,6 @@ void buildPTFReduceSinkDetails(PartitionedTableFunctionDef tabDef, } orderCols.add(colDef.getExprNode()); } - - ArrayList colInfoList = inputRR.getColumnInfos(); - /* - * construct the ReduceSinkRR - */ - int pos = 0; - for (ColumnInfo colInfo : colInfoList) { - ExprNodeDesc valueColExpr = new ExprNodeColumnDesc(colInfo); - valueCols.add(valueColExpr); - String internalName = SemanticAnalyzer.getColumnInternalName(pos++); - outputColumnNames.add(internalName); - colExprMap.put(internalName, valueColExpr); - - String[] alias = inputRR.reverseLookup(colInfo.getInternalName()); - ColumnInfo newColInfo = new ColumnInfo( - internalName, colInfo.getType(), alias[0], - colInfo.getIsVirtualCol(), colInfo.isHiddenVirtualCol()); - rsOpRR.put(alias[0], alias[1], newColInfo); - } - - /* - * construct the ExtractRR - */ - LinkedHashMap colsAddedByHaving = - new LinkedHashMap(); - pos = 0; - for (ColumnInfo colInfo : colInfoList) { - String[] alias = inputRR.reverseLookup(colInfo.getInternalName()); - /* - * if we have already encountered this colInfo internalName. - * We encounter it again because it must be put for the Having clause. - * We will add these entries in the end; in a loop on colsAddedByHaving. See below. - */ - if ( colsAddedByHaving.containsKey(alias)) { - continue; - } - ASTNode astNode = PTFTranslator.getASTNode(colInfo, inputRR); - ColumnInfo eColInfo = new ColumnInfo( - SemanticAnalyzer.getColumnInternalName(pos++), colInfo.getType(), alias[0], - colInfo.getIsVirtualCol(), colInfo.isHiddenVirtualCol()); - - if ( astNode == null ) { - extractRR.put(alias[0], alias[1], eColInfo); - } - else { - /* - * in case having clause refers to this column may have been added twice; - * once with the ASTNode.toStringTree as the alias - * and then with the real alias. - */ - extractRR.putExpression(astNode, eColInfo); - if ( !astNode.toStringTree().toLowerCase().equals(alias[1]) ) { - colsAddedByHaving.put(alias, eColInfo); - } - } - } - - for(Map.Entry columnAddedByHaving : colsAddedByHaving.entrySet() ) { - String[] alias = columnAddedByHaving.getKey(); - ColumnInfo eColInfo = columnAddedByHaving.getValue(); - extractRR.put(alias[0], alias[1], eColInfo); - } } private Operator genPTFPlanForComponentQuery(PTFInvocationSpec ptfQSpec, Operator input) @@ -12005,27 +11876,6 @@ private Operator genPTFPlanForComponentQuery(PTFInvocationSpec ptfQSpec, Operato RowResolver rr = opParseCtx.get(input).getRowResolver(); PTFDesc ptfDesc = translatePTFInvocationSpec(ptfQSpec, rr); - RowResolver rsOpRR = new RowResolver(); - /* - * Build an RR for the Extract Op from the ReduceSink Op's RR. - * Why? - * We need to remove the Virtual Columns present in the RS's RR. The OI - * that gets passed to Extract at runtime doesn't contain the Virtual Columns. - * So internal names get changed. Consider testCase testJoinWithLeadLag, - * which is a self join on part and also has a Windowing expression. - * The RR of the RS op at translation time looks something like this: - * (_co1,_col2,..,_col7, _col8(vc=true),_col9(vc=true), - * _col10,_col11,.._col15(vc=true),_col16(vc=true),..) - * At runtime the Virtual columns are removed and all the columns after _col7 - * are shifted 1 or 2 positions. - * So in child Operators ColumnExprNodeDesc's are no longer referring to the right columns. - * - * So we build a new RR for the Extract Op, with the Virtual Columns removed. - * We hand this to the PTFTranslator as the - * starting RR to use to translate a PTF Chain. - */ - RowResolver extractOpRR = new RowResolver(); - /* * 2. build Map-side Op Graph. Graph template is either: * Input -> PTF_map -> ReduceSink @@ -12056,10 +11906,7 @@ private Operator genPTFPlanForComponentQuery(PTFInvocationSpec ptfQSpec, Operato */ ArrayList partCols = new ArrayList(); - ArrayList valueCols = new ArrayList(); ArrayList orderCols = new ArrayList(); - Map colExprMap = new HashMap(); - List outputColumnNames = new ArrayList(); StringBuilder orderString = new StringBuilder(); /* @@ -12068,45 +11915,20 @@ private Operator genPTFPlanForComponentQuery(PTFInvocationSpec ptfQSpec, Operato * If the parent of ReduceSinkOperator is PTFOperator, use it's * output RR. */ - buildPTFReduceSinkDetails(tabDef, - rr, - partCols, - valueCols, - orderCols, - colExprMap, - outputColumnNames, - orderString, - rsOpRR, - extractOpRR); - - input = putOpInsertMap(OperatorFactory.getAndMakeChild(PlanUtils - .getReduceSinkDesc(orderCols, - valueCols, outputColumnNames, false, - -1, partCols, orderString.toString(), -1, AcidUtils.Operation.NOT_ACID), - new RowSchema(rsOpRR.getColumnInfos()), input), rsOpRR); - input.setColumnExprMap(colExprMap); + buildPTFReduceSinkDetails(tabDef, rr, partCols, orderCols, orderString); + input = genReduceSinkPlan(input, partCols, orderCols, orderString.toString(), -1, Operation.NOT_ACID); } /* * 3. build Reduce-side Op Graph */ { - /* - * b. Construct Extract Operator. - */ - input = putOpInsertMap(OperatorFactory.getAndMakeChild( - new ExtractDesc( - new ExprNodeColumnDesc(TypeInfoFactory.stringTypeInfo, - Utilities.ReduceField.VALUE - .toString(), "", false)), - new RowSchema(extractOpRR.getColumnInfos()), - input), extractOpRR); /* * c. Rebuilt the QueryDef. * Why? * - so that the ExprNodeDescriptors in the QueryDef are based on the - * Extract Operator's RowResolver + * Select Operator's RowResolver */ rr = opParseCtx.get(input).getRowResolver(); ptfDesc = translatePTFInvocationSpec(ptfQSpec, rr); @@ -12120,9 +11942,7 @@ private Operator genPTFPlanForComponentQuery(PTFInvocationSpec ptfQSpec, Operato input), ptfOpRR); } - return input; - } //--------------------------- Windowing handling: PTFInvocationSpec to PTFDesc -------------------- @@ -12150,7 +11970,7 @@ Operator genWindowingPlan(WindowingSpec wSpec, Operator input) throws SemanticEx private Operator genReduceSinkPlanForWindowing(WindowingSpec spec, RowResolver inputRR, Operator input) throws SemanticException{ - + ArrayList partCols = new ArrayList(); ArrayList orderCols = new ArrayList(); StringBuilder order = new StringBuilder(); @@ -12174,7 +11994,7 @@ private Operator genReduceSinkPlanForWindowing(WindowingSpec spec, } } - return genReduceSinkPlan(input, partCols, orderCols, order.toString(), -1); + return genReduceSinkPlan(input, partCols, orderCols, order.toString(), -1, Operation.NOT_ACID); } public static ArrayList parseSelect(String selectExprStr) diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/SelectDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/SelectDesc.java index fa6b548..cfcfe17 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/plan/SelectDesc.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/SelectDesc.java @@ -56,14 +56,6 @@ public SelectDesc( this.outputColumnNames = outputColumnNames; } - public SelectDesc( - final List colList, - final boolean selectStar, final boolean selStarNoCompute) { - this.colList = colList; - this.selectStar = selectStar; - this.selStarNoCompute = selStarNoCompute; - } - @Override public Object clone() { SelectDesc ret = new SelectDesc(); diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java index 41862e6..730823f 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java @@ -38,7 +38,7 @@ import org.apache.hadoop.hive.ql.exec.mr.ExecDriver; import org.apache.hadoop.hive.ql.exec.mr.MapRedTask; import org.apache.hadoop.hive.ql.io.AcidUtils; -import org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat; +import org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat; import org.apache.hadoop.hive.ql.metadata.Hive; import org.apache.hadoop.hive.ql.metadata.Table; import org.apache.hadoop.hive.ql.parse.SemanticException; @@ -47,7 +47,6 @@ import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; import org.apache.hadoop.hive.ql.plan.ExprNodeFieldDesc; import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc; -import org.apache.hadoop.hive.ql.plan.ExtractDesc; import org.apache.hadoop.hive.ql.plan.FileSinkDesc; import org.apache.hadoop.hive.ql.plan.FilterDesc; import org.apache.hadoop.hive.ql.plan.MapredWork; @@ -94,7 +93,7 @@ tmppath = new Path(tmpdir); fs = FileSystem.get(conf); - if (fs.exists(tmppath) && !fs.getFileStatus(tmppath).isDir()) { + if (fs.exists(tmppath) && !fs.getFileStatus(tmppath).isDirectory()) { throw new RuntimeException(tmpdir + " exists but is not a directory"); } @@ -137,7 +136,7 @@ for (String src : srctables) { db.dropTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, src, true, true); db.createTable(src, cols, null, TextInputFormat.class, - IgnoreKeyTextOutputFormat.class); + HiveIgnoreKeyTextOutputFormat.class); db.loadTable(hadoopDataFile[i], src, false, false, true, false, false); i++; } @@ -161,20 +160,19 @@ public static void addMapWork(MapredWork mr, Table tbl, String alias, Operator op3 = OperatorFactory.get(new FileSinkDesc(new Path(tmpdir + File.separator + "mapredplan1.out"), Utilities.defaultTd, false)); - Operator op2 = OperatorFactory.get(new ExtractDesc( - getStringColumn(Utilities.ReduceField.VALUE.toString())), op3); + List cols = new ArrayList(); + cols.add(getStringColumn(Utilities.ReduceField.VALUE.toString()+"."+outputColumns.get(1))); + List colNames = new ArrayList(); + colNames.add(HiveConf.getColumnInternalName(2)); + Operator op2 = OperatorFactory.get(new SelectDesc(cols, colNames), op3); rWork.setReducer(op2); } @@ -292,8 +293,10 @@ private void populateMapRedPlan2(Table src) throws Exception { Operator op3 = OperatorFactory.get(getTestFilterDesc("0"), op4); - Operator op2 = OperatorFactory.get(new ExtractDesc( - getStringColumn(Utilities.ReduceField.VALUE.toString())), op3); + List cols = new ArrayList(); + cols.add(getStringColumn(Utilities.ReduceField.KEY + ".reducesinkkey" + 0)); + cols.add(getStringColumn(Utilities.ReduceField.VALUE.toString()+"."+outputColumns.get(1))); + Operator op2 = OperatorFactory.get(new SelectDesc(cols, outputColumns), op3); rWork.setReducer(op2); } @@ -376,10 +379,10 @@ private void populateMapRedPlan4(Table src) throws SemanticException { // reduce side work Operator op3 = OperatorFactory.get(new FileSinkDesc(new Path(tmpdir + File.separator + "mapredplan4.out"), Utilities.defaultTd, false)); - - Operator op2 = OperatorFactory.get(new ExtractDesc( - getStringColumn(Utilities.ReduceField.VALUE.toString())), op3); - + List cols = new ArrayList(); + cols.add(getStringColumn(Utilities.ReduceField.KEY + ".reducesinkkey" + 0)); + cols.add(getStringColumn(Utilities.ReduceField.VALUE.toString()+"."+outputColumns.get(1))); + Operator op2 = OperatorFactory.get(new SelectDesc(cols, outputColumns), op3); rWork.setReducer(op2); } @@ -416,9 +419,10 @@ private void populateMapRedPlan5(Table src) throws SemanticException { Operator op3 = OperatorFactory.get(new FileSinkDesc(new Path(tmpdir + File.separator + "mapredplan5.out"), Utilities.defaultTd, false)); - Operator op2 = OperatorFactory.get(new ExtractDesc( - getStringColumn(Utilities.ReduceField.VALUE.toString())), op3); - + List cols = new ArrayList(); + cols.add(getStringColumn(Utilities.ReduceField.KEY + ".reducesinkkey" + 0)); + cols.add(getStringColumn(Utilities.ReduceField.VALUE.toString()+"."+outputColumns.get(1))); + Operator op2 = OperatorFactory.get(new SelectDesc(cols, outputColumns), op3); rWork.setReducer(op2); } @@ -459,8 +463,10 @@ private void populateMapRedPlan6(Table src) throws Exception { Operator op2 = OperatorFactory.get(getTestFilterDesc("0"), op3); - Operator op5 = OperatorFactory.get(new ExtractDesc( - getStringColumn(Utilities.ReduceField.VALUE.toString())), op2); + List cols = new ArrayList(); + cols.add(getStringColumn(Utilities.ReduceField.KEY + ".reducesinkkey" + 0)); + cols.add(getStringColumn(Utilities.ReduceField.VALUE.toString()+"."+outputColumns.get(1))); + Operator op5 = OperatorFactory.get(new SelectDesc(cols, outputColumns), op2); rWork.setReducer(op5); } diff --git a/ql/src/test/results/clientpositive/bucket1.q.out b/ql/src/test/results/clientpositive/bucket1.q.out index 13ec735..8009514 100644 --- a/ql/src/test/results/clientpositive/bucket1.q.out +++ b/ql/src/test/results/clientpositive/bucket1.q.out @@ -110,39 +110,37 @@ STAGE PLANS: /src [src] Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: UDFToInteger(VALUE._col0) (type: int), VALUE._col1 (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: UDFToInteger(_col0) (type: int), _col1 (type: string) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - GlobalTableId: 1 + File Output Operator + compressed: false + GlobalTableId: 1 #### A masked pattern was here #### - NumFilesPerFileSink: 1 - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + NumFilesPerFileSink: 1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE 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: - bucket_count 100 - bucket_field_name key - columns key,value - columns.comments - columns.types int:string + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count 100 + bucket_field_name key + columns key,value + columns.comments + columns.types int:string #### A masked pattern was here #### - name default.bucket1_1 - serialization.ddl struct bucket1_1 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name default.bucket1_1 + serialization.ddl struct bucket1_1 { i32 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.bucket1_1 - TotalFiles: 1 - GatherStats: true - MultiFileSpray: false + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket1_1 + TotalFiles: 1 + GatherStats: true + MultiFileSpray: false Stage: Stage-0 Move Operator diff --git a/ql/src/test/results/clientpositive/bucket2.q.out b/ql/src/test/results/clientpositive/bucket2.q.out index 32a77c3..75de8a9 100644 --- a/ql/src/test/results/clientpositive/bucket2.q.out +++ b/ql/src/test/results/clientpositive/bucket2.q.out @@ -110,39 +110,37 @@ STAGE PLANS: /src [src] Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: UDFToInteger(VALUE._col0) (type: int), VALUE._col1 (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: UDFToInteger(_col0) (type: int), _col1 (type: string) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - GlobalTableId: 1 + File Output Operator + compressed: false + GlobalTableId: 1 #### A masked pattern was here #### - NumFilesPerFileSink: 2 - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + NumFilesPerFileSink: 2 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE 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: - bucket_count 2 - bucket_field_name key - columns key,value - columns.comments - columns.types int:string + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types int:string #### A masked pattern was here #### - name default.bucket2_1 - serialization.ddl struct bucket2_1 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name default.bucket2_1 + serialization.ddl struct bucket2_1 { i32 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.bucket2_1 - TotalFiles: 2 - GatherStats: true - MultiFileSpray: true + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket2_1 + TotalFiles: 2 + GatherStats: true + MultiFileSpray: true Stage: Stage-0 Move Operator diff --git a/ql/src/test/results/clientpositive/bucket3.q.out b/ql/src/test/results/clientpositive/bucket3.q.out index ff7173e..c459870 100644 --- a/ql/src/test/results/clientpositive/bucket3.q.out +++ b/ql/src/test/results/clientpositive/bucket3.q.out @@ -114,42 +114,40 @@ STAGE PLANS: /src [src] Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: UDFToInteger(VALUE._col0) (type: int), VALUE._col1 (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: UDFToInteger(_col0) (type: int), _col1 (type: string) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - GlobalTableId: 1 + File Output Operator + compressed: false + GlobalTableId: 1 #### A masked pattern was here #### - NumFilesPerFileSink: 2 - Static Partition Specification: ds=1/ - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + NumFilesPerFileSink: 2 + Static Partition Specification: ds=1/ + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE 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: - bucket_count 2 - bucket_field_name key - columns key,value - columns.comments - columns.types int:string + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types int:string #### A masked pattern was here #### - name default.bucket3_1 - partition_columns ds - partition_columns.types string - serialization.ddl struct bucket3_1 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name default.bucket3_1 + partition_columns ds + partition_columns.types string + serialization.ddl struct bucket3_1 { i32 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.bucket3_1 - TotalFiles: 2 - GatherStats: true - MultiFileSpray: true + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket3_1 + TotalFiles: 2 + GatherStats: true + MultiFileSpray: true Stage: Stage-0 Move Operator diff --git a/ql/src/test/results/clientpositive/bucketsortoptimize_insert_1.q.out b/ql/src/test/results/clientpositive/bucketsortoptimize_insert_1.q.out index 75de953..0df6c41 100644 --- a/ql/src/test/results/clientpositive/bucketsortoptimize_insert_1.q.out +++ b/ql/src/test/results/clientpositive/bucketsortoptimize_insert_1.q.out @@ -66,14 +66,25 @@ STAGE PLANS: expressions: key (type: int), value (type: string) outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.test_table2 + value expressions: _col1 (type: string) + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table2 Stage: Stage-0 Move Operator @@ -173,14 +184,25 @@ STAGE PLANS: expressions: key (type: int), value (type: string) outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.test_table2 + value expressions: _col1 (type: string) + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table2 Stage: Stage-0 Move Operator @@ -282,14 +304,25 @@ STAGE PLANS: expressions: key (type: int), concat(value, value) (type: string) outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.test_table2 + value expressions: _col1 (type: string) + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table2 Stage: Stage-0 Move Operator @@ -343,9 +376,11 @@ STAGE PLANS: sort order: + Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int), _col1 (type: string) + value expressions: _col1 (type: string) Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false @@ -403,14 +438,25 @@ STAGE PLANS: expressions: key (type: int), concat(value, value) (type: string) outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.test_table2 + value expressions: _col1 (type: string) + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table2 Stage: Stage-0 Move Operator diff --git a/ql/src/test/results/clientpositive/bucketsortoptimize_insert_2.q.out b/ql/src/test/results/clientpositive/bucketsortoptimize_insert_2.q.out index 599b8b9..453ed97 100644 --- a/ql/src/test/results/clientpositive/bucketsortoptimize_insert_2.q.out +++ b/ql/src/test/results/clientpositive/bucketsortoptimize_insert_2.q.out @@ -95,21 +95,44 @@ FROM test_table1 a JOIN test_table2 b ON a.key = b.key WHERE a.ds = '1' and b.ds = '1' POSTHOOK: type: QUERY STAGE DEPENDENCIES: - Stage-1 is a root stage - Stage-0 depends on stages: Stage-1 + Stage-6 is a root stage , consists of Stage-7, Stage-8, Stage-1 + Stage-7 has a backup stage: Stage-1 + Stage-4 depends on stages: Stage-7 + Stage-0 depends on stages: Stage-1, Stage-4, Stage-5 Stage-2 depends on stages: Stage-0 + Stage-8 has a backup stage: Stage-1 + Stage-5 depends on stages: Stage-8 + Stage-1 STAGE PLANS: - Stage: Stage-1 + Stage: Stage-6 + Conditional Operator + + Stage: Stage-7 + Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + b + TableScan + alias: b + Filter Operator + predicate: key is not null (type: boolean) + HashTable Sink Operator + keys: + 0 key (type: int) + 1 key (type: int) + + Stage: Stage-4 Map Reduce Map Operator Tree: TableScan alias: a - Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE - Sorted Merge Bucket Map Join Operator + Map Join Operator condition map: Inner Join 0 to 1 keys: @@ -119,13 +142,24 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), concat(_col1, _col7) (type: string) outputColumnNames: _col0, _col1 - File Output Operator - compressed: false - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.test_table3 + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + value expressions: _col1 (type: string) + Local Work: + Map Reduce Local Work + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table3 Stage: Stage-0 Move Operator @@ -142,6 +176,95 @@ STAGE PLANS: Stage: Stage-2 Stats-Aggr Operator + Stage: Stage-8 + Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a + TableScan + alias: a + Filter Operator + predicate: key is not null (type: boolean) + HashTable Sink Operator + keys: + 0 key (type: int) + 1 key (type: int) + + Stage: Stage-5 + Map Reduce + Map Operator Tree: + TableScan + alias: b + Filter Operator + predicate: key is not null (type: boolean) + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 key (type: int) + 1 key (type: int) + outputColumnNames: _col0, _col1, _col7 + Select Operator + expressions: _col0 (type: int), concat(_col1, _col7) (type: string) + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + value expressions: _col1 (type: string) + Local Work: + Map Reduce Local Work + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table3 + + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 key (type: int) + 1 key (type: int) + outputColumnNames: _col0, _col1, _col7 + Select Operator + expressions: _col0 (type: int), concat(_col1, _col7) (type: string) + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + value expressions: _col1 (type: string) + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table3 + PREHOOK: query: INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') SELECT a.key, concat(a.value, b.value) FROM test_table1 a JOIN test_table2 b @@ -274,11 +397,13 @@ STAGE PLANS: key expressions: _col0 (type: int) sort order: + Map-reduce partition columns: _col0 (type: int) - value expressions: _col0 (type: int), _col1 (type: string) + value expressions: _col1 (type: string) Local Work: Map Reduce Local Work Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 File Output Operator compressed: false table: @@ -340,11 +465,13 @@ STAGE PLANS: key expressions: _col0 (type: int) sort order: + Map-reduce partition columns: _col0 (type: int) - value expressions: _col0 (type: int), _col1 (type: string) + value expressions: _col1 (type: string) Local Work: Map Reduce Local Work Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 File Output Operator compressed: false table: @@ -376,9 +503,11 @@ STAGE PLANS: key expressions: _col0 (type: int) sort order: + Map-reduce partition columns: _col0 (type: int) - value expressions: _col0 (type: int), _col1 (type: string) + value expressions: _col1 (type: string) Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 File Output Operator compressed: false table: @@ -492,21 +621,44 @@ FROM test_table1 a JOIN test_table2 b ON a.key = b.key WHERE a.ds = '1' and b.ds is not null POSTHOOK: type: QUERY STAGE DEPENDENCIES: - Stage-1 is a root stage - Stage-0 depends on stages: Stage-1 + Stage-6 is a root stage , consists of Stage-7, Stage-8, Stage-1 + Stage-7 has a backup stage: Stage-1 + Stage-4 depends on stages: Stage-7 + Stage-0 depends on stages: Stage-1, Stage-4, Stage-5 Stage-2 depends on stages: Stage-0 + Stage-8 has a backup stage: Stage-1 + Stage-5 depends on stages: Stage-8 + Stage-1 STAGE PLANS: - Stage: Stage-1 + Stage: Stage-6 + Conditional Operator + + Stage: Stage-7 + Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + b + TableScan + alias: b + Filter Operator + predicate: key is not null (type: boolean) + HashTable Sink Operator + keys: + 0 key (type: int) + 1 key (type: int) + + Stage: Stage-4 Map Reduce Map Operator Tree: TableScan alias: a - Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE - Sorted Merge Bucket Map Join Operator + Map Join Operator condition map: Inner Join 0 to 1 keys: @@ -516,13 +668,24 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), concat(_col1, _col7) (type: string) outputColumnNames: _col0, _col1 - File Output Operator - compressed: false - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.test_table3 + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + value expressions: _col1 (type: string) + Local Work: + Map Reduce Local Work + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table3 Stage: Stage-0 Move Operator @@ -539,6 +702,95 @@ STAGE PLANS: Stage: Stage-2 Stats-Aggr Operator + Stage: Stage-8 + Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a + TableScan + alias: a + Filter Operator + predicate: key is not null (type: boolean) + HashTable Sink Operator + keys: + 0 key (type: int) + 1 key (type: int) + + Stage: Stage-5 + Map Reduce + Map Operator Tree: + TableScan + alias: b + Filter Operator + predicate: key is not null (type: boolean) + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 key (type: int) + 1 key (type: int) + outputColumnNames: _col0, _col1, _col7 + Select Operator + expressions: _col0 (type: int), concat(_col1, _col7) (type: string) + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + value expressions: _col1 (type: string) + Local Work: + Map Reduce Local Work + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table3 + + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 key (type: int) + 1 key (type: int) + outputColumnNames: _col0, _col1, _col7 + Select Operator + expressions: _col0 (type: int), concat(_col1, _col7) (type: string) + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + value expressions: _col1 (type: string) + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table3 + PREHOOK: query: INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') SELECT a.key, concat(a.value, b.value) FROM test_table1 a JOIN test_table2 b @@ -648,25 +900,50 @@ JOIN ON a.key = b.key POSTHOOK: type: QUERY STAGE DEPENDENCIES: - Stage-1 is a root stage - Stage-0 depends on stages: Stage-1 + Stage-6 is a root stage , consists of Stage-7, Stage-8, Stage-1 + Stage-7 has a backup stage: Stage-1 + Stage-4 depends on stages: Stage-7 + Stage-0 depends on stages: Stage-1, Stage-4, Stage-5 Stage-2 depends on stages: Stage-0 + Stage-8 has a backup stage: Stage-1 + Stage-5 depends on stages: Stage-8 + Stage-1 STAGE PLANS: - Stage: Stage-1 + Stage: Stage-6 + Conditional Operator + + Stage: Stage-7 + Map Reduce Local Work + Alias -> Map Local Tables: + b:test_table2 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + b:test_table2 + TableScan + alias: test_table2 + Filter Operator + predicate: key is not null (type: boolean) + Select Operator + expressions: key (type: int), value (type: string) + outputColumnNames: _col0, _col1 + HashTable Sink Operator + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + + Stage: Stage-4 Map Reduce Map Operator Tree: TableScan alias: test_table1 - Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int), value (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE - Sorted Merge Bucket Map Join Operator + Map Join Operator condition map: Inner Join 0 to 1 keys: @@ -676,13 +953,24 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), concat(_col1, _col3) (type: string) outputColumnNames: _col0, _col1 - File Output Operator - compressed: false - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.test_table3 + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + value expressions: _col1 (type: string) + Local Work: + Map Reduce Local Work + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table3 Stage: Stage-0 Move Operator @@ -699,6 +987,105 @@ STAGE PLANS: Stage: Stage-2 Stats-Aggr Operator + Stage: Stage-8 + Map Reduce Local Work + Alias -> Map Local Tables: + a:test_table1 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a:test_table1 + TableScan + alias: test_table1 + Filter Operator + predicate: key is not null (type: boolean) + Select Operator + expressions: key (type: int), value (type: string) + outputColumnNames: _col0, _col1 + HashTable Sink Operator + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + + Stage: Stage-5 + Map Reduce + Map Operator Tree: + TableScan + alias: test_table2 + Filter Operator + predicate: key is not null (type: boolean) + Select Operator + expressions: key (type: int), value (type: string) + outputColumnNames: _col0, _col1 + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col3 + Select Operator + expressions: _col0 (type: int), concat(_col1, _col3) (type: string) + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + value expressions: _col1 (type: string) + Local Work: + Map Reduce Local Work + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table3 + + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: test_table1 + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: int), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col3 + Select Operator + expressions: _col0 (type: int), concat(_col1, _col3) (type: string) + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + value expressions: _col1 (type: string) + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table3 + PREHOOK: query: INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') SELECT a.key, concat(a.value, b.value) FROM @@ -790,25 +1177,50 @@ JOIN ON a.key = b.key POSTHOOK: type: QUERY STAGE DEPENDENCIES: - Stage-1 is a root stage - Stage-0 depends on stages: Stage-1 + Stage-6 is a root stage , consists of Stage-7, Stage-8, Stage-1 + Stage-7 has a backup stage: Stage-1 + Stage-4 depends on stages: Stage-7 + Stage-0 depends on stages: Stage-1, Stage-4, Stage-5 Stage-2 depends on stages: Stage-0 + Stage-8 has a backup stage: Stage-1 + Stage-5 depends on stages: Stage-8 + Stage-1 STAGE PLANS: - Stage: Stage-1 + Stage: Stage-6 + Conditional Operator + + Stage: Stage-7 + Map Reduce Local Work + Alias -> Map Local Tables: + b:test_table2 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + b:test_table2 + TableScan + alias: test_table2 + Filter Operator + predicate: key is not null (type: boolean) + Select Operator + expressions: key (type: int), concat(value, value) (type: string) + outputColumnNames: _col0, _col1 + HashTable Sink Operator + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + + Stage: Stage-4 Map Reduce Map Operator Tree: TableScan alias: test_table1 - Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int), concat(value, value) (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE - Sorted Merge Bucket Map Join Operator + Map Join Operator condition map: Inner Join 0 to 1 keys: @@ -818,13 +1230,24 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), concat(_col1, _col3) (type: string) outputColumnNames: _col0, _col1 - File Output Operator - compressed: false - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.test_table3 + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + value expressions: _col1 (type: string) + Local Work: + Map Reduce Local Work + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table3 Stage: Stage-0 Move Operator @@ -841,6 +1264,105 @@ STAGE PLANS: Stage: Stage-2 Stats-Aggr Operator + Stage: Stage-8 + Map Reduce Local Work + Alias -> Map Local Tables: + a:test_table1 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a:test_table1 + TableScan + alias: test_table1 + Filter Operator + predicate: key is not null (type: boolean) + Select Operator + expressions: key (type: int), concat(value, value) (type: string) + outputColumnNames: _col0, _col1 + HashTable Sink Operator + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + + Stage: Stage-5 + Map Reduce + Map Operator Tree: + TableScan + alias: test_table2 + Filter Operator + predicate: key is not null (type: boolean) + Select Operator + expressions: key (type: int), concat(value, value) (type: string) + outputColumnNames: _col0, _col1 + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col3 + Select Operator + expressions: _col0 (type: int), concat(_col1, _col3) (type: string) + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + value expressions: _col1 (type: string) + Local Work: + Map Reduce Local Work + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table3 + + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: test_table1 + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: int), concat(value, value) (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col3 + Select Operator + expressions: _col0 (type: int), concat(_col1, _col3) (type: string) + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + value expressions: _col1 (type: string) + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table3 + PREHOOK: query: INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') SELECT a.key, concat(a.v1, b.v2) FROM @@ -989,11 +1511,13 @@ STAGE PLANS: key expressions: _col0 (type: int) sort order: + Map-reduce partition columns: _col0 (type: int) - value expressions: _col0 (type: int), _col1 (type: string) + value expressions: _col1 (type: string) Local Work: Map Reduce Local Work Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 File Output Operator compressed: false table: @@ -1061,11 +1585,13 @@ STAGE PLANS: key expressions: _col0 (type: int) sort order: + Map-reduce partition columns: _col0 (type: int) - value expressions: _col0 (type: int), _col1 (type: string) + value expressions: _col1 (type: string) Local Work: Map Reduce Local Work Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 File Output Operator compressed: false table: @@ -1101,9 +1627,11 @@ STAGE PLANS: key expressions: _col0 (type: int) sort order: + Map-reduce partition columns: _col0 (type: int) - value expressions: _col0 (type: int), _col1 (type: string) + value expressions: _col1 (type: string) Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 File Output Operator compressed: false table: diff --git a/ql/src/test/results/clientpositive/bucketsortoptimize_insert_3.q.out b/ql/src/test/results/clientpositive/bucketsortoptimize_insert_3.q.out index 7456ab0..f4d52a9 100644 --- a/ql/src/test/results/clientpositive/bucketsortoptimize_insert_3.q.out +++ b/ql/src/test/results/clientpositive/bucketsortoptimize_insert_3.q.out @@ -64,14 +64,25 @@ STAGE PLANS: expressions: value (type: string), key (type: int) outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false + Reduce Output Operator + key expressions: _col1 (type: int) + sort order: + + Map-reduce partition columns: _col1 (type: int) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.test_table2 + value expressions: _col0 (type: string) + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: string), KEY.reducesinkkey0 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table2 Stage: Stage-0 Move Operator @@ -184,20 +195,18 @@ STAGE PLANS: Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: string) Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), UDFToInteger(VALUE._col1) (type: int) + outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: _col0 (type: int), UDFToInteger(_col1) (type: int) - outputColumnNames: _col0, _col1 + File Output Operator + compressed: false Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.test_table2 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table2 Stage: Stage-0 Move Operator diff --git a/ql/src/test/results/clientpositive/bucketsortoptimize_insert_4.q.out b/ql/src/test/results/clientpositive/bucketsortoptimize_insert_4.q.out index fd99597..1a644a9 100644 --- a/ql/src/test/results/clientpositive/bucketsortoptimize_insert_4.q.out +++ b/ql/src/test/results/clientpositive/bucketsortoptimize_insert_4.q.out @@ -71,21 +71,44 @@ FROM test_table1 a JOIN test_table2 b ON a.key = b.key WHERE a.ds = '1' and b.ds = '1' POSTHOOK: type: QUERY STAGE DEPENDENCIES: - Stage-1 is a root stage - Stage-0 depends on stages: Stage-1 + Stage-6 is a root stage , consists of Stage-7, Stage-8, Stage-1 + Stage-7 has a backup stage: Stage-1 + Stage-4 depends on stages: Stage-7 + Stage-0 depends on stages: Stage-1, Stage-4, Stage-5 Stage-2 depends on stages: Stage-0 + Stage-8 has a backup stage: Stage-1 + Stage-5 depends on stages: Stage-8 + Stage-1 STAGE PLANS: - Stage: Stage-1 + Stage: Stage-6 + Conditional Operator + + Stage: Stage-7 + Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + b + TableScan + alias: b + Filter Operator + predicate: key is not null (type: boolean) + HashTable Sink Operator + keys: + 0 key (type: int) + 1 key (type: int) + + Stage: Stage-4 Map Reduce Map Operator Tree: TableScan alias: a - Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE - Sorted Merge Bucket Map Join Operator + Map Join Operator condition map: Inner Join 0 to 1 keys: @@ -93,15 +116,26 @@ STAGE PLANS: 1 key (type: int) outputColumnNames: _col0, _col1, _col7 Select Operator - expressions: _col0 (type: int), _col0 (type: int), concat(_col1, _col7) (type: string) - outputColumnNames: _col0, _col1, _col2 - File Output Operator - compressed: false - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.test_table3 + expressions: _col0 (type: int), concat(_col1, _col7) (type: string) + outputColumnNames: _col1, _col2 + Reduce Output Operator + key expressions: _col1 (type: int) + sort order: + + Map-reduce partition columns: _col1 (type: int) + value expressions: _col2 (type: string) + Local Work: + Map Reduce Local Work + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table3 Stage: Stage-0 Move Operator @@ -118,6 +152,95 @@ STAGE PLANS: Stage: Stage-2 Stats-Aggr Operator + Stage: Stage-8 + Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a + TableScan + alias: a + Filter Operator + predicate: key is not null (type: boolean) + HashTable Sink Operator + keys: + 0 key (type: int) + 1 key (type: int) + + Stage: Stage-5 + Map Reduce + Map Operator Tree: + TableScan + alias: b + Filter Operator + predicate: key is not null (type: boolean) + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 key (type: int) + 1 key (type: int) + outputColumnNames: _col0, _col1, _col7 + Select Operator + expressions: _col0 (type: int), concat(_col1, _col7) (type: string) + outputColumnNames: _col1, _col2 + Reduce Output Operator + key expressions: _col1 (type: int) + sort order: + + Map-reduce partition columns: _col1 (type: int) + value expressions: _col2 (type: string) + Local Work: + Map Reduce Local Work + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table3 + + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 key (type: int) + 1 key (type: int) + outputColumnNames: _col0, _col1, _col7 + Select Operator + expressions: _col0 (type: int), concat(_col1, _col7) (type: string) + outputColumnNames: _col1, _col2 + Reduce Output Operator + key expressions: _col1 (type: int) + sort order: + + Map-reduce partition columns: _col1 (type: int) + value expressions: _col2 (type: string) + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table3 + PREHOOK: query: INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') SELECT a.key, a.key, concat(a.value, b.value) FROM test_table1 a JOIN test_table2 b @@ -266,11 +389,13 @@ STAGE PLANS: key expressions: _col1 (type: string) sort order: + Map-reduce partition columns: _col1 (type: string) - value expressions: _col0 (type: int), _col1 (type: string) + value expressions: _col0 (type: int) Local Work: Map Reduce Local Work Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey0 (type: string) + outputColumnNames: _col0, _col1 File Output Operator compressed: false table: @@ -329,11 +454,13 @@ STAGE PLANS: key expressions: _col1 (type: string) sort order: + Map-reduce partition columns: _col1 (type: string) - value expressions: _col0 (type: int), _col1 (type: string) + value expressions: _col0 (type: int) Local Work: Map Reduce Local Work Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey0 (type: string) + outputColumnNames: _col0, _col1 File Output Operator compressed: false table: @@ -362,9 +489,11 @@ STAGE PLANS: key expressions: _col1 (type: string) sort order: + Map-reduce partition columns: _col1 (type: string) - value expressions: _col0 (type: int), _col1 (type: string) + value expressions: _col0 (type: int) Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey0 (type: string) + outputColumnNames: _col0, _col1 File Output Operator compressed: false table: diff --git a/ql/src/test/results/clientpositive/bucketsortoptimize_insert_5.q.out b/ql/src/test/results/clientpositive/bucketsortoptimize_insert_5.q.out index 8130ab9..e4f90e4 100644 --- a/ql/src/test/results/clientpositive/bucketsortoptimize_insert_5.q.out +++ b/ql/src/test/results/clientpositive/bucketsortoptimize_insert_5.q.out @@ -122,11 +122,13 @@ STAGE PLANS: key expressions: _col0 (type: int) sort order: - Map-reduce partition columns: _col0 (type: int) - value expressions: _col0 (type: int), _col1 (type: string) + value expressions: _col1 (type: string) Local Work: Map Reduce Local Work Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 File Output Operator compressed: false table: @@ -188,11 +190,13 @@ STAGE PLANS: key expressions: _col0 (type: int) sort order: - Map-reduce partition columns: _col0 (type: int) - value expressions: _col0 (type: int), _col1 (type: string) + value expressions: _col1 (type: string) Local Work: Map Reduce Local Work Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 File Output Operator compressed: false table: @@ -224,9 +228,11 @@ STAGE PLANS: key expressions: _col0 (type: int) sort order: - Map-reduce partition columns: _col0 (type: int) - value expressions: _col0 (type: int), _col1 (type: string) + value expressions: _col1 (type: string) Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 File Output Operator compressed: false table: @@ -377,11 +383,13 @@ STAGE PLANS: key expressions: _col0 (type: int) sort order: - Map-reduce partition columns: _col0 (type: int) - value expressions: _col0 (type: int), _col1 (type: string) + value expressions: _col1 (type: string) Local Work: Map Reduce Local Work Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 File Output Operator compressed: false table: @@ -449,11 +457,13 @@ STAGE PLANS: key expressions: _col0 (type: int) sort order: - Map-reduce partition columns: _col0 (type: int) - value expressions: _col0 (type: int), _col1 (type: string) + value expressions: _col1 (type: string) Local Work: Map Reduce Local Work Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 File Output Operator compressed: false table: @@ -489,9 +499,11 @@ STAGE PLANS: key expressions: _col0 (type: int) sort order: - Map-reduce partition columns: _col0 (type: int) - value expressions: _col0 (type: int), _col1 (type: string) + value expressions: _col1 (type: string) Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 File Output Operator compressed: false table: diff --git a/ql/src/test/results/clientpositive/bucketsortoptimize_insert_6.q.out b/ql/src/test/results/clientpositive/bucketsortoptimize_insert_6.q.out index 627aba0..a6d1940 100644 --- a/ql/src/test/results/clientpositive/bucketsortoptimize_insert_6.q.out +++ b/ql/src/test/results/clientpositive/bucketsortoptimize_insert_6.q.out @@ -73,21 +73,44 @@ FROM test_table1 a JOIN test_table2 b ON a.key = b.key and a.key2 = b.key2 WHERE a.ds = '1' and b.ds = '1' POSTHOOK: type: QUERY STAGE DEPENDENCIES: - Stage-1 is a root stage - Stage-0 depends on stages: Stage-1 + Stage-6 is a root stage , consists of Stage-7, Stage-8, Stage-1 + Stage-7 has a backup stage: Stage-1 + Stage-4 depends on stages: Stage-7 + Stage-0 depends on stages: Stage-1, Stage-4, Stage-5 Stage-2 depends on stages: Stage-0 + Stage-8 has a backup stage: Stage-1 + Stage-5 depends on stages: Stage-8 + Stage-1 STAGE PLANS: - Stage: Stage-1 + Stage: Stage-6 + Conditional Operator + + Stage: Stage-7 + Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + b + TableScan + alias: b + Filter Operator + predicate: (key is not null and key2 is not null) (type: boolean) + HashTable Sink Operator + keys: + 0 key (type: int), key2 (type: int) + 1 key (type: int), key2 (type: int) + + Stage: Stage-4 Map Reduce Map Operator Tree: TableScan alias: a - Statistics: Num rows: 10 Data size: 91 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (key is not null and key2 is not null) (type: boolean) - Statistics: Num rows: 3 Data size: 27 Basic stats: COMPLETE Column stats: NONE - Sorted Merge Bucket Map Join Operator + Map Join Operator condition map: Inner Join 0 to 1 keys: @@ -97,13 +120,24 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), _col1 (type: int), concat(_col2, _col9) (type: string) outputColumnNames: _col0, _col1, _col2 - File Output Operator - compressed: false - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.test_table3 + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: +- + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + value expressions: _col2 (type: string) + Local Work: + Map Reduce Local Work + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table3 Stage: Stage-0 Move Operator @@ -120,6 +154,95 @@ STAGE PLANS: Stage: Stage-2 Stats-Aggr Operator + Stage: Stage-8 + Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a + TableScan + alias: a + Filter Operator + predicate: (key is not null and key2 is not null) (type: boolean) + HashTable Sink Operator + keys: + 0 key (type: int), key2 (type: int) + 1 key (type: int), key2 (type: int) + + Stage: Stage-5 + Map Reduce + Map Operator Tree: + TableScan + alias: b + Filter Operator + predicate: (key is not null and key2 is not null) (type: boolean) + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 key (type: int), key2 (type: int) + 1 key (type: int), key2 (type: int) + outputColumnNames: _col0, _col1, _col2, _col9 + Select Operator + expressions: _col0 (type: int), _col1 (type: int), concat(_col2, _col9) (type: string) + outputColumnNames: _col0, _col1, _col2 + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: +- + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + value expressions: _col2 (type: string) + Local Work: + Map Reduce Local Work + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table3 + + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 10 Data size: 91 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (key is not null and key2 is not null) (type: boolean) + Statistics: Num rows: 3 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 key (type: int), key2 (type: int) + 1 key (type: int), key2 (type: int) + outputColumnNames: _col0, _col1, _col2, _col9 + Select Operator + expressions: _col0 (type: int), _col1 (type: int), concat(_col2, _col9) (type: string) + outputColumnNames: _col0, _col1, _col2 + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: +- + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + value expressions: _col2 (type: string) + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table3 + PREHOOK: query: INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') SELECT a.key, a.key2, concat(a.value, b.value) FROM test_table1 a JOIN test_table2 b @@ -208,21 +331,44 @@ ON a.key = b.key and a.key2 = b.key2 WHERE a.ds = '1' and b.ds = '1' )subq1 POSTHOOK: type: QUERY STAGE DEPENDENCIES: - Stage-1 is a root stage - Stage-0 depends on stages: Stage-1 + Stage-6 is a root stage , consists of Stage-7, Stage-8, Stage-1 + Stage-7 has a backup stage: Stage-1 + Stage-4 depends on stages: Stage-7 + Stage-0 depends on stages: Stage-1, Stage-4, Stage-5 Stage-2 depends on stages: Stage-0 + Stage-8 has a backup stage: Stage-1 + Stage-5 depends on stages: Stage-8 + Stage-1 STAGE PLANS: - Stage: Stage-1 + Stage: Stage-6 + Conditional Operator + + Stage: Stage-7 + Map Reduce Local Work + Alias -> Map Local Tables: + subq1:b + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + subq1:b + TableScan + alias: b + Filter Operator + predicate: (key is not null and key2 is not null) (type: boolean) + HashTable Sink Operator + keys: + 0 key (type: int), key2 (type: int) + 1 key (type: int), key2 (type: int) + + Stage: Stage-4 Map Reduce Map Operator Tree: TableScan alias: a - Statistics: Num rows: 10 Data size: 91 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (key is not null and key2 is not null) (type: boolean) - Statistics: Num rows: 3 Data size: 27 Basic stats: COMPLETE Column stats: NONE - Sorted Merge Bucket Map Join Operator + Map Join Operator condition map: Inner Join 0 to 1 keys: @@ -232,13 +378,24 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), _col1 (type: int), concat(_col2, _col9) (type: string) outputColumnNames: _col0, _col1, _col2 - File Output Operator - compressed: false - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.test_table3 + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: +- + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + value expressions: _col2 (type: string) + Local Work: + Map Reduce Local Work + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table3 Stage: Stage-0 Move Operator @@ -255,6 +412,95 @@ STAGE PLANS: Stage: Stage-2 Stats-Aggr Operator + Stage: Stage-8 + Map Reduce Local Work + Alias -> Map Local Tables: + subq1:a + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + subq1:a + TableScan + alias: a + Filter Operator + predicate: (key is not null and key2 is not null) (type: boolean) + HashTable Sink Operator + keys: + 0 key (type: int), key2 (type: int) + 1 key (type: int), key2 (type: int) + + Stage: Stage-5 + Map Reduce + Map Operator Tree: + TableScan + alias: b + Filter Operator + predicate: (key is not null and key2 is not null) (type: boolean) + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 key (type: int), key2 (type: int) + 1 key (type: int), key2 (type: int) + outputColumnNames: _col0, _col1, _col2, _col9 + Select Operator + expressions: _col0 (type: int), _col1 (type: int), concat(_col2, _col9) (type: string) + outputColumnNames: _col0, _col1, _col2 + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: +- + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + value expressions: _col2 (type: string) + Local Work: + Map Reduce Local Work + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table3 + + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 10 Data size: 91 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (key is not null and key2 is not null) (type: boolean) + Statistics: Num rows: 3 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 key (type: int), key2 (type: int) + 1 key (type: int), key2 (type: int) + outputColumnNames: _col0, _col1, _col2, _col9 + Select Operator + expressions: _col0 (type: int), _col1 (type: int), concat(_col2, _col9) (type: string) + outputColumnNames: _col0, _col1, _col2 + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: +- + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + value expressions: _col2 (type: string) + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table3 + PREHOOK: query: INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') SELECT subq1.key, subq1.key2, subq1.value from ( @@ -394,11 +640,13 @@ STAGE PLANS: key expressions: _col0 (type: int), _col1 (type: int) sort order: +- Map-reduce partition columns: _col0 (type: int), _col1 (type: int) - value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string) + value expressions: _col2 (type: string) Local Work: Map Reduce Local Work Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1, _col2 File Output Operator compressed: false table: @@ -460,11 +708,13 @@ STAGE PLANS: key expressions: _col0 (type: int), _col1 (type: int) sort order: +- Map-reduce partition columns: _col0 (type: int), _col1 (type: int) - value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string) + value expressions: _col2 (type: string) Local Work: Map Reduce Local Work Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1, _col2 File Output Operator compressed: false table: @@ -496,9 +746,11 @@ STAGE PLANS: key expressions: _col0 (type: int), _col1 (type: int) sort order: +- Map-reduce partition columns: _col0 (type: int), _col1 (type: int) - value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string) + value expressions: _col2 (type: string) Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1, _col2 File Output Operator compressed: false table: @@ -581,11 +833,13 @@ STAGE PLANS: key expressions: _col0 (type: int), _col1 (type: int) sort order: +- Map-reduce partition columns: _col0 (type: int), _col1 (type: int) - value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string) + value expressions: _col2 (type: string) Local Work: Map Reduce Local Work Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1, _col2 File Output Operator compressed: false table: @@ -647,11 +901,13 @@ STAGE PLANS: key expressions: _col0 (type: int), _col1 (type: int) sort order: +- Map-reduce partition columns: _col0 (type: int), _col1 (type: int) - value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string) + value expressions: _col2 (type: string) Local Work: Map Reduce Local Work Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1, _col2 File Output Operator compressed: false table: @@ -683,9 +939,11 @@ STAGE PLANS: key expressions: _col0 (type: int), _col1 (type: int) sort order: +- Map-reduce partition columns: _col0 (type: int), _col1 (type: int) - value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string) + value expressions: _col2 (type: string) Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1, _col2 File Output Operator compressed: false table: @@ -723,21 +981,44 @@ ON a.key = b.key and a.key2 = b.key2 WHERE a.ds = '1' and b.ds = '1' )subq2 POSTHOOK: type: QUERY STAGE DEPENDENCIES: - Stage-1 is a root stage - Stage-0 depends on stages: Stage-1 + Stage-6 is a root stage , consists of Stage-7, Stage-8, Stage-1 + Stage-7 has a backup stage: Stage-1 + Stage-4 depends on stages: Stage-7 + Stage-0 depends on stages: Stage-1, Stage-4, Stage-5 Stage-2 depends on stages: Stage-0 + Stage-8 has a backup stage: Stage-1 + Stage-5 depends on stages: Stage-8 + Stage-1 STAGE PLANS: - Stage: Stage-1 + Stage: Stage-6 + Conditional Operator + + Stage: Stage-7 + Map Reduce Local Work + Alias -> Map Local Tables: + subq2:subq1:b + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + subq2:subq1:b + TableScan + alias: b + Filter Operator + predicate: (key is not null and key2 is not null) (type: boolean) + HashTable Sink Operator + keys: + 0 key (type: int), key2 (type: int) + 1 key (type: int), key2 (type: int) + + Stage: Stage-4 Map Reduce Map Operator Tree: TableScan alias: a - Statistics: Num rows: 10 Data size: 91 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (key is not null and key2 is not null) (type: boolean) - Statistics: Num rows: 3 Data size: 27 Basic stats: COMPLETE Column stats: NONE - Sorted Merge Bucket Map Join Operator + Map Join Operator condition map: Inner Join 0 to 1 keys: @@ -747,13 +1028,24 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), _col1 (type: int), concat(_col2, _col9) (type: string) outputColumnNames: _col0, _col1, _col2 - File Output Operator - compressed: false - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.test_table3 + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: +- + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + value expressions: _col2 (type: string) + Local Work: + Map Reduce Local Work + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table3 Stage: Stage-0 Move Operator @@ -770,6 +1062,95 @@ STAGE PLANS: Stage: Stage-2 Stats-Aggr Operator + Stage: Stage-8 + Map Reduce Local Work + Alias -> Map Local Tables: + subq2:subq1:a + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + subq2:subq1:a + TableScan + alias: a + Filter Operator + predicate: (key is not null and key2 is not null) (type: boolean) + HashTable Sink Operator + keys: + 0 key (type: int), key2 (type: int) + 1 key (type: int), key2 (type: int) + + Stage: Stage-5 + Map Reduce + Map Operator Tree: + TableScan + alias: b + Filter Operator + predicate: (key is not null and key2 is not null) (type: boolean) + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 key (type: int), key2 (type: int) + 1 key (type: int), key2 (type: int) + outputColumnNames: _col0, _col1, _col2, _col9 + Select Operator + expressions: _col0 (type: int), _col1 (type: int), concat(_col2, _col9) (type: string) + outputColumnNames: _col0, _col1, _col2 + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: +- + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + value expressions: _col2 (type: string) + Local Work: + Map Reduce Local Work + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table3 + + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 10 Data size: 91 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (key is not null and key2 is not null) (type: boolean) + Statistics: Num rows: 3 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 key (type: int), key2 (type: int) + 1 key (type: int), key2 (type: int) + outputColumnNames: _col0, _col1, _col2, _col9 + Select Operator + expressions: _col0 (type: int), _col1 (type: int), concat(_col2, _col9) (type: string) + outputColumnNames: _col0, _col1, _col2 + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: +- + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + value expressions: _col2 (type: string) + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table3 + PREHOOK: query: INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') SELECT subq2.key, subq2.key2, subq2.value from ( @@ -876,21 +1257,44 @@ ON a.key = b.key and a.key2 = b.key2 WHERE a.ds = '1' and b.ds = '1' )subq2 POSTHOOK: type: QUERY STAGE DEPENDENCIES: - Stage-1 is a root stage - Stage-0 depends on stages: Stage-1 + Stage-6 is a root stage , consists of Stage-7, Stage-8, Stage-1 + Stage-7 has a backup stage: Stage-1 + Stage-4 depends on stages: Stage-7 + Stage-0 depends on stages: Stage-1, Stage-4, Stage-5 Stage-2 depends on stages: Stage-0 + Stage-8 has a backup stage: Stage-1 + Stage-5 depends on stages: Stage-8 + Stage-1 STAGE PLANS: - Stage: Stage-1 + Stage: Stage-6 + Conditional Operator + + Stage: Stage-7 + Map Reduce Local Work + Alias -> Map Local Tables: + subq2:subq1:b + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + subq2:subq1:b + TableScan + alias: b + Filter Operator + predicate: (key is not null and key2 is not null) (type: boolean) + HashTable Sink Operator + keys: + 0 key (type: int), key2 (type: int) + 1 key (type: int), key2 (type: int) + + Stage: Stage-4 Map Reduce Map Operator Tree: TableScan alias: a - Statistics: Num rows: 10 Data size: 91 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (key is not null and key2 is not null) (type: boolean) - Statistics: Num rows: 3 Data size: 27 Basic stats: COMPLETE Column stats: NONE - Sorted Merge Bucket Map Join Operator + Map Join Operator condition map: Inner Join 0 to 1 keys: @@ -900,13 +1304,24 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), _col1 (type: int), concat(_col2, _col9) (type: string) outputColumnNames: _col0, _col1, _col2 - File Output Operator - compressed: false - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.test_table3 + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: +- + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + value expressions: _col2 (type: string) + Local Work: + Map Reduce Local Work + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table3 Stage: Stage-0 Move Operator @@ -923,6 +1338,95 @@ STAGE PLANS: Stage: Stage-2 Stats-Aggr Operator + Stage: Stage-8 + Map Reduce Local Work + Alias -> Map Local Tables: + subq2:subq1:a + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + subq2:subq1:a + TableScan + alias: a + Filter Operator + predicate: (key is not null and key2 is not null) (type: boolean) + HashTable Sink Operator + keys: + 0 key (type: int), key2 (type: int) + 1 key (type: int), key2 (type: int) + + Stage: Stage-5 + Map Reduce + Map Operator Tree: + TableScan + alias: b + Filter Operator + predicate: (key is not null and key2 is not null) (type: boolean) + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 key (type: int), key2 (type: int) + 1 key (type: int), key2 (type: int) + outputColumnNames: _col0, _col1, _col2, _col9 + Select Operator + expressions: _col0 (type: int), _col1 (type: int), concat(_col2, _col9) (type: string) + outputColumnNames: _col0, _col1, _col2 + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: +- + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + value expressions: _col2 (type: string) + Local Work: + Map Reduce Local Work + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table3 + + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 10 Data size: 91 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (key is not null and key2 is not null) (type: boolean) + Statistics: Num rows: 3 Data size: 27 Basic stats: COMPLETE Column stats: NONE + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 key (type: int), key2 (type: int) + 1 key (type: int), key2 (type: int) + outputColumnNames: _col0, _col1, _col2, _col9 + Select Operator + expressions: _col0 (type: int), _col1 (type: int), concat(_col2, _col9) (type: string) + outputColumnNames: _col0, _col1, _col2 + Reduce Output Operator + key expressions: _col0 (type: int), _col1 (type: int) + sort order: +- + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) + value expressions: _col2 (type: string) + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table3 + PREHOOK: query: INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') SELECT subq2.k2, subq2.k1, subq2.value from ( @@ -1090,11 +1594,13 @@ STAGE PLANS: key expressions: _col0 (type: int), _col1 (type: int) sort order: -- Map-reduce partition columns: _col0 (type: int), _col1 (type: int) - value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string) + value expressions: _col2 (type: string) Local Work: Map Reduce Local Work Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1, _col2 File Output Operator compressed: false table: @@ -1156,11 +1662,13 @@ STAGE PLANS: key expressions: _col0 (type: int), _col1 (type: int) sort order: -- Map-reduce partition columns: _col0 (type: int), _col1 (type: int) - value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string) + value expressions: _col2 (type: string) Local Work: Map Reduce Local Work Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1, _col2 File Output Operator compressed: false table: @@ -1192,9 +1700,11 @@ STAGE PLANS: key expressions: _col0 (type: int), _col1 (type: int) sort order: -- Map-reduce partition columns: _col0 (type: int), _col1 (type: int) - value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string) + value expressions: _col2 (type: string) Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1, _col2 File Output Operator compressed: false table: diff --git a/ql/src/test/results/clientpositive/bucketsortoptimize_insert_7.q.out b/ql/src/test/results/clientpositive/bucketsortoptimize_insert_7.q.out index 2b8f2ba..f63fe32 100644 --- a/ql/src/test/results/clientpositive/bucketsortoptimize_insert_7.q.out +++ b/ql/src/test/results/clientpositive/bucketsortoptimize_insert_7.q.out @@ -73,21 +73,44 @@ ON a.key = b.key WHERE a.ds = '1' and b.ds = '1' and (a.key = 0 or a.key = 5) POSTHOOK: type: QUERY STAGE DEPENDENCIES: - Stage-1 is a root stage - Stage-0 depends on stages: Stage-1 + Stage-6 is a root stage , consists of Stage-7, Stage-8, Stage-1 + Stage-7 has a backup stage: Stage-1 + Stage-4 depends on stages: Stage-7 + Stage-0 depends on stages: Stage-1, Stage-4, Stage-5 Stage-2 depends on stages: Stage-0 + Stage-8 has a backup stage: Stage-1 + Stage-5 depends on stages: Stage-8 + Stage-1 STAGE PLANS: - Stage: Stage-1 + Stage: Stage-6 + Conditional Operator + + Stage: Stage-7 + Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + b + TableScan + alias: b + Filter Operator + predicate: (key is not null and ((key = 0) or (key = 5))) (type: boolean) + HashTable Sink Operator + keys: + 0 key (type: int) + 1 key (type: int) + + Stage: Stage-4 Map Reduce Map Operator Tree: TableScan alias: a - Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (key is not null and ((key = 0) or (key = 5))) (type: boolean) - Statistics: Num rows: 4 Data size: 28 Basic stats: COMPLETE Column stats: NONE - Sorted Merge Bucket Map Join Operator + Map Join Operator condition map: Inner Join 0 to 1 keys: @@ -97,13 +120,24 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), concat(_col1, _col7) (type: string) outputColumnNames: _col0, _col1 - File Output Operator - compressed: false - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.test_table3 + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + value expressions: _col1 (type: string) + Local Work: + Map Reduce Local Work + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table3 Stage: Stage-0 Move Operator @@ -120,6 +154,95 @@ STAGE PLANS: Stage: Stage-2 Stats-Aggr Operator + Stage: Stage-8 + Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a + TableScan + alias: a + Filter Operator + predicate: (key is not null and ((key = 0) or (key = 5))) (type: boolean) + HashTable Sink Operator + keys: + 0 key (type: int) + 1 key (type: int) + + Stage: Stage-5 + Map Reduce + Map Operator Tree: + TableScan + alias: b + Filter Operator + predicate: (key is not null and ((key = 0) or (key = 5))) (type: boolean) + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 key (type: int) + 1 key (type: int) + outputColumnNames: _col0, _col1, _col7 + Select Operator + expressions: _col0 (type: int), concat(_col1, _col7) (type: string) + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + value expressions: _col1 (type: string) + Local Work: + Map Reduce Local Work + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table3 + + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (key is not null and ((key = 0) or (key = 5))) (type: boolean) + Statistics: Num rows: 4 Data size: 28 Basic stats: COMPLETE Column stats: NONE + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 key (type: int) + 1 key (type: int) + outputColumnNames: _col0, _col1, _col7 + Select Operator + expressions: _col0 (type: int), concat(_col1, _col7) (type: string) + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + value expressions: _col1 (type: string) + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table3 + PREHOOK: query: INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') SELECT a.key, concat(a.value, b.value) FROM test_table1 a JOIN test_table2 b @@ -203,25 +326,50 @@ JOIN ON a.key = b.key POSTHOOK: type: QUERY STAGE DEPENDENCIES: - Stage-1 is a root stage - Stage-0 depends on stages: Stage-1 + Stage-6 is a root stage , consists of Stage-7, Stage-8, Stage-1 + Stage-7 has a backup stage: Stage-1 + Stage-4 depends on stages: Stage-7 + Stage-0 depends on stages: Stage-1, Stage-4, Stage-5 Stage-2 depends on stages: Stage-0 + Stage-8 has a backup stage: Stage-1 + Stage-5 depends on stages: Stage-8 + Stage-1 STAGE PLANS: - Stage: Stage-1 + Stage: Stage-6 + Conditional Operator + + Stage: Stage-7 + Map Reduce Local Work + Alias -> Map Local Tables: + b:test_table2 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + b:test_table2 + TableScan + alias: test_table2 + Filter Operator + predicate: (((key = 0) or (key = 5)) and key is not null) (type: boolean) + Select Operator + expressions: key (type: int), value (type: string) + outputColumnNames: _col0, _col1 + HashTable Sink Operator + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + + Stage: Stage-4 Map Reduce Map Operator Tree: TableScan alias: test_table1 - Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (((key = 0) or (key = 5)) and key is not null) (type: boolean) - Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int), value (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE - Sorted Merge Bucket Map Join Operator + Map Join Operator condition map: Inner Join 0 to 1 keys: @@ -231,13 +379,24 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), concat(_col1, _col3) (type: string) outputColumnNames: _col0, _col1 - File Output Operator - compressed: false - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.test_table3 + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + value expressions: _col1 (type: string) + Local Work: + Map Reduce Local Work + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table3 Stage: Stage-0 Move Operator @@ -254,6 +413,105 @@ STAGE PLANS: Stage: Stage-2 Stats-Aggr Operator + Stage: Stage-8 + Map Reduce Local Work + Alias -> Map Local Tables: + a:test_table1 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a:test_table1 + TableScan + alias: test_table1 + Filter Operator + predicate: (((key = 0) or (key = 5)) and key is not null) (type: boolean) + Select Operator + expressions: key (type: int), value (type: string) + outputColumnNames: _col0, _col1 + HashTable Sink Operator + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + + Stage: Stage-5 + Map Reduce + Map Operator Tree: + TableScan + alias: test_table2 + Filter Operator + predicate: (((key = 0) or (key = 5)) and key is not null) (type: boolean) + Select Operator + expressions: key (type: int), value (type: string) + outputColumnNames: _col0, _col1 + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col3 + Select Operator + expressions: _col0 (type: int), concat(_col1, _col3) (type: string) + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + value expressions: _col1 (type: string) + Local Work: + Map Reduce Local Work + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table3 + + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: test_table1 + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (((key = 0) or (key = 5)) and key is not null) (type: boolean) + Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: int), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col3 + Select Operator + expressions: _col0 (type: int), concat(_col1, _col3) (type: string) + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + value expressions: _col1 (type: string) + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table3 + PREHOOK: query: INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') SELECT a.key, concat(a.value, b.value) FROM @@ -343,25 +601,50 @@ ON a.key = b.key WHERE a.key = 0 or a.key = 5 POSTHOOK: type: QUERY STAGE DEPENDENCIES: - Stage-1 is a root stage - Stage-0 depends on stages: Stage-1 + Stage-6 is a root stage , consists of Stage-7, Stage-8, Stage-1 + Stage-7 has a backup stage: Stage-1 + Stage-4 depends on stages: Stage-7 + Stage-0 depends on stages: Stage-1, Stage-4, Stage-5 Stage-2 depends on stages: Stage-0 + Stage-8 has a backup stage: Stage-1 + Stage-5 depends on stages: Stage-8 + Stage-1 STAGE PLANS: - Stage: Stage-1 + Stage: Stage-6 + Conditional Operator + + Stage: Stage-7 + Map Reduce Local Work + Alias -> Map Local Tables: + b:test_table2 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + b:test_table2 + TableScan + alias: test_table2 + Filter Operator + predicate: (((key < 8) and key is not null) and ((key = 0) or (key = 5))) (type: boolean) + Select Operator + expressions: key (type: int), value (type: string) + outputColumnNames: _col0, _col1 + HashTable Sink Operator + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + + Stage: Stage-4 Map Reduce Map Operator Tree: TableScan alias: test_table1 - Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (((key < 8) and key is not null) and ((key = 0) or (key = 5))) (type: boolean) - Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: int), value (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE - Sorted Merge Bucket Map Join Operator + Map Join Operator condition map: Inner Join 0 to 1 keys: @@ -371,13 +654,24 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), concat(_col1, _col3) (type: string) outputColumnNames: _col0, _col1 - File Output Operator - compressed: false - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.test_table3 + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + value expressions: _col1 (type: string) + Local Work: + Map Reduce Local Work + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table3 Stage: Stage-0 Move Operator @@ -394,6 +688,105 @@ STAGE PLANS: Stage: Stage-2 Stats-Aggr Operator + Stage: Stage-8 + Map Reduce Local Work + Alias -> Map Local Tables: + a:test_table1 + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a:test_table1 + TableScan + alias: test_table1 + Filter Operator + predicate: (((key < 8) and key is not null) and ((key = 0) or (key = 5))) (type: boolean) + Select Operator + expressions: key (type: int), value (type: string) + outputColumnNames: _col0, _col1 + HashTable Sink Operator + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + + Stage: Stage-5 + Map Reduce + Map Operator Tree: + TableScan + alias: test_table2 + Filter Operator + predicate: (((key < 8) and key is not null) and ((key = 0) or (key = 5))) (type: boolean) + Select Operator + expressions: key (type: int), value (type: string) + outputColumnNames: _col0, _col1 + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col3 + Select Operator + expressions: _col0 (type: int), concat(_col1, _col3) (type: string) + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + value expressions: _col1 (type: string) + Local Work: + Map Reduce Local Work + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table3 + + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: test_table1 + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (((key < 8) and key is not null) and ((key = 0) or (key = 5))) (type: boolean) + Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: int), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 2 Data size: 14 Basic stats: COMPLETE Column stats: NONE + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col0 (type: int) + 1 _col0 (type: int) + outputColumnNames: _col0, _col1, _col3 + Select Operator + expressions: _col0 (type: int), concat(_col1, _col3) (type: string) + outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + value expressions: _col1 (type: string) + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table3 + PREHOOK: query: INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') SELECT a.key, concat(a.value, b.value) FROM diff --git a/ql/src/test/results/clientpositive/bucketsortoptimize_insert_8.q.out b/ql/src/test/results/clientpositive/bucketsortoptimize_insert_8.q.out index 2b2e7bf..881dc56 100644 --- a/ql/src/test/results/clientpositive/bucketsortoptimize_insert_8.q.out +++ b/ql/src/test/results/clientpositive/bucketsortoptimize_insert_8.q.out @@ -71,21 +71,44 @@ FROM test_table1 a JOIN test_table2 b ON a.key = b.key WHERE a.ds = '1' and b.ds = '1' POSTHOOK: type: QUERY STAGE DEPENDENCIES: - Stage-1 is a root stage - Stage-0 depends on stages: Stage-1 + Stage-6 is a root stage , consists of Stage-7, Stage-8, Stage-1 + Stage-7 has a backup stage: Stage-1 + Stage-4 depends on stages: Stage-7 + Stage-0 depends on stages: Stage-1, Stage-4, Stage-5 Stage-2 depends on stages: Stage-0 + Stage-8 has a backup stage: Stage-1 + Stage-5 depends on stages: Stage-8 + Stage-1 STAGE PLANS: - Stage: Stage-1 + Stage: Stage-6 + Conditional Operator + + Stage: Stage-7 + Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + b + TableScan + alias: b + Filter Operator + predicate: key is not null (type: boolean) + HashTable Sink Operator + keys: + 0 key (type: int) + 1 key (type: int) + + Stage: Stage-4 Map Reduce Map Operator Tree: TableScan alias: a - Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE - Sorted Merge Bucket Map Join Operator + Map Join Operator condition map: Inner Join 0 to 1 keys: @@ -95,13 +118,24 @@ STAGE PLANS: Select Operator expressions: _col0 (type: int), _col6 (type: int), concat(_col1, _col7) (type: string) outputColumnNames: _col0, _col1, _col2 - File Output Operator - compressed: false - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.test_table3 + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + value expressions: _col1 (type: int), _col2 (type: string) + Local Work: + Map Reduce Local Work + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: int), VALUE._col1 (type: string) + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table3 Stage: Stage-0 Move Operator @@ -118,6 +152,95 @@ STAGE PLANS: Stage: Stage-2 Stats-Aggr Operator + Stage: Stage-8 + Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a + TableScan + alias: a + Filter Operator + predicate: key is not null (type: boolean) + HashTable Sink Operator + keys: + 0 key (type: int) + 1 key (type: int) + + Stage: Stage-5 + Map Reduce + Map Operator Tree: + TableScan + alias: b + Filter Operator + predicate: key is not null (type: boolean) + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 key (type: int) + 1 key (type: int) + outputColumnNames: _col0, _col1, _col6, _col7 + Select Operator + expressions: _col0 (type: int), _col6 (type: int), concat(_col1, _col7) (type: string) + outputColumnNames: _col0, _col1, _col2 + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + value expressions: _col1 (type: int), _col2 (type: string) + Local Work: + Map Reduce Local Work + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: int), VALUE._col1 (type: string) + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table3 + + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 key (type: int) + 1 key (type: int) + outputColumnNames: _col0, _col1, _col6, _col7 + Select Operator + expressions: _col0 (type: int), _col6 (type: int), concat(_col1, _col7) (type: string) + outputColumnNames: _col0, _col1, _col2 + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + value expressions: _col1 (type: int), _col2 (type: string) + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: int), VALUE._col1 (type: string) + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table3 + PREHOOK: query: INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') SELECT a.key, b.key, concat(a.value, b.value) FROM test_table1 a JOIN test_table2 b @@ -128,6 +251,8 @@ PREHOOK: Input: default@test_table1@ds=1 PREHOOK: Input: default@test_table2 PREHOOK: Input: default@test_table2@ds=1 PREHOOK: Output: default@test_table3@ds=1 +FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapredLocalTask +ATTEMPT: Execute BackupTask: org.apache.hadoop.hive.ql.exec.mr.MapRedTask POSTHOOK: query: INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') SELECT a.key, b.key, concat(a.value, b.value) FROM test_table1 a JOIN test_table2 b @@ -200,21 +325,44 @@ FROM test_table1 a JOIN test_table2 b ON a.key = b.key WHERE a.ds = '1' and b.ds = '1' POSTHOOK: type: QUERY STAGE DEPENDENCIES: - Stage-1 is a root stage - Stage-0 depends on stages: Stage-1 + Stage-6 is a root stage , consists of Stage-7, Stage-8, Stage-1 + Stage-7 has a backup stage: Stage-1 + Stage-4 depends on stages: Stage-7 + Stage-0 depends on stages: Stage-1, Stage-4, Stage-5 Stage-2 depends on stages: Stage-0 + Stage-8 has a backup stage: Stage-1 + Stage-5 depends on stages: Stage-8 + Stage-1 STAGE PLANS: - Stage: Stage-1 + Stage: Stage-6 + Conditional Operator + + Stage: Stage-7 + Map Reduce Local Work + Alias -> Map Local Tables: + b + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + b + TableScan + alias: b + Filter Operator + predicate: key is not null (type: boolean) + HashTable Sink Operator + keys: + 0 key (type: int) + 1 key (type: int) + + Stage: Stage-4 Map Reduce Map Operator Tree: TableScan alias: a - Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: key is not null (type: boolean) - Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE - Sorted Merge Bucket Map Join Operator + Map Join Operator condition map: Inner Join 0 to 1 keys: @@ -224,13 +372,24 @@ STAGE PLANS: Select Operator expressions: _col6 (type: int), _col0 (type: int), concat(_col1, _col7) (type: string) outputColumnNames: _col0, _col1, _col2 - File Output Operator - compressed: false - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.test_table3 + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + value expressions: _col1 (type: int), _col2 (type: string) + Local Work: + Map Reduce Local Work + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: int), VALUE._col1 (type: string) + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table3 Stage: Stage-0 Move Operator @@ -247,6 +406,95 @@ STAGE PLANS: Stage: Stage-2 Stats-Aggr Operator + Stage: Stage-8 + Map Reduce Local Work + Alias -> Map Local Tables: + a + Fetch Operator + limit: -1 + Alias -> Map Local Operator Tree: + a + TableScan + alias: a + Filter Operator + predicate: key is not null (type: boolean) + HashTable Sink Operator + keys: + 0 key (type: int) + 1 key (type: int) + + Stage: Stage-5 + Map Reduce + Map Operator Tree: + TableScan + alias: b + Filter Operator + predicate: key is not null (type: boolean) + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 key (type: int) + 1 key (type: int) + outputColumnNames: _col0, _col1, _col6, _col7 + Select Operator + expressions: _col6 (type: int), _col0 (type: int), concat(_col1, _col7) (type: string) + outputColumnNames: _col0, _col1, _col2 + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + value expressions: _col1 (type: int), _col2 (type: string) + Local Work: + Map Reduce Local Work + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: int), VALUE._col1 (type: string) + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table3 + + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: a + Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE + Sorted Merge Bucket Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 key (type: int) + 1 key (type: int) + outputColumnNames: _col0, _col1, _col6, _col7 + Select Operator + expressions: _col6 (type: int), _col0 (type: int), concat(_col1, _col7) (type: string) + outputColumnNames: _col0, _col1, _col2 + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + value expressions: _col1 (type: int), _col2 (type: string) + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: int), VALUE._col1 (type: string) + outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table3 + PREHOOK: query: INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') SELECT b.key, a.key, concat(a.value, b.value) FROM test_table1 a JOIN test_table2 b @@ -257,6 +505,8 @@ PREHOOK: Input: default@test_table1@ds=1 PREHOOK: Input: default@test_table2 PREHOOK: Input: default@test_table2@ds=1 PREHOOK: Output: default@test_table3@ds=1 +FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapredLocalTask +ATTEMPT: Execute BackupTask: org.apache.hadoop.hive.ql.exec.mr.MapRedTask POSTHOOK: query: INSERT OVERWRITE TABLE test_table3 PARTITION (ds = '1') SELECT b.key, a.key, concat(a.value, b.value) FROM test_table1 a JOIN test_table2 b diff --git a/ql/src/test/results/clientpositive/groupby_grouping_window.q.out b/ql/src/test/results/clientpositive/groupby_grouping_window.q.out index b82d2e5..c0e6e5f 100644 --- a/ql/src/test/results/clientpositive/groupby_grouping_window.q.out +++ b/ql/src/test/results/clientpositive/groupby_grouping_window.q.out @@ -86,9 +86,11 @@ STAGE PLANS: sort order: ++ Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int), _col2 (type: int), _col3 (type: int) + value expressions: _col2 (type: int) Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col1 (type: int), KEY.reducesinkkey1 (type: int) + outputColumnNames: _col0, _col2, _col3 Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE diff --git a/ql/src/test/results/clientpositive/load_dyn_part2.q.out b/ql/src/test/results/clientpositive/load_dyn_part2.q.out index 26f318a..0c5fe6b 100644 --- a/ql/src/test/results/clientpositive/load_dyn_part2.q.out +++ b/ql/src/test/results/clientpositive/load_dyn_part2.q.out @@ -60,7 +60,9 @@ STAGE PLANS: Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string) Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false diff --git a/ql/src/test/results/clientpositive/smb_mapjoin_20.q.out b/ql/src/test/results/clientpositive/smb_mapjoin_20.q.out index 999dabd..bb851c1 100644 --- a/ql/src/test/results/clientpositive/smb_mapjoin_20.q.out +++ b/ql/src/test/results/clientpositive/smb_mapjoin_20.q.out @@ -57,17 +57,19 @@ STAGE PLANS: alias: a Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: key (type: int), value (type: string), value (type: string) - outputColumnNames: _col0, _col1, _col2 + expressions: key (type: int), value (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: UDFToString(_col0) (type: string) sort order: + Map-reduce partition columns: UDFToString(_col0) (type: string) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string) + value expressions: _col0 (type: int), _col1 (type: string) Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col1 (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false @@ -176,17 +178,28 @@ STAGE PLANS: alias: a Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: value (type: string), key (type: int), value (type: string) - outputColumnNames: _col0, _col1, _col2 + expressions: value (type: string), key (type: int) + outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false + Reduce Output Operator + key expressions: _col1 (type: int) + sort order: + + Map-reduce partition columns: _col1 (type: int) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.test_table3 + value expressions: _col0 (type: string) + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: string), KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table3 Stage: Stage-0 Move Operator @@ -298,17 +311,19 @@ STAGE PLANS: alias: a Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: (key + key) (type: int), value (type: string), value (type: string) - outputColumnNames: _col0, _col1, _col2 + expressions: (key + key) (type: int), value (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: UDFToString(_col0) (type: string) sort order: + Map-reduce partition columns: UDFToString(_col0) (type: string) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string) + value expressions: _col0 (type: int), _col1 (type: string) Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col1 (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false diff --git a/ql/src/test/results/clientpositive/smb_mapjoin_21.q.out b/ql/src/test/results/clientpositive/smb_mapjoin_21.q.out index 539b70e..0203949 100644 --- a/ql/src/test/results/clientpositive/smb_mapjoin_21.q.out +++ b/ql/src/test/results/clientpositive/smb_mapjoin_21.q.out @@ -60,14 +60,25 @@ STAGE PLANS: expressions: key (type: int), value (type: string) outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.test_table2 + value expressions: _col1 (type: string) + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table2 Stage: Stage-0 Move Operator @@ -135,9 +146,11 @@ STAGE PLANS: sort order: - Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int), _col1 (type: string) + value expressions: _col1 (type: string) Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false @@ -214,9 +227,10 @@ STAGE PLANS: sort order: ++ Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int), _col1 (type: string) Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false @@ -293,9 +307,11 @@ STAGE PLANS: sort order: + Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int), _col1 (type: string) + value expressions: _col0 (type: int) Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey0 (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false @@ -372,9 +388,11 @@ STAGE PLANS: sort order: + Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int), _col1 (type: string) + value expressions: _col1 (type: string) Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false @@ -452,7 +470,9 @@ STAGE PLANS: Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col1 (type: string) Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false diff --git a/ql/src/test/results/clientpositive/spark/bucket2.q.out b/ql/src/test/results/clientpositive/spark/bucket2.q.out index 5eb28fa..89c3b4c 100644 --- a/ql/src/test/results/clientpositive/spark/bucket2.q.out +++ b/ql/src/test/results/clientpositive/spark/bucket2.q.out @@ -116,39 +116,37 @@ STAGE PLANS: Reducer 2 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: UDFToInteger(VALUE._col0) (type: int), VALUE._col1 (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: UDFToInteger(_col0) (type: int), _col1 (type: string) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - GlobalTableId: 1 + File Output Operator + compressed: false + GlobalTableId: 1 #### A masked pattern was here #### - NumFilesPerFileSink: 2 - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + NumFilesPerFileSink: 2 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE 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: - bucket_count 2 - bucket_field_name key - columns key,value - columns.comments - columns.types int:string + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types int:string #### A masked pattern was here #### - name default.bucket2_1 - serialization.ddl struct bucket2_1 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name default.bucket2_1 + serialization.ddl struct bucket2_1 { i32 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.bucket2_1 - TotalFiles: 2 - GatherStats: true - MultiFileSpray: true + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket2_1 + TotalFiles: 2 + GatherStats: true + MultiFileSpray: true Stage: Stage-0 Move Operator diff --git a/ql/src/test/results/clientpositive/spark/bucket3.q.out b/ql/src/test/results/clientpositive/spark/bucket3.q.out index 1b1010a..2fc4855 100644 --- a/ql/src/test/results/clientpositive/spark/bucket3.q.out +++ b/ql/src/test/results/clientpositive/spark/bucket3.q.out @@ -120,42 +120,40 @@ STAGE PLANS: Reducer 2 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: UDFToInteger(VALUE._col0) (type: int), VALUE._col1 (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: UDFToInteger(_col0) (type: int), _col1 (type: string) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - GlobalTableId: 1 + File Output Operator + compressed: false + GlobalTableId: 1 #### A masked pattern was here #### - NumFilesPerFileSink: 2 - Static Partition Specification: ds=1/ - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + NumFilesPerFileSink: 2 + Static Partition Specification: ds=1/ + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE 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: - bucket_count 2 - bucket_field_name key - columns key,value - columns.comments - columns.types int:string + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types int:string #### A masked pattern was here #### - name default.bucket3_1 - partition_columns ds - partition_columns.types string - serialization.ddl struct bucket3_1 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name default.bucket3_1 + partition_columns ds + partition_columns.types string + serialization.ddl struct bucket3_1 { i32 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.bucket3_1 - TotalFiles: 2 - GatherStats: true - MultiFileSpray: true + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket3_1 + TotalFiles: 2 + GatherStats: true + MultiFileSpray: true Stage: Stage-0 Move Operator diff --git a/ql/src/test/results/clientpositive/spark/bucket4.q.out b/ql/src/test/results/clientpositive/spark/bucket4.q.out index 7dd49ac..44e0f9f 100644 --- a/ql/src/test/results/clientpositive/spark/bucket4.q.out +++ b/ql/src/test/results/clientpositive/spark/bucket4.q.out @@ -113,40 +113,38 @@ STAGE PLANS: Reducer 2 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: UDFToInteger(VALUE._col0) (type: int), VALUE._col1 (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: UDFToInteger(_col0) (type: int), _col1 (type: string) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - GlobalTableId: 1 + File Output Operator + compressed: false + GlobalTableId: 1 #### A masked pattern was here #### - NumFilesPerFileSink: 2 - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + NumFilesPerFileSink: 2 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE 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: - SORTBUCKETCOLSPREFIX TRUE - bucket_count 2 - bucket_field_name key - columns key,value - columns.comments - columns.types int:string + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types int:string #### A masked pattern was here #### - name default.bucket4_1 - serialization.ddl struct bucket4_1 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name default.bucket4_1 + serialization.ddl struct bucket4_1 { i32 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.bucket4_1 - TotalFiles: 2 - GatherStats: true - MultiFileSpray: true + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket4_1 + TotalFiles: 2 + GatherStats: true + MultiFileSpray: true Stage: Stage-0 Move Operator diff --git a/ql/src/test/results/clientpositive/spark/bucketsortoptimize_insert_2.q.out b/ql/src/test/results/clientpositive/spark/bucketsortoptimize_insert_2.q.out index 365306e..bcef03c 100644 --- a/ql/src/test/results/clientpositive/spark/bucketsortoptimize_insert_2.q.out +++ b/ql/src/test/results/clientpositive/spark/bucketsortoptimize_insert_2.q.out @@ -153,12 +153,14 @@ STAGE PLANS: sort order: + Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 46 Data size: 404 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int), _col1 (type: string) + value expressions: _col1 (type: string) Local Work: Map Reduce Local Work Reducer 3 Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 46 Data size: 404 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false @@ -323,12 +325,14 @@ STAGE PLANS: sort order: + Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 46 Data size: 404 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int), _col1 (type: string) + value expressions: _col1 (type: string) Local Work: Map Reduce Local Work Reducer 3 Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 46 Data size: 404 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false @@ -517,12 +521,14 @@ STAGE PLANS: sort order: + Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 92 Data size: 809 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int), _col1 (type: string) + value expressions: _col1 (type: string) Local Work: Map Reduce Local Work Reducer 3 Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 92 Data size: 809 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false @@ -723,12 +729,14 @@ STAGE PLANS: sort order: + Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 46 Data size: 404 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int), _col1 (type: string) + value expressions: _col1 (type: string) Local Work: Map Reduce Local Work Reducer 3 Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 46 Data size: 404 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false @@ -911,12 +919,14 @@ STAGE PLANS: sort order: + Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 46 Data size: 404 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int), _col1 (type: string) + value expressions: _col1 (type: string) Local Work: Map Reduce Local Work Reducer 3 Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 46 Data size: 404 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false @@ -1099,12 +1109,14 @@ STAGE PLANS: sort order: + Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 46 Data size: 404 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int), _col1 (type: string) + value expressions: _col1 (type: string) Local Work: Map Reduce Local Work Reducer 3 Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 46 Data size: 404 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false diff --git a/ql/src/test/results/clientpositive/spark/bucketsortoptimize_insert_4.q.out b/ql/src/test/results/clientpositive/spark/bucketsortoptimize_insert_4.q.out index 3846de7..f3c8f04 100644 --- a/ql/src/test/results/clientpositive/spark/bucketsortoptimize_insert_4.q.out +++ b/ql/src/test/results/clientpositive/spark/bucketsortoptimize_insert_4.q.out @@ -121,20 +121,22 @@ STAGE PLANS: 0 Map 1 Statistics: Num rows: 46 Data size: 404 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: _col0 (type: int), _col0 (type: int), concat(_col1, _col7) (type: string) - outputColumnNames: _col0, _col1, _col2 + expressions: _col0 (type: int), concat(_col1, _col7) (type: string) + outputColumnNames: _col1, _col2 Statistics: Num rows: 46 Data size: 404 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col1 (type: int) sort order: + Map-reduce partition columns: _col1 (type: int) Statistics: Num rows: 46 Data size: 404 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string) + value expressions: _col2 (type: string) Local Work: Map Reduce Local Work Reducer 3 Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 46 Data size: 404 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false @@ -314,12 +316,14 @@ STAGE PLANS: sort order: + Map-reduce partition columns: _col1 (type: string) Statistics: Num rows: 46 Data size: 404 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int), _col1 (type: string) + value expressions: _col0 (type: int) Local Work: Map Reduce Local Work Reducer 3 Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey0 (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 46 Data size: 404 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false diff --git a/ql/src/test/results/clientpositive/spark/bucketsortoptimize_insert_6.q.out b/ql/src/test/results/clientpositive/spark/bucketsortoptimize_insert_6.q.out index 5b559c4..25f61b8 100644 --- a/ql/src/test/results/clientpositive/spark/bucketsortoptimize_insert_6.q.out +++ b/ql/src/test/results/clientpositive/spark/bucketsortoptimize_insert_6.q.out @@ -131,12 +131,14 @@ STAGE PLANS: sort order: +- Map-reduce partition columns: _col0 (type: int), _col1 (type: int) Statistics: Num rows: 23 Data size: 268 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string) + value expressions: _col2 (type: string) Local Work: Map Reduce Local Work Reducer 3 Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 23 Data size: 268 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false @@ -308,12 +310,14 @@ STAGE PLANS: sort order: +- Map-reduce partition columns: _col0 (type: int), _col1 (type: int) Statistics: Num rows: 23 Data size: 268 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string) + value expressions: _col2 (type: string) Local Work: Map Reduce Local Work Reducer 3 Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 23 Data size: 268 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false @@ -485,12 +489,14 @@ STAGE PLANS: sort order: +- Map-reduce partition columns: _col0 (type: int), _col1 (type: int) Statistics: Num rows: 23 Data size: 268 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string) + value expressions: _col2 (type: string) Local Work: Map Reduce Local Work Reducer 3 Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 23 Data size: 268 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false @@ -597,12 +603,14 @@ STAGE PLANS: sort order: +- Map-reduce partition columns: _col0 (type: int), _col1 (type: int) Statistics: Num rows: 23 Data size: 268 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string) + value expressions: _col2 (type: string) Local Work: Map Reduce Local Work Reducer 3 Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 23 Data size: 268 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false @@ -715,12 +723,14 @@ STAGE PLANS: sort order: +- Map-reduce partition columns: _col0 (type: int), _col1 (type: int) Statistics: Num rows: 23 Data size: 268 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string) + value expressions: _col2 (type: string) Local Work: Map Reduce Local Work Reducer 3 Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 23 Data size: 268 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false @@ -910,12 +920,14 @@ STAGE PLANS: sort order: +- Map-reduce partition columns: _col0 (type: int), _col1 (type: int) Statistics: Num rows: 23 Data size: 268 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string) + value expressions: _col2 (type: string) Local Work: Map Reduce Local Work Reducer 3 Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 23 Data size: 268 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false @@ -1115,12 +1127,14 @@ STAGE PLANS: sort order: -- Map-reduce partition columns: _col0 (type: int), _col1 (type: int) Statistics: Num rows: 23 Data size: 268 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string) + value expressions: _col2 (type: string) Local Work: Map Reduce Local Work Reducer 3 Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 23 Data size: 268 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false diff --git a/ql/src/test/results/clientpositive/spark/bucketsortoptimize_insert_7.q.out b/ql/src/test/results/clientpositive/spark/bucketsortoptimize_insert_7.q.out index cefc6aa..7c087ec 100644 --- a/ql/src/test/results/clientpositive/spark/bucketsortoptimize_insert_7.q.out +++ b/ql/src/test/results/clientpositive/spark/bucketsortoptimize_insert_7.q.out @@ -131,12 +131,14 @@ STAGE PLANS: sort order: + Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 46 Data size: 404 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int), _col1 (type: string) + value expressions: _col1 (type: string) Local Work: Map Reduce Local Work Reducer 3 Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 46 Data size: 404 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false @@ -311,12 +313,14 @@ STAGE PLANS: sort order: + Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 46 Data size: 404 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int), _col1 (type: string) + value expressions: _col1 (type: string) Local Work: Map Reduce Local Work Reducer 3 Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 46 Data size: 404 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false @@ -497,12 +501,14 @@ STAGE PLANS: sort order: + Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 15 Data size: 134 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int), _col1 (type: string) + value expressions: _col1 (type: string) Local Work: Map Reduce Local Work Reducer 3 Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 15 Data size: 134 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false diff --git a/ql/src/test/results/clientpositive/spark/bucketsortoptimize_insert_8.q.out b/ql/src/test/results/clientpositive/spark/bucketsortoptimize_insert_8.q.out index ca44d7c..dade75c 100644 --- a/ql/src/test/results/clientpositive/spark/bucketsortoptimize_insert_8.q.out +++ b/ql/src/test/results/clientpositive/spark/bucketsortoptimize_insert_8.q.out @@ -129,12 +129,14 @@ STAGE PLANS: sort order: + Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 46 Data size: 404 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string) + value expressions: _col1 (type: int), _col2 (type: string) Local Work: Map Reduce Local Work Reducer 3 Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: int), VALUE._col1 (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 46 Data size: 404 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false @@ -300,12 +302,14 @@ STAGE PLANS: sort order: + Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 46 Data size: 404 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string) + value expressions: _col1 (type: int), _col2 (type: string) Local Work: Map Reduce Local Work Reducer 3 Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: int), VALUE._col1 (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 46 Data size: 404 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false diff --git a/ql/src/test/results/clientpositive/spark/disable_merge_for_bucketing.q.out b/ql/src/test/results/clientpositive/spark/disable_merge_for_bucketing.q.out index 3864c44..d39c5be 100644 --- a/ql/src/test/results/clientpositive/spark/disable_merge_for_bucketing.q.out +++ b/ql/src/test/results/clientpositive/spark/disable_merge_for_bucketing.q.out @@ -112,39 +112,37 @@ STAGE PLANS: Reducer 2 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: UDFToInteger(VALUE._col0) (type: int), VALUE._col1 (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: UDFToInteger(_col0) (type: int), _col1 (type: string) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - GlobalTableId: 1 + File Output Operator + compressed: false + GlobalTableId: 1 #### A masked pattern was here #### - NumFilesPerFileSink: 2 - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + NumFilesPerFileSink: 2 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE 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: - bucket_count 2 - bucket_field_name key - columns key,value - columns.comments - columns.types int:string + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types int:string #### A masked pattern was here #### - name default.bucket2_1 - serialization.ddl struct bucket2_1 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name default.bucket2_1 + serialization.ddl struct bucket2_1 { i32 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.bucket2_1 - TotalFiles: 2 - GatherStats: true - MultiFileSpray: true + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket2_1 + TotalFiles: 2 + GatherStats: true + MultiFileSpray: true Stage: Stage-0 Move Operator diff --git a/ql/src/test/results/clientpositive/spark/load_dyn_part2.q.out b/ql/src/test/results/clientpositive/spark/load_dyn_part2.q.out index a8cef34..9177290 100644 --- a/ql/src/test/results/clientpositive/spark/load_dyn_part2.q.out +++ b/ql/src/test/results/clientpositive/spark/load_dyn_part2.q.out @@ -66,7 +66,9 @@ STAGE PLANS: value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string) Reducer 2 Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false diff --git a/ql/src/test/results/clientpositive/spark/smb_mapjoin_20.q.out b/ql/src/test/results/clientpositive/spark/smb_mapjoin_20.q.out index 9815d24..de340b3 100644 --- a/ql/src/test/results/clientpositive/spark/smb_mapjoin_20.q.out +++ b/ql/src/test/results/clientpositive/spark/smb_mapjoin_20.q.out @@ -62,18 +62,20 @@ STAGE PLANS: alias: a Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: key (type: int), value (type: string), value (type: string) - outputColumnNames: _col0, _col1, _col2 + expressions: key (type: int), value (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: UDFToString(_col0) (type: string) sort order: + Map-reduce partition columns: UDFToString(_col0) (type: string) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string) + value expressions: _col0 (type: int), _col1 (type: string) Reducer 2 Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col1 (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false @@ -177,6 +179,8 @@ STAGE DEPENDENCIES: STAGE PLANS: Stage: Stage-1 Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -185,17 +189,29 @@ STAGE PLANS: alias: a Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: value (type: string), key (type: int), value (type: string) - outputColumnNames: _col0, _col1, _col2 + expressions: value (type: string), key (type: int) + outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false + Reduce Output Operator + key expressions: _col1 (type: int) + sort order: + + Map-reduce partition columns: _col1 (type: int) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.test_table3 + value expressions: _col0 (type: string) + Reducer 2 + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: string), KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table3 Stage: Stage-0 Move Operator @@ -312,18 +328,20 @@ STAGE PLANS: alias: a Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: (key + key) (type: int), value (type: string), value (type: string) - outputColumnNames: _col0, _col1, _col2 + expressions: (key + key) (type: int), value (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: UDFToString(_col0) (type: string) sort order: + Map-reduce partition columns: UDFToString(_col0) (type: string) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string) + value expressions: _col0 (type: int), _col1 (type: string) Reducer 2 Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), VALUE._col1 (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false diff --git a/ql/src/test/results/clientpositive/spark/smb_mapjoin_21.q.out b/ql/src/test/results/clientpositive/spark/smb_mapjoin_21.q.out index 44bae6e..035c5d2 100644 --- a/ql/src/test/results/clientpositive/spark/smb_mapjoin_21.q.out +++ b/ql/src/test/results/clientpositive/spark/smb_mapjoin_21.q.out @@ -52,6 +52,8 @@ STAGE DEPENDENCIES: STAGE PLANS: Stage: Stage-1 Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -63,14 +65,26 @@ STAGE PLANS: expressions: key (type: int), value (type: string) outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.test_table2 + value expressions: _col1 (type: string) + Reducer 2 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.test_table2 Stage: Stage-0 Move Operator @@ -143,10 +157,12 @@ STAGE PLANS: sort order: - Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int), _col1 (type: string) + value expressions: _col1 (type: string) Reducer 2 Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false @@ -228,10 +244,11 @@ STAGE PLANS: sort order: ++ Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int), _col1 (type: string) Reducer 2 Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false @@ -313,10 +330,12 @@ STAGE PLANS: sort order: + Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int), _col1 (type: string) + value expressions: _col0 (type: int) Reducer 2 Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey0 (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false @@ -398,10 +417,12 @@ STAGE PLANS: sort order: + Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: int), _col1 (type: string) + value expressions: _col1 (type: string) Reducer 2 Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false @@ -485,7 +506,9 @@ STAGE PLANS: value expressions: _col0 (type: int), _col1 (type: string) Reducer 2 Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false diff --git a/ql/src/test/results/clientpositive/spark/stats10.q.out b/ql/src/test/results/clientpositive/spark/stats10.q.out index 4bec7fc..c840ab7 100644 --- a/ql/src/test/results/clientpositive/spark/stats10.q.out +++ b/ql/src/test/results/clientpositive/spark/stats10.q.out @@ -42,20 +42,18 @@ STAGE PLANS: value expressions: _col0 (type: string), _col1 (type: string) Reducer 2 Reduce Operator Tree: - Extract + Select Operator + expressions: UDFToInteger(VALUE._col0) (type: int), VALUE._col1 (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: UDFToInteger(_col0) (type: int), _col1 (type: string) - outputColumnNames: _col0, _col1 + File Output Operator + compressed: false Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket3_1 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket3_1 Stage: Stage-0 Move Operator diff --git a/ql/src/test/results/clientpositive/spark/vectorized_ptf.q.out b/ql/src/test/results/clientpositive/spark/vectorized_ptf.q.out index f3b61ce..f46f335 100644 --- a/ql/src/test/results/clientpositive/spark/vectorized_ptf.q.out +++ b/ql/src/test/results/clientpositive/spark/vectorized_ptf.q.out @@ -244,7 +244,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -298,7 +298,9 @@ STAGE PLANS: Reducer 2 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -672,12 +674,14 @@ STAGE PLANS: Map-reduce partition columns: _col2 (type: string) Statistics: Num rows: 14 Data size: 8823 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) + value expressions: _col0 (type: int), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) auto parallelism: false Reducer 3 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 Statistics: Num rows: 14 Data size: 8823 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 14 Data size: 8823 Basic stats: COMPLETE Column stats: NONE @@ -850,7 +854,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -904,7 +908,9 @@ STAGE PLANS: Reducer 2 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -1107,7 +1113,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -1161,7 +1167,9 @@ STAGE PLANS: Reducer 2 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -1397,7 +1405,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -1451,7 +1459,9 @@ STAGE PLANS: Reducer 2 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -1697,7 +1707,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -1751,7 +1761,9 @@ STAGE PLANS: Reducer 2 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -1963,7 +1975,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -2084,7 +2096,9 @@ STAGE PLANS: Reducer 2 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -2343,7 +2357,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -2433,7 +2447,9 @@ STAGE PLANS: Reducer 4 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -2600,7 +2616,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -2654,7 +2670,9 @@ STAGE PLANS: Reducer 2 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), KEY.reducesinkkey2 (type: int), VALUE._col3 (type: string), VALUE._col4 (type: double), VALUE._col5 (type: string), VALUE._col6 (type: bigint), VALUE._col7 (type: string), VALUE._col8 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -2878,7 +2896,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -2932,7 +2950,9 @@ STAGE PLANS: Reducer 2 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -3159,7 +3179,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -3213,7 +3233,9 @@ STAGE PLANS: Reducer 2 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -3450,7 +3472,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -3504,7 +3526,9 @@ STAGE PLANS: Reducer 2 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -3516,12 +3540,14 @@ STAGE PLANS: Map-reduce partition columns: _col2 (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) + value expressions: _col0 (type: int), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) auto parallelism: false Reducer 3 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -3778,7 +3804,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -3832,7 +3858,9 @@ STAGE PLANS: Reducer 2 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -4166,7 +4194,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -4287,7 +4315,9 @@ STAGE PLANS: Reducer 2 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -4494,7 +4524,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -4548,7 +4578,9 @@ STAGE PLANS: Reducer 2 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -4843,13 +4875,15 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 13 Data size: 8021 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: double) + value expressions: _col1 (type: string), _col2 (type: double) auto parallelism: false Execution mode: vectorized Reducer 3 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string), VALUE._col1 (type: double) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 13 Data size: 8021 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 13 Data size: 8021 Basic stats: COMPLETE Column stats: NONE @@ -5239,7 +5273,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -5388,7 +5422,9 @@ STAGE PLANS: Reducer 6 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -5403,7 +5439,9 @@ STAGE PLANS: Reducer 7 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -5757,7 +5795,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_name (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -5811,7 +5849,9 @@ STAGE PLANS: Reducer 2 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: string), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: double), VALUE._col7 (type: string), VALUE._col8 (type: bigint), VALUE._col9 (type: string), VALUE._col10 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -5823,12 +5863,14 @@ STAGE PLANS: Map-reduce partition columns: _col2 (type: string), _col1 (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) + value expressions: _col0 (type: int), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) auto parallelism: false Reducer 3 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -6116,7 +6158,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_name (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -6170,7 +6212,9 @@ STAGE PLANS: Reducer 2 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: string), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: double), VALUE._col7 (type: string), VALUE._col8 (type: bigint), VALUE._col9 (type: string), VALUE._col10 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -6180,12 +6224,14 @@ STAGE PLANS: Map-reduce partition columns: _col2 (type: string), _col1 (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) + value expressions: _col0 (type: int), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) auto parallelism: false Reducer 3 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -6195,12 +6241,14 @@ STAGE PLANS: Map-reduce partition columns: _col2 (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) + value expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) auto parallelism: false Reducer 4 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: string), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: double), VALUE._col7 (type: string), VALUE._col8 (type: bigint), VALUE._col9 (type: string), VALUE._col10 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -6471,7 +6519,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string), p_name (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -6525,7 +6573,9 @@ STAGE PLANS: Reducer 2 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -6535,12 +6585,14 @@ STAGE PLANS: Map-reduce partition columns: _col2 (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) + value expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) auto parallelism: false Reducer 3 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: string), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: double), VALUE._col7 (type: string), VALUE._col8 (type: bigint), VALUE._col9 (type: string), VALUE._col10 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -6823,7 +6875,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string), p_name (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -6877,7 +6929,9 @@ STAGE PLANS: Reducer 2 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -6887,12 +6941,14 @@ STAGE PLANS: Map-reduce partition columns: _col2 (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) + value expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) auto parallelism: false Reducer 3 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: string), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: double), VALUE._col7 (type: string), VALUE._col8 (type: bigint), VALUE._col9 (type: string), VALUE._col10 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -6904,12 +6960,14 @@ STAGE PLANS: Map-reduce partition columns: _col2 (type: string), _col1 (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) + value expressions: _col0 (type: int), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) auto parallelism: false Reducer 4 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -7223,7 +7281,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string), p_name (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -7277,7 +7335,9 @@ STAGE PLANS: Reducer 2 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -7289,12 +7349,14 @@ STAGE PLANS: Map-reduce partition columns: _col2 (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) + value expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) auto parallelism: false Reducer 3 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: string), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: double), VALUE._col7 (type: string), VALUE._col8 (type: bigint), VALUE._col9 (type: string), VALUE._col10 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -7574,7 +7636,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string), p_name (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -7628,7 +7690,9 @@ STAGE PLANS: Reducer 2 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -7640,12 +7704,14 @@ STAGE PLANS: Map-reduce partition columns: _col2 (type: string), _col1 (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) + value expressions: _col0 (type: int), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) auto parallelism: false Reducer 3 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE diff --git a/ql/src/test/results/clientpositive/stats10.q.out b/ql/src/test/results/clientpositive/stats10.q.out index 804e1b8..dc8aa7f 100644 --- a/ql/src/test/results/clientpositive/stats10.q.out +++ b/ql/src/test/results/clientpositive/stats10.q.out @@ -36,20 +36,18 @@ STAGE PLANS: Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: string), _col1 (type: string) Reduce Operator Tree: - Extract + Select Operator + expressions: UDFToInteger(VALUE._col0) (type: int), VALUE._col1 (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: UDFToInteger(_col0) (type: int), _col1 (type: string) - outputColumnNames: _col0, _col1 + File Output Operator + compressed: false Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket3_1 + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket3_1 Stage: Stage-0 Move Operator diff --git a/ql/src/test/results/clientpositive/tez/bucket2.q.out b/ql/src/test/results/clientpositive/tez/bucket2.q.out index 9536ed0..55aa220 100644 --- a/ql/src/test/results/clientpositive/tez/bucket2.q.out +++ b/ql/src/test/results/clientpositive/tez/bucket2.q.out @@ -117,39 +117,37 @@ STAGE PLANS: Reducer 2 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: UDFToInteger(VALUE._col0) (type: int), VALUE._col1 (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: UDFToInteger(_col0) (type: int), _col1 (type: string) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - GlobalTableId: 1 + File Output Operator + compressed: false + GlobalTableId: 1 #### A masked pattern was here #### - NumFilesPerFileSink: 2 - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + NumFilesPerFileSink: 2 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE 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: - bucket_count 2 - bucket_field_name key - columns key,value - columns.comments - columns.types int:string + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types int:string #### A masked pattern was here #### - name default.bucket2_1 - serialization.ddl struct bucket2_1 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name default.bucket2_1 + serialization.ddl struct bucket2_1 { i32 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.bucket2_1 - TotalFiles: 2 - GatherStats: true - MultiFileSpray: true + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket2_1 + TotalFiles: 2 + GatherStats: true + MultiFileSpray: true Stage: Stage-2 Dependency Collection diff --git a/ql/src/test/results/clientpositive/tez/bucket3.q.out b/ql/src/test/results/clientpositive/tez/bucket3.q.out index 3427474..bbd9c67 100644 --- a/ql/src/test/results/clientpositive/tez/bucket3.q.out +++ b/ql/src/test/results/clientpositive/tez/bucket3.q.out @@ -121,42 +121,40 @@ STAGE PLANS: Reducer 2 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: UDFToInteger(VALUE._col0) (type: int), VALUE._col1 (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: UDFToInteger(_col0) (type: int), _col1 (type: string) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - GlobalTableId: 1 + File Output Operator + compressed: false + GlobalTableId: 1 #### A masked pattern was here #### - NumFilesPerFileSink: 2 - Static Partition Specification: ds=1/ - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + NumFilesPerFileSink: 2 + Static Partition Specification: ds=1/ + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE 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: - bucket_count 2 - bucket_field_name key - columns key,value - columns.comments - columns.types int:string + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types int:string #### A masked pattern was here #### - name default.bucket3_1 - partition_columns ds - partition_columns.types string - serialization.ddl struct bucket3_1 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name default.bucket3_1 + partition_columns ds + partition_columns.types string + serialization.ddl struct bucket3_1 { i32 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.bucket3_1 - TotalFiles: 2 - GatherStats: true - MultiFileSpray: true + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket3_1 + TotalFiles: 2 + GatherStats: true + MultiFileSpray: true Stage: Stage-2 Dependency Collection diff --git a/ql/src/test/results/clientpositive/tez/bucket4.q.out b/ql/src/test/results/clientpositive/tez/bucket4.q.out index ca3584e..ed8671c 100644 --- a/ql/src/test/results/clientpositive/tez/bucket4.q.out +++ b/ql/src/test/results/clientpositive/tez/bucket4.q.out @@ -114,40 +114,38 @@ STAGE PLANS: Reducer 2 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: UDFToInteger(VALUE._col0) (type: int), VALUE._col1 (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: UDFToInteger(_col0) (type: int), _col1 (type: string) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - GlobalTableId: 1 + File Output Operator + compressed: false + GlobalTableId: 1 #### A masked pattern was here #### - NumFilesPerFileSink: 2 - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + NumFilesPerFileSink: 2 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE 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: - SORTBUCKETCOLSPREFIX TRUE - bucket_count 2 - bucket_field_name key - columns key,value - columns.comments - columns.types int:string + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + SORTBUCKETCOLSPREFIX TRUE + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types int:string #### A masked pattern was here #### - name default.bucket4_1 - serialization.ddl struct bucket4_1 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name default.bucket4_1 + serialization.ddl struct bucket4_1 { i32 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.bucket4_1 - TotalFiles: 2 - GatherStats: true - MultiFileSpray: true + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket4_1 + TotalFiles: 2 + GatherStats: true + MultiFileSpray: true Stage: Stage-2 Dependency Collection diff --git a/ql/src/test/results/clientpositive/tez/disable_merge_for_bucketing.q.out b/ql/src/test/results/clientpositive/tez/disable_merge_for_bucketing.q.out index dcffd87..84e0d2c 100644 --- a/ql/src/test/results/clientpositive/tez/disable_merge_for_bucketing.q.out +++ b/ql/src/test/results/clientpositive/tez/disable_merge_for_bucketing.q.out @@ -113,39 +113,37 @@ STAGE PLANS: Reducer 2 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: UDFToInteger(VALUE._col0) (type: int), VALUE._col1 (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: UDFToInteger(_col0) (type: int), _col1 (type: string) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - GlobalTableId: 1 + File Output Operator + compressed: false + GlobalTableId: 1 #### A masked pattern was here #### - NumFilesPerFileSink: 2 - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + NumFilesPerFileSink: 2 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE 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: - bucket_count 2 - bucket_field_name key - columns key,value - columns.comments - columns.types int:string + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + properties: + bucket_count 2 + bucket_field_name key + columns key,value + columns.comments + columns.types int:string #### A masked pattern was here #### - name default.bucket2_1 - serialization.ddl struct bucket2_1 { i32 key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name default.bucket2_1 + serialization.ddl struct bucket2_1 { i32 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.bucket2_1 - TotalFiles: 2 - GatherStats: true - MultiFileSpray: true + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.bucket2_1 + TotalFiles: 2 + GatherStats: true + MultiFileSpray: true Stage: Stage-2 Dependency Collection diff --git a/ql/src/test/results/clientpositive/tez/load_dyn_part2.q.out b/ql/src/test/results/clientpositive/tez/load_dyn_part2.q.out index d1c4a10..1c0472f 100644 --- a/ql/src/test/results/clientpositive/tez/load_dyn_part2.q.out +++ b/ql/src/test/results/clientpositive/tez/load_dyn_part2.q.out @@ -67,7 +67,9 @@ STAGE PLANS: value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string) Reducer 2 Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false diff --git a/ql/src/test/results/clientpositive/tez/vector_bucket.q.out b/ql/src/test/results/clientpositive/tez/vector_bucket.q.out index e4deb04..410212a 100644 --- a/ql/src/test/results/clientpositive/tez/vector_bucket.q.out +++ b/ql/src/test/results/clientpositive/tez/vector_bucket.q.out @@ -41,20 +41,18 @@ STAGE PLANS: value expressions: _col0 (type: string), _col1 (type: string) Reducer 2 Reduce Operator Tree: - Extract + Select Operator + expressions: UDFToInteger(VALUE._col0) (type: int), VALUE._col1 (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 0 Data size: 26 Basic stats: PARTIAL Column stats: NONE - Select Operator - expressions: UDFToInteger(_col0) (type: int), _col1 (type: string) - outputColumnNames: _col0, _col1 + File Output Operator + compressed: false Statistics: Num rows: 0 Data size: 26 Basic stats: PARTIAL Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 0 Data size: 26 Basic stats: PARTIAL Column stats: NONE - table: - input format: org.apache.hadoop.mapred.SequenceFileInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.non_orc_table + 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 + name: default.non_orc_table Execution mode: vectorized Stage: Stage-2 diff --git a/ql/src/test/results/clientpositive/tez/vectorized_ptf.q.out b/ql/src/test/results/clientpositive/tez/vectorized_ptf.q.out index 928c9f0..811238c 100644 --- a/ql/src/test/results/clientpositive/tez/vectorized_ptf.q.out +++ b/ql/src/test/results/clientpositive/tez/vectorized_ptf.q.out @@ -244,7 +244,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: true Path -> Alias: #### A masked pattern was here #### @@ -298,7 +298,9 @@ STAGE PLANS: Reducer 2 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -673,12 +675,14 @@ STAGE PLANS: Map-reduce partition columns: _col2 (type: string) Statistics: Num rows: 14 Data size: 8823 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) + value expressions: _col0 (type: int), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) auto parallelism: true Reducer 3 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 Statistics: Num rows: 14 Data size: 8823 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 14 Data size: 8823 Basic stats: COMPLETE Column stats: NONE @@ -851,7 +855,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: true Path -> Alias: #### A masked pattern was here #### @@ -905,7 +909,9 @@ STAGE PLANS: Reducer 2 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -1108,7 +1114,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: true Path -> Alias: #### A masked pattern was here #### @@ -1162,7 +1168,9 @@ STAGE PLANS: Reducer 2 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -1398,7 +1406,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: true Path -> Alias: #### A masked pattern was here #### @@ -1452,7 +1460,9 @@ STAGE PLANS: Reducer 2 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -1698,7 +1708,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: true Path -> Alias: #### A masked pattern was here #### @@ -1752,7 +1762,9 @@ STAGE PLANS: Reducer 2 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -1964,7 +1976,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: true Path -> Alias: #### A masked pattern was here #### @@ -2085,7 +2097,9 @@ STAGE PLANS: Reducer 2 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -2345,7 +2359,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: true Path -> Alias: #### A masked pattern was here #### @@ -2436,7 +2450,9 @@ STAGE PLANS: Reducer 4 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -2603,7 +2619,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: true Path -> Alias: #### A masked pattern was here #### @@ -2657,7 +2673,9 @@ STAGE PLANS: Reducer 2 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), KEY.reducesinkkey2 (type: int), VALUE._col3 (type: string), VALUE._col4 (type: double), VALUE._col5 (type: string), VALUE._col6 (type: bigint), VALUE._col7 (type: string), VALUE._col8 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -2881,7 +2899,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: true Path -> Alias: #### A masked pattern was here #### @@ -2935,7 +2953,9 @@ STAGE PLANS: Reducer 2 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -3162,7 +3182,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: true Path -> Alias: #### A masked pattern was here #### @@ -3216,7 +3236,9 @@ STAGE PLANS: Reducer 2 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -3453,7 +3475,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: true Path -> Alias: #### A masked pattern was here #### @@ -3507,7 +3529,9 @@ STAGE PLANS: Reducer 2 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -3519,12 +3543,14 @@ STAGE PLANS: Map-reduce partition columns: _col2 (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) + value expressions: _col0 (type: int), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) auto parallelism: true Reducer 3 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -3781,7 +3807,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: true Path -> Alias: #### A masked pattern was here #### @@ -3835,7 +3861,9 @@ STAGE PLANS: Reducer 2 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -4169,7 +4197,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: true Path -> Alias: #### A masked pattern was here #### @@ -4290,7 +4318,9 @@ STAGE PLANS: Reducer 2 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -4498,7 +4528,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: true Path -> Alias: #### A masked pattern was here #### @@ -4552,7 +4582,9 @@ STAGE PLANS: Reducer 2 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -4847,13 +4879,15 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 13 Data size: 8021 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: double) + value expressions: _col1 (type: string), _col2 (type: double) auto parallelism: true Execution mode: vectorized Reducer 3 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string), VALUE._col1 (type: double) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 13 Data size: 8021 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 13 Data size: 8021 Basic stats: COMPLETE Column stats: NONE @@ -5243,7 +5277,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: true Path -> Alias: #### A masked pattern was here #### @@ -5297,7 +5331,9 @@ STAGE PLANS: Reducer 2 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -5757,7 +5793,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_name (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: true Path -> Alias: #### A masked pattern was here #### @@ -5811,7 +5847,9 @@ STAGE PLANS: Reducer 2 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: string), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: double), VALUE._col7 (type: string), VALUE._col8 (type: bigint), VALUE._col9 (type: string), VALUE._col10 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -5823,12 +5861,14 @@ STAGE PLANS: Map-reduce partition columns: _col2 (type: string), _col1 (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) + value expressions: _col0 (type: int), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) auto parallelism: true Reducer 3 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -6116,7 +6156,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_name (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: true Path -> Alias: #### A masked pattern was here #### @@ -6170,7 +6210,9 @@ STAGE PLANS: Reducer 2 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: string), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: double), VALUE._col7 (type: string), VALUE._col8 (type: bigint), VALUE._col9 (type: string), VALUE._col10 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -6180,12 +6222,14 @@ STAGE PLANS: Map-reduce partition columns: _col2 (type: string), _col1 (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) + value expressions: _col0 (type: int), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) auto parallelism: true Reducer 3 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -6195,12 +6239,14 @@ STAGE PLANS: Map-reduce partition columns: _col2 (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) + value expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) auto parallelism: true Reducer 4 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: string), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: double), VALUE._col7 (type: string), VALUE._col8 (type: bigint), VALUE._col9 (type: string), VALUE._col10 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -6471,7 +6517,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string), p_name (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: true Path -> Alias: #### A masked pattern was here #### @@ -6525,7 +6571,9 @@ STAGE PLANS: Reducer 2 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -6535,12 +6583,14 @@ STAGE PLANS: Map-reduce partition columns: _col2 (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) + value expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) auto parallelism: true Reducer 3 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: string), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: double), VALUE._col7 (type: string), VALUE._col8 (type: bigint), VALUE._col9 (type: string), VALUE._col10 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -6823,7 +6873,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string), p_name (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: true Path -> Alias: #### A masked pattern was here #### @@ -6877,7 +6927,9 @@ STAGE PLANS: Reducer 2 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -6887,12 +6939,14 @@ STAGE PLANS: Map-reduce partition columns: _col2 (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) + value expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) auto parallelism: true Reducer 3 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: string), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: double), VALUE._col7 (type: string), VALUE._col8 (type: bigint), VALUE._col9 (type: string), VALUE._col10 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -6904,12 +6958,14 @@ STAGE PLANS: Map-reduce partition columns: _col2 (type: string), _col1 (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) + value expressions: _col0 (type: int), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) auto parallelism: true Reducer 4 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -7223,7 +7279,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string), p_name (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: true Path -> Alias: #### A masked pattern was here #### @@ -7277,7 +7333,9 @@ STAGE PLANS: Reducer 2 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -7289,12 +7347,14 @@ STAGE PLANS: Map-reduce partition columns: _col2 (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) + value expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) auto parallelism: true Reducer 3 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: string), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: double), VALUE._col7 (type: string), VALUE._col8 (type: bigint), VALUE._col9 (type: string), VALUE._col10 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -7574,7 +7634,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string), p_name (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: true Path -> Alias: #### A masked pattern was here #### @@ -7628,7 +7688,9 @@ STAGE PLANS: Reducer 2 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -7640,12 +7702,14 @@ STAGE PLANS: Map-reduce partition columns: _col2 (type: string), _col1 (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) + value expressions: _col0 (type: int), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) auto parallelism: true Reducer 3 Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE diff --git a/ql/src/test/results/clientpositive/vector_bucket.q.out b/ql/src/test/results/clientpositive/vector_bucket.q.out index 952c033..1ac37a7 100644 --- a/ql/src/test/results/clientpositive/vector_bucket.q.out +++ b/ql/src/test/results/clientpositive/vector_bucket.q.out @@ -34,20 +34,18 @@ STAGE PLANS: Statistics: Num rows: 0 Data size: 26 Basic stats: PARTIAL Column stats: NONE value expressions: _col0 (type: string), _col1 (type: string) Reduce Operator Tree: - Extract + Select Operator + expressions: UDFToInteger(VALUE._col0) (type: int), VALUE._col1 (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 0 Data size: 26 Basic stats: PARTIAL Column stats: NONE - Select Operator - expressions: UDFToInteger(_col0) (type: int), _col1 (type: string) - outputColumnNames: _col0, _col1 + File Output Operator + compressed: false Statistics: Num rows: 0 Data size: 26 Basic stats: PARTIAL Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 0 Data size: 26 Basic stats: PARTIAL Column stats: NONE - table: - input format: org.apache.hadoop.mapred.SequenceFileInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.non_orc_table + 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 + name: default.non_orc_table Stage: Stage-0 Move Operator diff --git a/ql/src/test/results/clientpositive/vectorized_ptf.q.out b/ql/src/test/results/clientpositive/vectorized_ptf.q.out index 7fdd1d8..22b5621 100644 --- a/ql/src/test/results/clientpositive/vectorized_ptf.q.out +++ b/ql/src/test/results/clientpositive/vectorized_ptf.q.out @@ -239,7 +239,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -292,7 +292,9 @@ STAGE PLANS: /part_orc [part_orc] Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -678,7 +680,7 @@ STAGE PLANS: Map-reduce partition columns: _col2 (type: string) Statistics: Num rows: 14 Data size: 8823 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) + value expressions: _col0 (type: int), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -707,7 +709,9 @@ STAGE PLANS: #### A masked pattern was here #### Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8 Statistics: Num rows: 14 Data size: 8823 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 14 Data size: 8823 Basic stats: COMPLETE Column stats: NONE @@ -922,7 +926,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -975,7 +979,9 @@ STAGE PLANS: /part_orc [part_orc] Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -1173,7 +1179,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -1226,7 +1232,9 @@ STAGE PLANS: /part_orc [part_orc] Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -1504,7 +1512,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -1557,7 +1565,9 @@ STAGE PLANS: /part_orc [part_orc] Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -1845,7 +1855,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -1898,7 +1908,9 @@ STAGE PLANS: /part_orc [part_orc] Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -2198,7 +2210,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -2251,7 +2263,9 @@ STAGE PLANS: /part_orc [part_orc] Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -2545,7 +2559,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -2598,7 +2612,9 @@ STAGE PLANS: /part_orc [part_orc] Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -2903,7 +2919,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -2956,7 +2972,9 @@ STAGE PLANS: /part_orc [part_orc] Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), KEY.reducesinkkey2 (type: int), VALUE._col3 (type: string), VALUE._col4 (type: double), VALUE._col5 (type: string), VALUE._col6 (type: bigint), VALUE._col7 (type: string), VALUE._col8 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -3222,7 +3240,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -3275,7 +3293,9 @@ STAGE PLANS: /part_orc [part_orc] Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -3544,7 +3564,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -3597,7 +3617,9 @@ STAGE PLANS: /part_orc [part_orc] Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -3876,7 +3898,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -3929,7 +3951,9 @@ STAGE PLANS: /part_orc [part_orc] Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -3964,7 +3988,7 @@ STAGE PLANS: Map-reduce partition columns: _col2 (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) + value expressions: _col0 (type: int), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -3993,7 +4017,9 @@ STAGE PLANS: #### A masked pattern was here #### Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -4292,7 +4318,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -4345,7 +4371,9 @@ STAGE PLANS: /part_orc [sub1:part_orc] Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -4721,7 +4749,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -4774,7 +4802,9 @@ STAGE PLANS: /part_orc [part_orc] Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -5130,7 +5160,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -5183,7 +5213,9 @@ STAGE PLANS: /part_orc [part_orc] Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -5541,7 +5573,7 @@ STAGE PLANS: Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 13 Data size: 8021 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: double) + value expressions: _col1 (type: string), _col2 (type: double) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -5570,7 +5602,9 @@ STAGE PLANS: #### A masked pattern was here #### Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string), VALUE._col1 (type: double) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 13 Data size: 8021 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 13 Data size: 8021 Basic stats: COMPLETE Column stats: NONE @@ -6001,7 +6035,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -6054,7 +6088,9 @@ STAGE PLANS: /part_orc [part_orc] Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -6647,7 +6683,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_name (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -6700,7 +6736,9 @@ STAGE PLANS: /part_orc [part_orc] Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: string), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: double), VALUE._col7 (type: string), VALUE._col8 (type: bigint), VALUE._col9 (type: string), VALUE._col10 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -6735,7 +6773,7 @@ STAGE PLANS: Map-reduce partition columns: _col2 (type: string), _col1 (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) + value expressions: _col0 (type: int), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -6764,7 +6802,9 @@ STAGE PLANS: #### A masked pattern was here #### Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -7094,7 +7134,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_name (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -7147,7 +7187,9 @@ STAGE PLANS: /part_orc [part_orc] Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: string), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: double), VALUE._col7 (type: string), VALUE._col8 (type: bigint), VALUE._col9 (type: string), VALUE._col10 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -7180,7 +7222,7 @@ STAGE PLANS: Map-reduce partition columns: _col2 (type: string), _col1 (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) + value expressions: _col0 (type: int), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -7209,7 +7251,9 @@ STAGE PLANS: #### A masked pattern was here #### Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -7242,7 +7286,7 @@ STAGE PLANS: Map-reduce partition columns: _col2 (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) + value expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -7271,7 +7315,9 @@ STAGE PLANS: #### A masked pattern was here #### Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: string), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: double), VALUE._col7 (type: string), VALUE._col8 (type: bigint), VALUE._col9 (type: string), VALUE._col10 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -7584,7 +7630,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string), p_name (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -7637,7 +7683,9 @@ STAGE PLANS: /part_orc [part_orc] Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -7670,7 +7718,7 @@ STAGE PLANS: Map-reduce partition columns: _col2 (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) + value expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -7699,7 +7747,9 @@ STAGE PLANS: #### A masked pattern was here #### Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: string), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: double), VALUE._col7 (type: string), VALUE._col8 (type: bigint), VALUE._col9 (type: string), VALUE._col10 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -8024,7 +8074,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string), p_name (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -8077,7 +8127,9 @@ STAGE PLANS: /part_orc [part_orc] Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -8110,7 +8162,7 @@ STAGE PLANS: Map-reduce partition columns: _col2 (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) + value expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -8139,7 +8191,9 @@ STAGE PLANS: #### A masked pattern was here #### Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: string), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: double), VALUE._col7 (type: string), VALUE._col8 (type: bigint), VALUE._col9 (type: string), VALUE._col10 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -8174,7 +8228,7 @@ STAGE PLANS: Map-reduce partition columns: _col2 (type: string), _col1 (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) + value expressions: _col0 (type: int), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -8203,7 +8257,9 @@ STAGE PLANS: #### A masked pattern was here #### Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -8559,7 +8615,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string), p_name (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -8612,7 +8668,9 @@ STAGE PLANS: /part_orc [part_orc] Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -8647,7 +8705,7 @@ STAGE PLANS: Map-reduce partition columns: _col2 (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) + value expressions: _col0 (type: int), _col1 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -8676,7 +8734,9 @@ STAGE PLANS: #### A masked pattern was here #### Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: string), VALUE._col4 (type: int), VALUE._col5 (type: string), VALUE._col6 (type: double), VALUE._col7 (type: string), VALUE._col8 (type: bigint), VALUE._col9 (type: string), VALUE._col10 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -8998,7 +9058,7 @@ STAGE PLANS: Map-reduce partition columns: p_mfgr (type: string), p_name (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) + value expressions: p_partkey (type: int), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string), BLOCK__OFFSET__INSIDE__FILE (type: bigint), INPUT__FILE__NAME (type: string), ROW__ID (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -9051,7 +9111,9 @@ STAGE PLANS: /part_orc [part_orc] Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE @@ -9086,7 +9148,7 @@ STAGE PLANS: Map-reduce partition columns: _col2 (type: string), _col1 (type: string) Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) + value expressions: _col0 (type: int), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: bigint), _col10 (type: string), _col11 (type: struct) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -9115,7 +9177,9 @@ STAGE PLANS: #### A masked pattern was here #### Needs Tagging: false Reduce Operator Tree: - Extract + Select Operator + expressions: VALUE._col0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: double), VALUE._col6 (type: string), VALUE._col7 (type: bigint), VALUE._col8 (type: string), VALUE._col9 (type: struct) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE PTF Operator Statistics: Num rows: 26 Data size: 16042 Basic stats: COMPLETE Column stats: NONE