You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by xu...@apache.org on 2014/12/10 04:45:31 UTC
svn commit: r1644328 - in /hive/branches/spark: itests/src/test/resources/
ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/
ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/
ql/src/java/org/apache/hadoop/hive/ql/plan/ ql/src/test/queries/...
Author: xuefu
Date: Wed Dec 10 03:45:31 2014
New Revision: 1644328
URL: http://svn.apache.org/r1644328
Log:
HIVE-9042: Support multiple mapjoin operators in one work [Spark Branch] (Jimmy via Xuefu)
Added:
hive/branches/spark/ql/src/test/queries/clientpositive/bucket_map_join_spark4.q
hive/branches/spark/ql/src/test/results/clientpositive/bucket_map_join_spark4.q.out
hive/branches/spark/ql/src/test/results/clientpositive/spark/bucket_map_join_spark4.q.out
Modified:
hive/branches/spark/itests/src/test/resources/testconfiguration.properties
hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkMapJoinResolver.java
hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkMapJoinOptimizer.java
hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/plan/SparkBucketMapJoinContext.java
Modified: hive/branches/spark/itests/src/test/resources/testconfiguration.properties
URL: http://svn.apache.org/viewvc/hive/branches/spark/itests/src/test/resources/testconfiguration.properties?rev=1644328&r1=1644327&r2=1644328&view=diff
==============================================================================
--- hive/branches/spark/itests/src/test/resources/testconfiguration.properties (original)
+++ hive/branches/spark/itests/src/test/resources/testconfiguration.properties Wed Dec 10 03:45:31 2014
@@ -539,6 +539,7 @@ spark.query.files=add_part_multiple.q, \
bucket_map_join_spark1.q \
bucket_map_join_spark2.q \
bucket_map_join_spark3.q \
+ bucket_map_join_spark4.q \
bucket_map_join_tez1.q, \
bucket_map_join_tez2.q, \
column_access_stats.q, \
Modified: hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkMapJoinResolver.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkMapJoinResolver.java?rev=1644328&r1=1644327&r2=1644328&view=diff
==============================================================================
--- hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkMapJoinResolver.java (original)
+++ hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkMapJoinResolver.java Wed Dec 10 03:45:31 2014
@@ -43,11 +43,13 @@ import org.apache.hadoop.hive.ql.lib.Nod
import org.apache.hadoop.hive.ql.lib.TaskGraphWalker;
import org.apache.hadoop.hive.ql.parse.SemanticException;
import org.apache.hadoop.hive.ql.plan.BaseWork;
+import org.apache.hadoop.hive.ql.plan.BucketMapJoinContext;
import org.apache.hadoop.hive.ql.plan.MapJoinDesc;
import org.apache.hadoop.hive.ql.plan.MapWork;
import org.apache.hadoop.hive.ql.plan.MapredLocalWork;
import org.apache.hadoop.hive.ql.plan.OperatorDesc;
import org.apache.hadoop.hive.ql.plan.SparkBucketMapJoinContext;
+import org.apache.hadoop.hive.ql.plan.SparkHashTableSinkDesc;
import org.apache.hadoop.hive.ql.plan.SparkWork;
public class SparkMapJoinResolver implements PhysicalPlanResolver {
@@ -67,10 +69,11 @@ public class SparkMapJoinResolver implem
// Check whether the specified BaseWork's operator tree contains a operator
// of the specified operator class
private boolean containsOp(BaseWork work, Class<?> clazz) {
- return getOp(work, clazz) != null;
+ Set<Operator<? extends OperatorDesc>> matchingOps = getOp(work, clazz);
+ return matchingOps != null && !matchingOps.isEmpty();
}
- private Operator<? extends OperatorDesc> getOp(BaseWork work, Class<?> clazz) {
+ private Set<Operator<? extends OperatorDesc>> getOp(BaseWork work, Class<?> clazz) {
Set<Operator<? extends OperatorDesc>> ops = new HashSet<Operator<? extends OperatorDesc>>();
if (work instanceof MapWork) {
Collection<Operator<?>> opSet = ((MapWork) work).getAliasToWork().values();
@@ -88,12 +91,14 @@ public class SparkMapJoinResolver implem
ops.addAll(work.getAllOperators());
}
+ Set<Operator<? extends OperatorDesc>> matchingOps =
+ new HashSet<Operator<? extends OperatorDesc>>();
for (Operator<? extends OperatorDesc> op : ops) {
if (clazz.isInstance(op)) {
- return op;
+ matchingOps.add(op);
}
}
- return null;
+ return matchingOps;
}
@SuppressWarnings("unchecked")
@@ -174,50 +179,64 @@ public class SparkMapJoinResolver implem
Context ctx = physicalContext.getContext();
for (BaseWork work : allBaseWorks) {
- Operator<? extends OperatorDesc> op = getOp(work, MapJoinOperator.class);
- if (op != null) {
+ Set<Operator<? extends OperatorDesc>> ops = getOp(work, MapJoinOperator.class);
+ if (ops == null || ops.isEmpty()) {
+ continue;
+ }
+ Path tmpPath = Utilities.generateTmpPath(ctx.getMRTmpPath(), originalTask.getId());
+ MapredLocalWork bigTableLocalWork = work.getMapRedLocalWork();
+ List<Operator<? extends OperatorDesc>> dummyOps =
+ new ArrayList<Operator<? extends OperatorDesc>>(work.getDummyOps());
+ bigTableLocalWork.setDummyParentOp(dummyOps);
+ bigTableLocalWork.setTmpPath(tmpPath);
+
+ // In one work, only one map join operator can be bucketed
+ SparkBucketMapJoinContext bucketMJCxt = null;
+ for (Operator<? extends OperatorDesc> op: ops) {
MapJoinOperator mapJoinOp = (MapJoinOperator) op;
- Path tmpPath = Utilities.generateTmpPath(ctx.getMRTmpPath(), originalTask.getId());
- MapredLocalWork bigTableLocalWork = work.getMapRedLocalWork();
- List<Operator<? extends OperatorDesc>> dummyOps =
- new ArrayList<Operator<? extends OperatorDesc>>(work.getDummyOps());
- bigTableLocalWork.setDummyParentOp(dummyOps);
-
- SparkBucketMapJoinContext bucketMJCxt = null;
MapJoinDesc mapJoinDesc = mapJoinOp.getConf();
if (mapJoinDesc.isBucketMapJoin()) {
- bucketMJCxt = new SparkBucketMapJoinContext();
- bigTableLocalWork.setBucketMapjoinContext(bucketMJCxt);
- bucketMJCxt.setAliasBucketFileNameMapping(
- mapJoinDesc.getAliasBucketFileNameMapping());
- bucketMJCxt.setBucketFileNameMapping(
- mapJoinDesc.getBigTableBucketNumMapping());
- bucketMJCxt.setMapJoinBigTableAlias(mapJoinDesc.getBigTableAlias());
+ bucketMJCxt = new SparkBucketMapJoinContext(mapJoinDesc);
bucketMJCxt.setBucketMatcherClass(
org.apache.hadoop.hive.ql.exec.DefaultBucketMatcher.class);
- bucketMJCxt.setBigTablePartSpecToFileMapping(
- mapJoinDesc.getBigTablePartSpecToFileMapping());
bucketMJCxt.setPosToAliasMap(mapJoinOp.getPosToAliasMap());
((MapWork) work).setUseBucketizedHiveInputFormat(true);
+ bigTableLocalWork.setBucketMapjoinContext(bucketMJCxt);
bigTableLocalWork.setInputFileChangeSensitive(true);
+ break;
}
+ }
- for (BaseWork parentWork : originalWork.getParents(work)) {
- if (containsOp(parentWork,SparkHashTableSinkOperator.class)) {
- MapredLocalWork parentLocalWork = parentWork.getMapRedLocalWork();
- parentLocalWork.setTmpHDFSPath(tmpPath);
- if (bucketMJCxt != null) {
+ for (BaseWork parentWork : originalWork.getParents(work)) {
+ Set<Operator<? extends OperatorDesc>> hashTableSinkOps =
+ getOp(parentWork, SparkHashTableSinkOperator.class);
+ if (hashTableSinkOps == null || hashTableSinkOps.isEmpty()) {
+ continue;
+ }
+ MapredLocalWork parentLocalWork = parentWork.getMapRedLocalWork();
+ parentLocalWork.setTmpHDFSPath(tmpPath);
+ if (bucketMJCxt != null) {
+ // We only need to update the work with the hashtable
+ // sink operator with the same mapjoin desc. We can tell
+ // that by comparing the bucket file name mapping map
+ // instance. They should be exactly the same one due to
+ // the way how the bucket mapjoin context is constructed.
+ for (Operator<? extends OperatorDesc> op: hashTableSinkOps) {
+ SparkHashTableSinkOperator hashTableSinkOp = (SparkHashTableSinkOperator) op;
+ SparkHashTableSinkDesc hashTableSinkDesc = hashTableSinkOp.getConf();
+ BucketMapJoinContext original = hashTableSinkDesc.getBucketMapjoinContext();
+ if (original != null && original.getBucketFileNameMapping()
+ == bucketMJCxt.getBucketFileNameMapping()) {
((MapWork) parentWork).setUseBucketizedHiveInputFormat(true);
parentLocalWork.setBucketMapjoinContext(bucketMJCxt);
parentLocalWork.setInputFileChangeSensitive(true);
+ break;
}
}
}
-
- bigTableLocalWork.setTmpPath(tmpPath);
- // TODO: set inputFileChangeSensitive and BucketMapjoinContext,
- // TODO: enable non-staged mapjoin
}
+
+ // TODO: enable non-staged mapjoin
}
}
Modified: hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkMapJoinOptimizer.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkMapJoinOptimizer.java?rev=1644328&r1=1644327&r2=1644328&view=diff
==============================================================================
--- hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkMapJoinOptimizer.java (original)
+++ hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkMapJoinOptimizer.java Wed Dec 10 03:45:31 2014
@@ -147,8 +147,11 @@ public class SparkMapJoinOptimizer imple
numBuckets = convertJoinBucketMapJoin(joinOp, mapJoinOp,
context, mapJoinConversionPos);
if (numBuckets > 1) {
+ LOG.info("Converted to map join with " + numBuckets + " buckets");
bucketColNames = joinOp.getOpTraits().getBucketColNames();
mapJoinInfo[2] /= numBuckets;
+ } else {
+ LOG.info("Can not convert to bucketed map join");
}
}
Modified: hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/plan/SparkBucketMapJoinContext.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/plan/SparkBucketMapJoinContext.java?rev=1644328&r1=1644327&r2=1644328&view=diff
==============================================================================
--- hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/plan/SparkBucketMapJoinContext.java (original)
+++ hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/plan/SparkBucketMapJoinContext.java Wed Dec 10 03:45:31 2014
@@ -25,6 +25,10 @@ public class SparkBucketMapJoinContext e
private Map<Integer, Set<String>> posToAliasMap;
+ public SparkBucketMapJoinContext(MapJoinDesc clone) {
+ super(clone);
+ }
+
public void setPosToAliasMap(Map<Integer, Set<String>> posToAliasMap) {
this.posToAliasMap = posToAliasMap;
}
Added: hive/branches/spark/ql/src/test/queries/clientpositive/bucket_map_join_spark4.q
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/test/queries/clientpositive/bucket_map_join_spark4.q?rev=1644328&view=auto
==============================================================================
--- hive/branches/spark/ql/src/test/queries/clientpositive/bucket_map_join_spark4.q (added)
+++ hive/branches/spark/ql/src/test/queries/clientpositive/bucket_map_join_spark4.q Wed Dec 10 03:45:31 2014
@@ -0,0 +1,42 @@
+set hive.enforce.bucketing = true;
+set hive.enforce.sorting = true;
+set hive.exec.reducers.max = 1;
+
+-- SORT_QUERY_RESULTS
+
+CREATE TABLE tbl1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS;
+CREATE TABLE tbl2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS;
+CREATE TABLE tbl3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS;
+
+insert overwrite table tbl1
+select * from src where key < 10;
+
+insert overwrite table tbl2
+select * from src where key < 10;
+
+insert overwrite table tbl3
+select * from src where key < 10;
+
+set hive.enforce.bucketing = false;
+set hive.enforce.sorting = false;
+set hive.exec.reducers.max = 100;
+
+set hive.auto.convert.join=true;
+
+set hive.optimize.bucketmapjoin = true;
+
+explain extended
+select a.key as key, a.value as val1, b.value as val2, c.value as val3
+from tbl1 a join tbl2 b on a.key = b.key join tbl3 c on a.value = c.value;
+
+select a.key as key, a.value as val1, b.value as val2, c.value as val3
+from tbl1 a join tbl2 b on a.key = b.key join tbl3 c on a.value = c.value;
+
+set hive.optimize.bucketmapjoin = false;
+
+explain extended
+select a.key as key, a.value as val1, b.value as val2, c.value as val3
+from tbl1 a join tbl2 b on a.key = b.key join tbl3 c on a.value = c.value;
+
+select a.key as key, a.value as val1, b.value as val2, c.value as val3
+from tbl1 a join tbl2 b on a.key = b.key join tbl3 c on a.value = c.value;
Added: hive/branches/spark/ql/src/test/results/clientpositive/bucket_map_join_spark4.q.out
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/test/results/clientpositive/bucket_map_join_spark4.q.out?rev=1644328&view=auto
==============================================================================
--- hive/branches/spark/ql/src/test/results/clientpositive/bucket_map_join_spark4.q.out (added)
+++ hive/branches/spark/ql/src/test/results/clientpositive/bucket_map_join_spark4.q.out Wed Dec 10 03:45:31 2014
@@ -0,0 +1,896 @@
+PREHOOK: query: -- SORT_QUERY_RESULTS
+
+CREATE TABLE tbl1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@tbl1
+POSTHOOK: query: -- SORT_QUERY_RESULTS
+
+CREATE TABLE tbl1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@tbl1
+PREHOOK: query: CREATE TABLE tbl2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@tbl2
+POSTHOOK: query: CREATE TABLE tbl2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@tbl2
+PREHOOK: query: CREATE TABLE tbl3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@tbl3
+POSTHOOK: query: CREATE TABLE tbl3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@tbl3
+PREHOOK: query: insert overwrite table tbl1
+select * from src where key < 10
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@tbl1
+POSTHOOK: query: insert overwrite table tbl1
+select * from src where key < 10
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@tbl1
+POSTHOOK: Lineage: tbl1.key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: tbl1.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+PREHOOK: query: insert overwrite table tbl2
+select * from src where key < 10
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@tbl2
+POSTHOOK: query: insert overwrite table tbl2
+select * from src where key < 10
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@tbl2
+POSTHOOK: Lineage: tbl2.key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: tbl2.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+PREHOOK: query: insert overwrite table tbl3
+select * from src where key < 10
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@tbl3
+POSTHOOK: query: insert overwrite table tbl3
+select * from src where key < 10
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@tbl3
+POSTHOOK: Lineage: tbl3.key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: tbl3.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+PREHOOK: query: explain extended
+select a.key as key, a.value as val1, b.value as val2, c.value as val3
+from tbl1 a join tbl2 b on a.key = b.key join tbl3 c on a.value = c.value
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended
+select a.key as key, a.value as val1, b.value as val2, c.value as val3
+from tbl1 a join tbl2 b on a.key = b.key join tbl3 c on a.value = c.value
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+
+TOK_QUERY
+ TOK_FROM
+ TOK_JOIN
+ TOK_JOIN
+ TOK_TABREF
+ TOK_TABNAME
+ tbl1
+ a
+ TOK_TABREF
+ TOK_TABNAME
+ tbl2
+ b
+ =
+ .
+ TOK_TABLE_OR_COL
+ a
+ key
+ .
+ TOK_TABLE_OR_COL
+ b
+ key
+ TOK_TABREF
+ TOK_TABNAME
+ tbl3
+ c
+ =
+ .
+ TOK_TABLE_OR_COL
+ a
+ value
+ .
+ TOK_TABLE_OR_COL
+ c
+ value
+ TOK_INSERT
+ TOK_DESTINATION
+ TOK_DIR
+ TOK_TMP_FILE
+ TOK_SELECT
+ TOK_SELEXPR
+ .
+ TOK_TABLE_OR_COL
+ a
+ key
+ key
+ TOK_SELEXPR
+ .
+ TOK_TABLE_OR_COL
+ a
+ value
+ val1
+ TOK_SELEXPR
+ .
+ TOK_TABLE_OR_COL
+ b
+ value
+ val2
+ TOK_SELEXPR
+ .
+ TOK_TABLE_OR_COL
+ c
+ value
+ val3
+
+
+STAGE DEPENDENCIES:
+ Stage-7 is a root stage
+ Stage-5 depends on stages: Stage-7
+ Stage-0 depends on stages: Stage-5
+
+STAGE PLANS:
+ Stage: Stage-7
+ Map Reduce Local Work
+ Alias -> Map Local Tables:
+ a
+ Fetch Operator
+ limit: -1
+ c
+ Fetch Operator
+ limit: -1
+ Alias -> Map Local Operator Tree:
+ a
+ TableScan
+ alias: a
+ Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE
+ GatherStats: false
+ Filter Operator
+ isSamplingPred: false
+ predicate: (key is not null and value is not null) (type: boolean)
+ Statistics: Num rows: 3 Data size: 21 Basic stats: COMPLETE Column stats: NONE
+ HashTable Sink Operator
+ condition expressions:
+ 0 {value}
+ 1 {value}
+ keys:
+ 0 key (type: int)
+ 1 key (type: int)
+ Position of Big Table: 1
+ c
+ TableScan
+ alias: c
+ Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE
+ GatherStats: false
+ Filter Operator
+ isSamplingPred: false
+ predicate: value is not null (type: boolean)
+ Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE
+ HashTable Sink Operator
+ condition expressions:
+ 0 {_col0} {_col1} {_col6}
+ 1
+ keys:
+ 0 _col1 (type: string)
+ 1 value (type: string)
+ Position of Big Table: 0
+
+ Stage: Stage-5
+ Map Reduce
+ Map Operator Tree:
+ TableScan
+ alias: b
+ Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE
+ GatherStats: false
+ Filter Operator
+ isSamplingPred: false
+ predicate: key is not null (type: boolean)
+ Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {key} {value}
+ 1 {value}
+ keys:
+ 0 key (type: int)
+ 1 key (type: int)
+ outputColumnNames: _col0, _col1, _col6
+ Position of Big Table: 1
+ Statistics: Num rows: 5 Data size: 38 Basic stats: COMPLETE Column stats: NONE
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {_col0} {_col1} {_col6}
+ 1 {value}
+ keys:
+ 0 _col1 (type: string)
+ 1 value (type: string)
+ outputColumnNames: _col0, _col1, _col6, _col11
+ Position of Big Table: 0
+ Statistics: Num rows: 5 Data size: 41 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: int), _col1 (type: string), _col6 (type: string), _col11 (type: string)
+ outputColumnNames: _col0, _col1, _col2, _col3
+ Statistics: Num rows: 5 Data size: 41 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+#### A masked pattern was here ####
+ NumFilesPerFileSink: 1
+ Statistics: Num rows: 5 Data size: 41 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:
+ columns _col0,_col1,_col2,_col3
+ columns.types int:string:string:string
+ escape.delim \
+ hive.serialization.extend.nesting.levels true
+ serialization.format 1
+ serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ TotalFiles: 1
+ GatherStats: false
+ MultiFileSpray: false
+ Local Work:
+ Map Reduce Local Work
+ Path -> Alias:
+#### A masked pattern was here ####
+ Path -> Partition:
+#### A masked pattern was here ####
+ Partition
+ base file name: tbl1
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ properties:
+ COLUMN_STATS_ACCURATE true
+ 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.tbl1
+ numFiles 2
+ numRows 10
+ rawDataSize 70
+ serialization.ddl struct tbl1 { i32 key, string value}
+ serialization.format 1
+ serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ totalSize 80
+#### A masked pattern was here ####
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ properties:
+ COLUMN_STATS_ACCURATE true
+ 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.tbl1
+ numFiles 2
+ numRows 10
+ rawDataSize 70
+ serialization.ddl struct tbl1 { i32 key, string value}
+ serialization.format 1
+ serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ totalSize 80
+#### A masked pattern was here ####
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ name: default.tbl1
+ name: default.tbl1
+#### A masked pattern was here ####
+ Partition
+ base file name: tbl2
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ properties:
+ COLUMN_STATS_ACCURATE true
+ 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.tbl2
+ numFiles 2
+ numRows 10
+ rawDataSize 70
+ serialization.ddl struct tbl2 { i32 key, string value}
+ serialization.format 1
+ serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ totalSize 80
+#### A masked pattern was here ####
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ properties:
+ COLUMN_STATS_ACCURATE true
+ 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.tbl2
+ numFiles 2
+ numRows 10
+ rawDataSize 70
+ serialization.ddl struct tbl2 { i32 key, string value}
+ serialization.format 1
+ serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ totalSize 80
+#### A masked pattern was here ####
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ name: default.tbl2
+ name: default.tbl2
+#### A masked pattern was here ####
+ Partition
+ base file name: tbl3
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ properties:
+ COLUMN_STATS_ACCURATE true
+ 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.tbl3
+ numFiles 2
+ numRows 10
+ rawDataSize 70
+ serialization.ddl struct tbl3 { i32 key, string value}
+ serialization.format 1
+ serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ totalSize 80
+#### A masked pattern was here ####
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ properties:
+ COLUMN_STATS_ACCURATE true
+ 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.tbl3
+ numFiles 2
+ numRows 10
+ rawDataSize 70
+ serialization.ddl struct tbl3 { i32 key, string value}
+ serialization.format 1
+ serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ totalSize 80
+#### A masked pattern was here ####
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ name: default.tbl3
+ name: default.tbl3
+ Truncated Path -> Alias:
+ /tbl2 [b]
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: select a.key as key, a.value as val1, b.value as val2, c.value as val3
+from tbl1 a join tbl2 b on a.key = b.key join tbl3 c on a.value = c.value
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl1
+PREHOOK: Input: default@tbl2
+PREHOOK: Input: default@tbl3
+#### A masked pattern was here ####
+POSTHOOK: query: select a.key as key, a.value as val1, b.value as val2, c.value as val3
+from tbl1 a join tbl2 b on a.key = b.key join tbl3 c on a.value = c.value
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl1
+POSTHOOK: Input: default@tbl2
+POSTHOOK: Input: default@tbl3
+#### A masked pattern was here ####
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+2 val_2 val_2 val_2
+4 val_4 val_4 val_4
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+8 val_8 val_8 val_8
+9 val_9 val_9 val_9
+PREHOOK: query: explain extended
+select a.key as key, a.value as val1, b.value as val2, c.value as val3
+from tbl1 a join tbl2 b on a.key = b.key join tbl3 c on a.value = c.value
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended
+select a.key as key, a.value as val1, b.value as val2, c.value as val3
+from tbl1 a join tbl2 b on a.key = b.key join tbl3 c on a.value = c.value
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+
+TOK_QUERY
+ TOK_FROM
+ TOK_JOIN
+ TOK_JOIN
+ TOK_TABREF
+ TOK_TABNAME
+ tbl1
+ a
+ TOK_TABREF
+ TOK_TABNAME
+ tbl2
+ b
+ =
+ .
+ TOK_TABLE_OR_COL
+ a
+ key
+ .
+ TOK_TABLE_OR_COL
+ b
+ key
+ TOK_TABREF
+ TOK_TABNAME
+ tbl3
+ c
+ =
+ .
+ TOK_TABLE_OR_COL
+ a
+ value
+ .
+ TOK_TABLE_OR_COL
+ c
+ value
+ TOK_INSERT
+ TOK_DESTINATION
+ TOK_DIR
+ TOK_TMP_FILE
+ TOK_SELECT
+ TOK_SELEXPR
+ .
+ TOK_TABLE_OR_COL
+ a
+ key
+ key
+ TOK_SELEXPR
+ .
+ TOK_TABLE_OR_COL
+ a
+ value
+ val1
+ TOK_SELEXPR
+ .
+ TOK_TABLE_OR_COL
+ b
+ value
+ val2
+ TOK_SELEXPR
+ .
+ TOK_TABLE_OR_COL
+ c
+ value
+ val3
+
+
+STAGE DEPENDENCIES:
+ Stage-7 is a root stage
+ Stage-5 depends on stages: Stage-7
+ Stage-0 depends on stages: Stage-5
+
+STAGE PLANS:
+ Stage: Stage-7
+ Map Reduce Local Work
+ Alias -> Map Local Tables:
+ a
+ Fetch Operator
+ limit: -1
+ c
+ Fetch Operator
+ limit: -1
+ Alias -> Map Local Operator Tree:
+ a
+ TableScan
+ alias: a
+ Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE
+ GatherStats: false
+ Filter Operator
+ isSamplingPred: false
+ predicate: (key is not null and value is not null) (type: boolean)
+ Statistics: Num rows: 3 Data size: 21 Basic stats: COMPLETE Column stats: NONE
+ HashTable Sink Operator
+ condition expressions:
+ 0 {value}
+ 1 {value}
+ keys:
+ 0 key (type: int)
+ 1 key (type: int)
+ Position of Big Table: 1
+ c
+ TableScan
+ alias: c
+ Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE
+ GatherStats: false
+ Filter Operator
+ isSamplingPred: false
+ predicate: value is not null (type: boolean)
+ Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE
+ HashTable Sink Operator
+ condition expressions:
+ 0 {_col0} {_col1} {_col6}
+ 1
+ keys:
+ 0 _col1 (type: string)
+ 1 value (type: string)
+ Position of Big Table: 0
+
+ Stage: Stage-5
+ Map Reduce
+ Map Operator Tree:
+ TableScan
+ alias: b
+ Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE
+ GatherStats: false
+ Filter Operator
+ isSamplingPred: false
+ predicate: key is not null (type: boolean)
+ Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {key} {value}
+ 1 {value}
+ keys:
+ 0 key (type: int)
+ 1 key (type: int)
+ outputColumnNames: _col0, _col1, _col6
+ Position of Big Table: 1
+ Statistics: Num rows: 5 Data size: 38 Basic stats: COMPLETE Column stats: NONE
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {_col0} {_col1} {_col6}
+ 1 {value}
+ keys:
+ 0 _col1 (type: string)
+ 1 value (type: string)
+ outputColumnNames: _col0, _col1, _col6, _col11
+ Position of Big Table: 0
+ Statistics: Num rows: 5 Data size: 41 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: int), _col1 (type: string), _col6 (type: string), _col11 (type: string)
+ outputColumnNames: _col0, _col1, _col2, _col3
+ Statistics: Num rows: 5 Data size: 41 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+#### A masked pattern was here ####
+ NumFilesPerFileSink: 1
+ Statistics: Num rows: 5 Data size: 41 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:
+ columns _col0,_col1,_col2,_col3
+ columns.types int:string:string:string
+ escape.delim \
+ hive.serialization.extend.nesting.levels true
+ serialization.format 1
+ serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ TotalFiles: 1
+ GatherStats: false
+ MultiFileSpray: false
+ Local Work:
+ Map Reduce Local Work
+ Path -> Alias:
+#### A masked pattern was here ####
+ Path -> Partition:
+#### A masked pattern was here ####
+ Partition
+ base file name: tbl1
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ properties:
+ COLUMN_STATS_ACCURATE true
+ 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.tbl1
+ numFiles 2
+ numRows 10
+ rawDataSize 70
+ serialization.ddl struct tbl1 { i32 key, string value}
+ serialization.format 1
+ serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ totalSize 80
+#### A masked pattern was here ####
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ properties:
+ COLUMN_STATS_ACCURATE true
+ 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.tbl1
+ numFiles 2
+ numRows 10
+ rawDataSize 70
+ serialization.ddl struct tbl1 { i32 key, string value}
+ serialization.format 1
+ serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ totalSize 80
+#### A masked pattern was here ####
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ name: default.tbl1
+ name: default.tbl1
+#### A masked pattern was here ####
+ Partition
+ base file name: tbl2
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ properties:
+ COLUMN_STATS_ACCURATE true
+ 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.tbl2
+ numFiles 2
+ numRows 10
+ rawDataSize 70
+ serialization.ddl struct tbl2 { i32 key, string value}
+ serialization.format 1
+ serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ totalSize 80
+#### A masked pattern was here ####
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ properties:
+ COLUMN_STATS_ACCURATE true
+ 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.tbl2
+ numFiles 2
+ numRows 10
+ rawDataSize 70
+ serialization.ddl struct tbl2 { i32 key, string value}
+ serialization.format 1
+ serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ totalSize 80
+#### A masked pattern was here ####
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ name: default.tbl2
+ name: default.tbl2
+#### A masked pattern was here ####
+ Partition
+ base file name: tbl3
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ properties:
+ COLUMN_STATS_ACCURATE true
+ 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.tbl3
+ numFiles 2
+ numRows 10
+ rawDataSize 70
+ serialization.ddl struct tbl3 { i32 key, string value}
+ serialization.format 1
+ serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ totalSize 80
+#### A masked pattern was here ####
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ properties:
+ COLUMN_STATS_ACCURATE true
+ 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.tbl3
+ numFiles 2
+ numRows 10
+ rawDataSize 70
+ serialization.ddl struct tbl3 { i32 key, string value}
+ serialization.format 1
+ serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ totalSize 80
+#### A masked pattern was here ####
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ name: default.tbl3
+ name: default.tbl3
+ Truncated Path -> Alias:
+ /tbl2 [b]
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: select a.key as key, a.value as val1, b.value as val2, c.value as val3
+from tbl1 a join tbl2 b on a.key = b.key join tbl3 c on a.value = c.value
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl1
+PREHOOK: Input: default@tbl2
+PREHOOK: Input: default@tbl3
+#### A masked pattern was here ####
+POSTHOOK: query: select a.key as key, a.value as val1, b.value as val2, c.value as val3
+from tbl1 a join tbl2 b on a.key = b.key join tbl3 c on a.value = c.value
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl1
+POSTHOOK: Input: default@tbl2
+POSTHOOK: Input: default@tbl3
+#### A masked pattern was here ####
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+2 val_2 val_2 val_2
+4 val_4 val_4 val_4
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+8 val_8 val_8 val_8
+9 val_9 val_9 val_9
Added: hive/branches/spark/ql/src/test/results/clientpositive/spark/bucket_map_join_spark4.q.out
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/test/results/clientpositive/spark/bucket_map_join_spark4.q.out?rev=1644328&view=auto
==============================================================================
--- hive/branches/spark/ql/src/test/results/clientpositive/spark/bucket_map_join_spark4.q.out (added)
+++ hive/branches/spark/ql/src/test/results/clientpositive/spark/bucket_map_join_spark4.q.out Wed Dec 10 03:45:31 2014
@@ -0,0 +1,884 @@
+PREHOOK: query: -- SORT_QUERY_RESULTS
+
+CREATE TABLE tbl1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@tbl1
+POSTHOOK: query: -- SORT_QUERY_RESULTS
+
+CREATE TABLE tbl1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@tbl1
+PREHOOK: query: CREATE TABLE tbl2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@tbl2
+POSTHOOK: query: CREATE TABLE tbl2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@tbl2
+PREHOOK: query: CREATE TABLE tbl3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@tbl3
+POSTHOOK: query: CREATE TABLE tbl3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@tbl3
+PREHOOK: query: insert overwrite table tbl1
+select * from src where key < 10
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@tbl1
+POSTHOOK: query: insert overwrite table tbl1
+select * from src where key < 10
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@tbl1
+POSTHOOK: Lineage: tbl1.key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: tbl1.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+PREHOOK: query: insert overwrite table tbl2
+select * from src where key < 10
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@tbl2
+POSTHOOK: query: insert overwrite table tbl2
+select * from src where key < 10
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@tbl2
+POSTHOOK: Lineage: tbl2.key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: tbl2.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+PREHOOK: query: insert overwrite table tbl3
+select * from src where key < 10
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@tbl3
+POSTHOOK: query: insert overwrite table tbl3
+select * from src where key < 10
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@tbl3
+POSTHOOK: Lineage: tbl3.key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: tbl3.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+PREHOOK: query: explain extended
+select a.key as key, a.value as val1, b.value as val2, c.value as val3
+from tbl1 a join tbl2 b on a.key = b.key join tbl3 c on a.value = c.value
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended
+select a.key as key, a.value as val1, b.value as val2, c.value as val3
+from tbl1 a join tbl2 b on a.key = b.key join tbl3 c on a.value = c.value
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+
+TOK_QUERY
+ TOK_FROM
+ TOK_JOIN
+ TOK_JOIN
+ TOK_TABREF
+ TOK_TABNAME
+ tbl1
+ a
+ TOK_TABREF
+ TOK_TABNAME
+ tbl2
+ b
+ =
+ .
+ TOK_TABLE_OR_COL
+ a
+ key
+ .
+ TOK_TABLE_OR_COL
+ b
+ key
+ TOK_TABREF
+ TOK_TABNAME
+ tbl3
+ c
+ =
+ .
+ TOK_TABLE_OR_COL
+ a
+ value
+ .
+ TOK_TABLE_OR_COL
+ c
+ value
+ TOK_INSERT
+ TOK_DESTINATION
+ TOK_DIR
+ TOK_TMP_FILE
+ TOK_SELECT
+ TOK_SELEXPR
+ .
+ TOK_TABLE_OR_COL
+ a
+ key
+ key
+ TOK_SELEXPR
+ .
+ TOK_TABLE_OR_COL
+ a
+ value
+ val1
+ TOK_SELEXPR
+ .
+ TOK_TABLE_OR_COL
+ b
+ value
+ val2
+ TOK_SELEXPR
+ .
+ TOK_TABLE_OR_COL
+ c
+ value
+ val3
+
+
+STAGE DEPENDENCIES:
+ Stage-2 is a root stage
+ Stage-1 depends on stages: Stage-2
+ Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+ Stage: Stage-2
+ Spark
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: a
+ Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE
+ GatherStats: false
+ Filter Operator
+ isSamplingPred: false
+ predicate: (key is not null and value is not null) (type: boolean)
+ Statistics: Num rows: 3 Data size: 21 Basic stats: COMPLETE Column stats: NONE
+ Spark HashTable Sink Operator
+ condition expressions:
+ 0 {value}
+ 1 {value}
+ keys:
+ 0 key (type: int)
+ 1 key (type: int)
+ Position of Big Table: 1
+ Local Work:
+ Map Reduce Local Work
+ Bucket Mapjoin Context:
+ Alias Bucket File Name Mapping:
+#### A masked pattern was here ####
+ Alias Bucket Output File Name Mapping:
+#### A masked pattern was here ####
+ Path -> Alias:
+#### A masked pattern was here ####
+ Path -> Partition:
+#### A masked pattern was here ####
+ Partition
+ base file name: tbl1
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ properties:
+ COLUMN_STATS_ACCURATE true
+ 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.tbl1
+ numFiles 2
+ numRows 10
+ rawDataSize 70
+ serialization.ddl struct tbl1 { i32 key, string value}
+ serialization.format 1
+ serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ totalSize 80
+#### A masked pattern was here ####
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ properties:
+ COLUMN_STATS_ACCURATE true
+ 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.tbl1
+ numFiles 2
+ numRows 10
+ rawDataSize 70
+ serialization.ddl struct tbl1 { i32 key, string value}
+ serialization.format 1
+ serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ totalSize 80
+#### A masked pattern was here ####
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ name: default.tbl1
+ name: default.tbl1
+ Truncated Path -> Alias:
+ /tbl1 [a]
+ Map 3
+ Map Operator Tree:
+ TableScan
+ alias: c
+ Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE
+ GatherStats: false
+ Filter Operator
+ isSamplingPred: false
+ predicate: value is not null (type: boolean)
+ Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE
+ Spark HashTable Sink Operator
+ condition expressions:
+ 0 {_col0} {_col1} {_col6}
+ 1
+ keys:
+ 0 _col1 (type: string)
+ 1 value (type: string)
+ Position of Big Table: 0
+ Local Work:
+ Map Reduce Local Work
+ Path -> Alias:
+#### A masked pattern was here ####
+ Path -> Partition:
+#### A masked pattern was here ####
+ Partition
+ base file name: tbl3
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ properties:
+ COLUMN_STATS_ACCURATE true
+ 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.tbl3
+ numFiles 2
+ numRows 10
+ rawDataSize 70
+ serialization.ddl struct tbl3 { i32 key, string value}
+ serialization.format 1
+ serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ totalSize 80
+#### A masked pattern was here ####
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ properties:
+ COLUMN_STATS_ACCURATE true
+ 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.tbl3
+ numFiles 2
+ numRows 10
+ rawDataSize 70
+ serialization.ddl struct tbl3 { i32 key, string value}
+ serialization.format 1
+ serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ totalSize 80
+#### A masked pattern was here ####
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ name: default.tbl3
+ name: default.tbl3
+ Truncated Path -> Alias:
+ /tbl3 [c]
+
+ Stage: Stage-1
+ Spark
+#### A masked pattern was here ####
+ Vertices:
+ Map 2
+ Map Operator Tree:
+ TableScan
+ alias: b
+ Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE
+ GatherStats: false
+ Filter Operator
+ isSamplingPred: false
+ predicate: key is not null (type: boolean)
+ Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {key} {value}
+ 1 {value}
+ keys:
+ 0 key (type: int)
+ 1 key (type: int)
+ outputColumnNames: _col0, _col1, _col6
+ input vertices:
+ 0 Map 1
+ Position of Big Table: 1
+ Statistics: Num rows: 5 Data size: 38 Basic stats: COMPLETE Column stats: NONE
+ BucketMapJoin: true
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {_col0} {_col1} {_col6}
+ 1 {value}
+ keys:
+ 0 _col1 (type: string)
+ 1 value (type: string)
+ outputColumnNames: _col0, _col1, _col6, _col11
+ input vertices:
+ 1 Map 3
+ Position of Big Table: 0
+ Statistics: Num rows: 5 Data size: 41 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: int), _col1 (type: string), _col6 (type: string), _col11 (type: string)
+ outputColumnNames: _col0, _col1, _col2, _col3
+ Statistics: Num rows: 5 Data size: 41 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+#### A masked pattern was here ####
+ NumFilesPerFileSink: 1
+ Statistics: Num rows: 5 Data size: 41 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:
+ columns _col0,_col1,_col2,_col3
+ columns.types int:string:string:string
+ escape.delim \
+ hive.serialization.extend.nesting.levels true
+ serialization.format 1
+ serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ TotalFiles: 1
+ GatherStats: false
+ MultiFileSpray: false
+ Local Work:
+ Map Reduce Local Work
+ Bucket Mapjoin Context:
+ Alias Bucket File Name Mapping:
+#### A masked pattern was here ####
+ Alias Bucket Output File Name Mapping:
+#### A masked pattern was here ####
+ Path -> Alias:
+#### A masked pattern was here ####
+ Path -> Partition:
+#### A masked pattern was here ####
+ Partition
+ base file name: tbl2
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ properties:
+ COLUMN_STATS_ACCURATE true
+ 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.tbl2
+ numFiles 2
+ numRows 10
+ rawDataSize 70
+ serialization.ddl struct tbl2 { i32 key, string value}
+ serialization.format 1
+ serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ totalSize 80
+#### A masked pattern was here ####
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ properties:
+ COLUMN_STATS_ACCURATE true
+ 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.tbl2
+ numFiles 2
+ numRows 10
+ rawDataSize 70
+ serialization.ddl struct tbl2 { i32 key, string value}
+ serialization.format 1
+ serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ totalSize 80
+#### A masked pattern was here ####
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ name: default.tbl2
+ name: default.tbl2
+ Truncated Path -> Alias:
+ /tbl2 [b]
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: select a.key as key, a.value as val1, b.value as val2, c.value as val3
+from tbl1 a join tbl2 b on a.key = b.key join tbl3 c on a.value = c.value
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl1
+PREHOOK: Input: default@tbl2
+PREHOOK: Input: default@tbl3
+#### A masked pattern was here ####
+Status: Failed
+POSTHOOK: query: select a.key as key, a.value as val1, b.value as val2, c.value as val3
+from tbl1 a join tbl2 b on a.key = b.key join tbl3 c on a.value = c.value
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl1
+POSTHOOK: Input: default@tbl2
+POSTHOOK: Input: default@tbl3
+#### A masked pattern was here ####
+PREHOOK: query: explain extended
+select a.key as key, a.value as val1, b.value as val2, c.value as val3
+from tbl1 a join tbl2 b on a.key = b.key join tbl3 c on a.value = c.value
+PREHOOK: type: QUERY
+POSTHOOK: query: explain extended
+select a.key as key, a.value as val1, b.value as val2, c.value as val3
+from tbl1 a join tbl2 b on a.key = b.key join tbl3 c on a.value = c.value
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+
+TOK_QUERY
+ TOK_FROM
+ TOK_JOIN
+ TOK_JOIN
+ TOK_TABREF
+ TOK_TABNAME
+ tbl1
+ a
+ TOK_TABREF
+ TOK_TABNAME
+ tbl2
+ b
+ =
+ .
+ TOK_TABLE_OR_COL
+ a
+ key
+ .
+ TOK_TABLE_OR_COL
+ b
+ key
+ TOK_TABREF
+ TOK_TABNAME
+ tbl3
+ c
+ =
+ .
+ TOK_TABLE_OR_COL
+ a
+ value
+ .
+ TOK_TABLE_OR_COL
+ c
+ value
+ TOK_INSERT
+ TOK_DESTINATION
+ TOK_DIR
+ TOK_TMP_FILE
+ TOK_SELECT
+ TOK_SELEXPR
+ .
+ TOK_TABLE_OR_COL
+ a
+ key
+ key
+ TOK_SELEXPR
+ .
+ TOK_TABLE_OR_COL
+ a
+ value
+ val1
+ TOK_SELEXPR
+ .
+ TOK_TABLE_OR_COL
+ b
+ value
+ val2
+ TOK_SELEXPR
+ .
+ TOK_TABLE_OR_COL
+ c
+ value
+ val3
+
+
+STAGE DEPENDENCIES:
+ Stage-2 is a root stage
+ Stage-1 depends on stages: Stage-2
+ Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+ Stage: Stage-2
+ Spark
+#### A masked pattern was here ####
+ Vertices:
+ Map 1
+ Map Operator Tree:
+ TableScan
+ alias: a
+ Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE
+ GatherStats: false
+ Filter Operator
+ isSamplingPred: false
+ predicate: (key is not null and value is not null) (type: boolean)
+ Statistics: Num rows: 3 Data size: 21 Basic stats: COMPLETE Column stats: NONE
+ Spark HashTable Sink Operator
+ condition expressions:
+ 0 {value}
+ 1 {value}
+ keys:
+ 0 key (type: int)
+ 1 key (type: int)
+ Position of Big Table: 1
+ Local Work:
+ Map Reduce Local Work
+ Path -> Alias:
+#### A masked pattern was here ####
+ Path -> Partition:
+#### A masked pattern was here ####
+ Partition
+ base file name: tbl1
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ properties:
+ COLUMN_STATS_ACCURATE true
+ 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.tbl1
+ numFiles 2
+ numRows 10
+ rawDataSize 70
+ serialization.ddl struct tbl1 { i32 key, string value}
+ serialization.format 1
+ serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ totalSize 80
+#### A masked pattern was here ####
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ properties:
+ COLUMN_STATS_ACCURATE true
+ 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.tbl1
+ numFiles 2
+ numRows 10
+ rawDataSize 70
+ serialization.ddl struct tbl1 { i32 key, string value}
+ serialization.format 1
+ serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ totalSize 80
+#### A masked pattern was here ####
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ name: default.tbl1
+ name: default.tbl1
+ Truncated Path -> Alias:
+ /tbl1 [a]
+ Map 3
+ Map Operator Tree:
+ TableScan
+ alias: c
+ Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE
+ GatherStats: false
+ Filter Operator
+ isSamplingPred: false
+ predicate: value is not null (type: boolean)
+ Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE
+ Spark HashTable Sink Operator
+ condition expressions:
+ 0 {_col0} {_col1} {_col6}
+ 1
+ keys:
+ 0 _col1 (type: string)
+ 1 value (type: string)
+ Position of Big Table: 0
+ Local Work:
+ Map Reduce Local Work
+ Path -> Alias:
+#### A masked pattern was here ####
+ Path -> Partition:
+#### A masked pattern was here ####
+ Partition
+ base file name: tbl3
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ properties:
+ COLUMN_STATS_ACCURATE true
+ 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.tbl3
+ numFiles 2
+ numRows 10
+ rawDataSize 70
+ serialization.ddl struct tbl3 { i32 key, string value}
+ serialization.format 1
+ serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ totalSize 80
+#### A masked pattern was here ####
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ properties:
+ COLUMN_STATS_ACCURATE true
+ 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.tbl3
+ numFiles 2
+ numRows 10
+ rawDataSize 70
+ serialization.ddl struct tbl3 { i32 key, string value}
+ serialization.format 1
+ serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ totalSize 80
+#### A masked pattern was here ####
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ name: default.tbl3
+ name: default.tbl3
+ Truncated Path -> Alias:
+ /tbl3 [c]
+
+ Stage: Stage-1
+ Spark
+#### A masked pattern was here ####
+ Vertices:
+ Map 2
+ Map Operator Tree:
+ TableScan
+ alias: b
+ Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: NONE
+ GatherStats: false
+ Filter Operator
+ isSamplingPred: false
+ predicate: key is not null (type: boolean)
+ Statistics: Num rows: 5 Data size: 35 Basic stats: COMPLETE Column stats: NONE
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {key} {value}
+ 1 {value}
+ keys:
+ 0 key (type: int)
+ 1 key (type: int)
+ outputColumnNames: _col0, _col1, _col6
+ input vertices:
+ 0 Map 1
+ Position of Big Table: 1
+ Statistics: Num rows: 5 Data size: 38 Basic stats: COMPLETE Column stats: NONE
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {_col0} {_col1} {_col6}
+ 1 {value}
+ keys:
+ 0 _col1 (type: string)
+ 1 value (type: string)
+ outputColumnNames: _col0, _col1, _col6, _col11
+ input vertices:
+ 1 Map 3
+ Position of Big Table: 0
+ Statistics: Num rows: 5 Data size: 41 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: int), _col1 (type: string), _col6 (type: string), _col11 (type: string)
+ outputColumnNames: _col0, _col1, _col2, _col3
+ Statistics: Num rows: 5 Data size: 41 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+#### A masked pattern was here ####
+ NumFilesPerFileSink: 1
+ Statistics: Num rows: 5 Data size: 41 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:
+ columns _col0,_col1,_col2,_col3
+ columns.types int:string:string:string
+ escape.delim \
+ hive.serialization.extend.nesting.levels true
+ serialization.format 1
+ serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ TotalFiles: 1
+ GatherStats: false
+ MultiFileSpray: false
+ Local Work:
+ Map Reduce Local Work
+ Path -> Alias:
+#### A masked pattern was here ####
+ Path -> Partition:
+#### A masked pattern was here ####
+ Partition
+ base file name: tbl2
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ properties:
+ COLUMN_STATS_ACCURATE true
+ 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.tbl2
+ numFiles 2
+ numRows 10
+ rawDataSize 70
+ serialization.ddl struct tbl2 { i32 key, string value}
+ serialization.format 1
+ serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ totalSize 80
+#### A masked pattern was here ####
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ properties:
+ COLUMN_STATS_ACCURATE true
+ 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.tbl2
+ numFiles 2
+ numRows 10
+ rawDataSize 70
+ serialization.ddl struct tbl2 { i32 key, string value}
+ serialization.format 1
+ serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ totalSize 80
+#### A masked pattern was here ####
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ name: default.tbl2
+ name: default.tbl2
+ Truncated Path -> Alias:
+ /tbl2 [b]
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: select a.key as key, a.value as val1, b.value as val2, c.value as val3
+from tbl1 a join tbl2 b on a.key = b.key join tbl3 c on a.value = c.value
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl1
+PREHOOK: Input: default@tbl2
+PREHOOK: Input: default@tbl3
+#### A masked pattern was here ####
+POSTHOOK: query: select a.key as key, a.value as val1, b.value as val2, c.value as val3
+from tbl1 a join tbl2 b on a.key = b.key join tbl3 c on a.value = c.value
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl1
+POSTHOOK: Input: default@tbl2
+POSTHOOK: Input: default@tbl3
+#### A masked pattern was here ####
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+0 val_0 val_0 val_0
+2 val_2 val_2 val_2
+4 val_4 val_4 val_4
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+5 val_5 val_5 val_5
+8 val_8 val_8 val_8
+9 val_9 val_9 val_9