diff --git ql/if/queryplan.thrift ql/if/queryplan.thrift index eafbe5a..dbf35e6 100644 --- ql/if/queryplan.thrift +++ ql/if/queryplan.thrift @@ -56,6 +56,7 @@ enum OperatorType { PTF, MUX, DEMUX, + MERGEJOIN, } struct Operator { diff --git ql/src/gen/thrift/gen-cpp/queryplan_types.cpp ql/src/gen/thrift/gen-cpp/queryplan_types.cpp index 96dbb29..34e9007 100644 --- ql/src/gen/thrift/gen-cpp/queryplan_types.cpp +++ ql/src/gen/thrift/gen-cpp/queryplan_types.cpp @@ -51,7 +51,8 @@ int _kOperatorTypeValues[] = { OperatorType::HASHTABLEDUMMY, OperatorType::PTF, OperatorType::MUX, - OperatorType::DEMUX + OperatorType::DEMUX, + OperatorType::MERGEJOIN }; const char* _kOperatorTypeNames[] = { "JOIN", @@ -74,9 +75,10 @@ const char* _kOperatorTypeNames[] = { "HASHTABLEDUMMY", "PTF", "MUX", - "DEMUX" + "DEMUX", + "MERGEJOIN" }; -const std::map _OperatorType_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(21, _kOperatorTypeValues, _kOperatorTypeNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL)); +const std::map _OperatorType_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(22, _kOperatorTypeValues, _kOperatorTypeNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL)); int _kTaskTypeValues[] = { TaskType::MAP, diff --git ql/src/gen/thrift/gen-cpp/queryplan_types.h ql/src/gen/thrift/gen-cpp/queryplan_types.h index 634dd55..71b5c88 100644 --- ql/src/gen/thrift/gen-cpp/queryplan_types.h +++ ql/src/gen/thrift/gen-cpp/queryplan_types.h @@ -56,7 +56,8 @@ struct OperatorType { HASHTABLEDUMMY = 17, PTF = 18, MUX = 19, - DEMUX = 20 + DEMUX = 20, + MERGEJOIN = 21 }; }; diff --git ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/OperatorType.java ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/OperatorType.java index aa094ee..b286e06 100644 --- ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/OperatorType.java +++ ql/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/ql/plan/api/OperatorType.java @@ -32,7 +32,8 @@ HASHTABLEDUMMY(17), PTF(18), MUX(19), - DEMUX(20); + DEMUX(20), + MERGEJOIN(21); private final int value; @@ -95,6 +96,8 @@ public static OperatorType findByValue(int value) { return MUX; case 20: return DEMUX; + case 21: + return MERGEJOIN; default: return null; } diff --git ql/src/gen/thrift/gen-php/Types.php ql/src/gen/thrift/gen-php/Types.php index 5164b2c..f97ebf8 100644 --- ql/src/gen/thrift/gen-php/Types.php +++ ql/src/gen/thrift/gen-php/Types.php @@ -56,6 +56,7 @@ final class OperatorType { const PTF = 18; const MUX = 19; const DEMUX = 20; + const MERGEJOIN = 21; static public $__names = array( 0 => 'JOIN', 1 => 'MAPJOIN', @@ -78,6 +79,7 @@ final class OperatorType { 18 => 'PTF', 19 => 'MUX', 20 => 'DEMUX', + 21 => 'MERGEJOIN', ); } diff --git ql/src/gen/thrift/gen-py/queryplan/ttypes.py ql/src/gen/thrift/gen-py/queryplan/ttypes.py index 2a3f745..6c8970c 100644 --- ql/src/gen/thrift/gen-py/queryplan/ttypes.py +++ ql/src/gen/thrift/gen-py/queryplan/ttypes.py @@ -66,6 +66,7 @@ class OperatorType: PTF = 18 MUX = 19 DEMUX = 20 + MERGEJOIN = 21 _VALUES_TO_NAMES = { 0: "JOIN", @@ -89,6 +90,7 @@ class OperatorType: 18: "PTF", 19: "MUX", 20: "DEMUX", + 21: "MERGEJOIN", } _NAMES_TO_VALUES = { @@ -113,6 +115,7 @@ class OperatorType: "PTF": 18, "MUX": 19, "DEMUX": 20, + "MERGEJOIN": 21, } class TaskType: diff --git ql/src/gen/thrift/gen-rb/queryplan_types.rb ql/src/gen/thrift/gen-rb/queryplan_types.rb index 35e1f0f..82ae7a5 100644 --- ql/src/gen/thrift/gen-rb/queryplan_types.rb +++ ql/src/gen/thrift/gen-rb/queryplan_types.rb @@ -42,8 +42,9 @@ module OperatorType PTF = 18 MUX = 19 DEMUX = 20 - VALUE_MAP = {0 => "JOIN", 1 => "MAPJOIN", 2 => "EXTRACT", 3 => "FILTER", 4 => "FORWARD", 5 => "GROUPBY", 6 => "LIMIT", 7 => "SCRIPT", 8 => "SELECT", 9 => "TABLESCAN", 10 => "FILESINK", 11 => "REDUCESINK", 12 => "UNION", 13 => "UDTF", 14 => "LATERALVIEWJOIN", 15 => "LATERALVIEWFORWARD", 16 => "HASHTABLESINK", 17 => "HASHTABLEDUMMY", 18 => "PTF", 19 => "MUX", 20 => "DEMUX"} - VALID_VALUES = Set.new([JOIN, MAPJOIN, EXTRACT, FILTER, FORWARD, GROUPBY, LIMIT, SCRIPT, SELECT, TABLESCAN, FILESINK, REDUCESINK, UNION, UDTF, LATERALVIEWJOIN, LATERALVIEWFORWARD, HASHTABLESINK, HASHTABLEDUMMY, PTF, MUX, DEMUX]).freeze + MERGEJOIN = 21 + VALUE_MAP = {0 => "JOIN", 1 => "MAPJOIN", 2 => "EXTRACT", 3 => "FILTER", 4 => "FORWARD", 5 => "GROUPBY", 6 => "LIMIT", 7 => "SCRIPT", 8 => "SELECT", 9 => "TABLESCAN", 10 => "FILESINK", 11 => "REDUCESINK", 12 => "UNION", 13 => "UDTF", 14 => "LATERALVIEWJOIN", 15 => "LATERALVIEWFORWARD", 16 => "HASHTABLESINK", 17 => "HASHTABLEDUMMY", 18 => "PTF", 19 => "MUX", 20 => "DEMUX", 21 => "MERGEJOIN"} + VALID_VALUES = Set.new([JOIN, MAPJOIN, EXTRACT, FILTER, FORWARD, GROUPBY, LIMIT, SCRIPT, SELECT, TABLESCAN, FILESINK, REDUCESINK, UNION, UDTF, LATERALVIEWJOIN, LATERALVIEWFORWARD, HASHTABLESINK, HASHTABLEDUMMY, PTF, MUX, DEMUX, MERGEJOIN]).freeze end module TaskType diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/CommonJoinOperator.java ql/src/java/org/apache/hadoop/hive/ql/exec/CommonJoinOperator.java index 03194a4..747337d 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/CommonJoinOperator.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/CommonJoinOperator.java @@ -200,15 +200,18 @@ protected void initializeOp(Configuration hconf) throws HiveException { storage = new AbstractRowContainer[tagLen]; numAliases = conf.getExprs().size(); + LOG.info("VIKRAM> num aliases: " + numAliases); joinValues = new List[tagLen]; joinFilters = new List[tagLen]; order = conf.getTagOrder(); + LOG.info("VIKRAM: order size: " + order.length); condn = conf.getConds(); nullsafes = conf.getNullSafes(); noOuterJoin = conf.isNoOuterJoin(); + LOG.info("VIKRAM: no outer join: " + noOuterJoin); totalSz = JoinUtil.populateJoinKeyValue(joinValues, conf.getExprs(), order,NOTSKIPBIGTABLE); @@ -217,6 +220,11 @@ protected void initializeOp(Configuration hconf) throws HiveException { joinFilters = new List[tagLen]; JoinUtil.populateJoinKeyValue(joinFilters, conf.getFilters(),order,NOTSKIPBIGTABLE); + for (List list : joinValues) { + LOG.info("VIKRAM> Join values: " + Arrays.toString(list.toArray())); + } + + LOG.info(Arrays.toString(inputObjInspectors)); joinValuesObjectInspectors = JoinUtil.getObjectInspectorsFromEvaluators(joinValues, inputObjInspectors,NOTSKIPBIGTABLE, tagLen); @@ -225,11 +233,28 @@ protected void initializeOp(Configuration hconf) throws HiveException { joinValuesStandardObjectInspectors = JoinUtil.getStandardObjectInspectors( joinValuesObjectInspectors,NOTSKIPBIGTABLE, tagLen); + LOG.info("Our join values standard object inspectors are : " + + joinValuesStandardObjectInspectors.length); + int count = 0; + for (List listOI : joinValuesStandardObjectInspectors) { + if (listOI != null) { + LOG.info("VIKRAM> list OI: " + listOI.size() + " location: " + count + + " list of OI: " + Arrays.toString(listOI.toArray())); + } else { +// listOI = new ArrayList(); +// joinValuesStandardObjectInspectors[count] = listOI; + LOG.info("VIKRAM> List OI is null for count " + count); + } + count++; + } + filterMaps = conf.getFilterMap(); if (noOuterJoin) { + LOG.info("NO outer join."); rowContainerStandardObjectInspectors = joinValuesStandardObjectInspectors; } else { + LOG.info("VIKRAM: alias = " + alias); List[] rowContainerObjectInspectors = new List[tagLen]; for (Byte alias : order) { ArrayList rcOIs = new ArrayList(); @@ -271,6 +296,8 @@ protected void initializeOp(Configuration hconf) throws HiveException { } dummyObj[pos] = nr; // there should be only 1 dummy object in the RowContainer + LOG.info("VIKRAM: pos = " + pos + " size of rowContainerStandardObjectInspectors: " + + rowContainerStandardObjectInspectors.length); RowContainer> values = JoinUtil.getRowContainer(hconf, rowContainerStandardObjectInspectors[pos], alias, 1, spillTableDesc, conf, !hasFilter(pos), reporter); diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/CommonMergeJoinOperator.java ql/src/java/org/apache/hadoop/hive/ql/exec/CommonMergeJoinOperator.java new file mode 100644 index 0000000..1701e8d --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/exec/CommonMergeJoinOperator.java @@ -0,0 +1,433 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.exec; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.ql.exec.persistence.RowContainer; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.plan.CommonMergeJoinDesc; +import org.apache.hadoop.hive.ql.plan.OperatorDesc; +import org.apache.hadoop.hive.ql.plan.api.OperatorType; +import org.apache.hadoop.io.WritableComparable; +import org.apache.hadoop.io.WritableComparator; + +public class CommonMergeJoinOperator extends AbstractMapJoinOperator implements + Serializable { + + private static final long serialVersionUID = 1L; + JoinOperator joinOp; + private boolean isBigTableWork; + private static final Log LOG = LogFactory.getLog(CommonMergeJoinOperator.class.getName()); + private Map aliasToInputNameMap; + Map> aliasToKeyMap = new HashMap>(); + private static int count = 0; + transient List[] keyWritables; + transient List[] nextKeyWritables; + RowContainer>[] nextGroupStorage; + RowContainer>[] candidateStorage; + + transient String[] tagToAlias; + private transient boolean[] fetchDone; + private transient boolean[] foundNextKeyGroup; + transient boolean firstFetchHappened = false; + transient boolean localWorkInited = false; + transient boolean initDone = false; + transient List otherKey = null; + transient List values = null; + + public CommonMergeJoinOperator() { + this.joinOp = null; + } + + public CommonMergeJoinOperator(JoinOperator joinOp) { + super(); + this.joinOp = joinOp; + } + + @Override + public void initializeOp(Configuration hconf) throws HiveException { + super.initializeOp(hconf); + } + + /* + * (non-Javadoc) + * + * @see org.apache.hadoop.hive.ql.exec.Operator#processOp(java.lang.Object, + * int) this processor has a push-pull model. First call to this method is a + * push but the rest is pulled until we run out of records. + */ + @Override + public void processOp(Object row, int tag) throws HiveException { + + byte alias = (byte) tag; + + // compute keys and values as StandardObjects + List key = mergeJoinComputeKeys(row, alias); + + List value = getFilteredValue(alias, row); + + + //have we reached a new key group? + boolean nextKeyGroup = processKey(alias, key); + if (nextKeyGroup) { + //assert this.nextGroupStorage[alias].size() == 0; + this.nextGroupStorage[alias].addRow(value); + foundNextKeyGroup[tag] = true; + if (tag != posBigTable) { + return; + } + } + + reportProgress(); + numMapRowsRead++; + + // the big table has reached a new key group. try to let the small tables + // catch up with the big table. + if (nextKeyGroup) { + assert tag == posBigTable; + List smallestPos = null; + do { + smallestPos = joinOneGroup(); + //jump out the loop if we need input from the big table + } while (smallestPos != null && smallestPos.size() > 0 + && !smallestPos.contains(this.posBigTable)); + + return; + } + + assert !nextKeyGroup; + candidateStorage[tag].addRow(value); + + } + + private List joinOneGroup() throws HiveException { + int[] smallestPos = findSmallestKey(); + List listOfNeedFetchNext = null; + if (smallestPos != null) { + listOfNeedFetchNext = joinObject(smallestPos); + if (listOfNeedFetchNext.size() > 0) { + // listOfNeedFetchNext contains all tables that we have joined data in their + // candidateStorage, and we need to clear candidate storage and promote their + // nextGroupStorage to candidateStorage and fetch data until we reach a + // new group. + for (Byte b : listOfNeedFetchNext) { + try { + fetchNextGroup(b); + } catch (Exception e) { + throw new HiveException(e); + } + } + } + } + return listOfNeedFetchNext; + } + + private List joinObject(int[] smallestPos) throws HiveException { + List needFetchList = new ArrayList(); + byte index = (byte) (smallestPos.length - 1); + for (; index >= 0; index--) { + if (smallestPos[index] > 0 || keyWritables[index] == null) { + putDummyOrEmpty(index); + continue; + } + storage[index] = candidateStorage[index]; + needFetchList.add(index); + if (smallestPos[index] < 0) { + break; + } + } + for (index--; index >= 0; index--) { + putDummyOrEmpty(index); + } + checkAndGenObject(); + for (Byte pos : needFetchList) { + this.candidateStorage[pos].clearRows(); + this.keyWritables[pos] = null; + } + return needFetchList; + } + + private void putDummyOrEmpty(Byte i) { + // put a empty list or null + if (noOuterJoin) { + storage[i] = emptyList; + } else { + storage[i] = dummyObjVectors[i]; + } + } + + private int[] findSmallestKey() { + int[] result = new int[order.length]; + List smallestOne = null; + + for (byte pos = 0; pos < order.length; pos++) { + List key = keyWritables[pos]; + if (key == null) { + continue; + } + if (smallestOne == null) { + smallestOne = key; + result[pos] = -1; + continue; + } + result[pos] = compareKeys(key, smallestOne); + if (result[pos] < 0) { + smallestOne = key; + } + } + return smallestOne == null ? null : result; + } + + private void fetchNextGroup(Byte t) throws Exception { + if (foundNextKeyGroup[t]) { + // first promote the next group to be the current group if we reached a + // new group in the previous fetch + if (this.nextKeyWritables[t] != null) { + promoteNextGroupToCandidate(t); + } else { + this.keyWritables[t] = null; + this.candidateStorage[t] = null; + this.nextGroupStorage[t] = null; + } + foundNextKeyGroup[t] = false; + } + // for the big table, we only need to promote the next group to the current group. + if (t == posBigTable) { + return; + } + + // for tables other than the big table, we need to fetch more data until reach a new group or + // done. + while (!foundNextKeyGroup[t]) { + if (fetchDone[t]) { + break; + } + fetchOneRow(t); + } + if (!foundNextKeyGroup[t] && fetchDone[t]) { + this.nextKeyWritables[t] = null; + } + } + + private void fetchOneRow(byte tag) throws Exception { + Object rowOfSide = getParentOperators().get(tag).getNextRow(aliasToInputNameMap.get(tag)); + while (rowOfSide != null) { + values = getFilteredValue(tag, rowOfSide); + otherKey = mergeJoinComputeKeys(rowOfSide, tag); + // FIXME + // int comparedKey = compareKeys(key, otherKey); + int comparedKey = 0; + if (comparedKey < 0) { + this.storage[alias].clearRows(); + aliasToKeyMap.put(tag, otherKey); + this.storage[tag].addRow(values); + return; + } else if (comparedKey > 0) { + continue; + } + this.storage[tag].addRow(values); + rowOfSide = getParentOperators().get(tag).getNextRow(aliasToInputNameMap.get(tag)); + } + } + + private void promoteNextGroupToCandidate(Byte t) throws HiveException { + this.keyWritables[t] = this.nextKeyWritables[t]; + this.nextKeyWritables[t] = null; + RowContainer> oldRowContainer = this.candidateStorage[t]; + oldRowContainer.clearRows(); + this.candidateStorage[t] = this.nextGroupStorage[t]; + this.nextGroupStorage[t] = oldRowContainer; + } + + private boolean processKey(byte alias, List key) throws HiveException { + List keyWritable = keyWritables[alias]; + if (keyWritable == null) { + // the first group. + keyWritables[alias] = key; + return false; + } else { + int cmp = compareKeys(key, keyWritable); + if (cmp != 0) { + nextKeyWritables[alias] = key; + return true; + } + return false; + } + } + + /* + * this is the algorithm for this join. Once we get a row, we get all rows + * corresponding to the key from the other table(s). Once we have all the + * matching rows, we continue to fetch the rows from the current table until + * we run out of keys that match the current one. Then we call + * checkAndGenObject to produce and forward the joined rows. + */ + /* + @Override + public void processOp(Object row, int tag) throws HiveException { + LOG.info("We are in the processing phase. Tag is " + tag); + LOG.info("Is custom merge? " + getConf().getCustomMerge() + " Big table position? " + + getConf().getBigTablePosition() + " Number of buckets? " + getConf().getNumBuckets()); + // HACK FIXME need to use the algorithm below + if (this.getParentOperators() == null) { + joinOp.setChildOperators(this.getChildOperators()); + joinOp.processOp(row, tag); + } + + alias = (byte) tag; + + List key = mergeJoinComputeKeys(row, alias); + LOG.info("alias is " + alias + " row is null? " + (row == null) + + " this storage for alias is null? " + (storage == null)); + List value = getFilteredValue(alias, row); + this.storage[alias].addRow(value); + + try { + for (byte i = 0; i < this.getParentOperators().size(); i++) { + if (tag == conf.getBigTablePosition()) { + continue; + } + + int keyCompare = compareKeys(key, aliasToKeyMap.get(i)); + if (keyCompare < 0) { + return; + } else if (keyCompare > 0) { + this.storage[i].clearRows(); + } + + // fetch the rows from the other tables having same key as the current + // row being processed. + Object rowOfSide = getParentOperators().get(i).getNextRow(aliasToInputNameMap.get(i)); + while (rowOfSide != null) { + List values = getFilteredValue(i, rowOfSide); + List otherKey = mergeJoinComputeKeys(rowOfSide, i); + int comparedKey = compareKeys(key, otherKey); + if (comparedKey < 0) { + this.storage[alias].clearRows(); + aliasToKeyMap.put(i, otherKey); + this.storage[i].addRow(values); + return; + } else if (comparedKey > 0) { + continue; + } + this.storage[i].addRow(values); + rowOfSide = getParentOperators().get(i).getNextRow(aliasToInputNameMap.get(i)); + } + } + } catch (Exception e) { + throw new HiveException(e); + } + checkAndGenObject(); + } + */ + + private int compareKeys(List k1, List k2) { + int ret = 0; + + // join keys have difference sizes? + ret = k1.size() - k2.size(); + if (ret != 0) { + return ret; + } + + for (int i = 0; i < k1.size(); i++) { + WritableComparable key_1 = (WritableComparable) k1.get(i); + WritableComparable key_2 = (WritableComparable) k2.get(i); + if (key_1 == null && key_2 == null) { + return nullsafes != null && nullsafes[i] ? 0 : -1; // just return k1 is + // smaller than k2 + } else if (key_1 == null) { + return -1; + } else if (key_2 == null) { + return 1; + } + ret = WritableComparator.get(key_1.getClass()).compare(key_1, key_2); + if (ret != 0) { + return ret; + } + } + return ret; + } + + private List mergeJoinComputeKeys(Object row, Byte alias) throws HiveException { + return null; + } + + @Override + public String getName() { + return getOperatorName(); + } + + static public String getOperatorName() { + return "MERGEJOIN"; + } + + @Override + public OperatorType getType() { + return OperatorType.MERGEJOIN; + } + + @Override + public void initializeLocalWork(Configuration hconf) throws HiveException { + Operator parent = null; + if (count != 0) { + throw new HiveException("2nd count"); + } + count++; + for (Operator parentOp : parentOperators) { + if (parentOp != null) { + parent = parentOp; + break; + } + } + + if (parent == null) { + throw new HiveException("No valid parents."); + } + LOG.info("VIKRAM> fetch connect ops."); + List dummyOps = parent.getConnectOps(); + LOG.info("VIKRAM> fetched connect ops"); + for (Operator connectOp : dummyOps) { + parentOperators.add(connectOp); + connectOp.getChildOperators().add(this); + } + super.initializeLocalWork(hconf); + return; + } + + public void setJoinOp(JoinOperator joinOp) { + this.joinOp = joinOp; + } + + public boolean isBigTableWork() { + return isBigTableWork; + } + + public void setIsBigTableWork(boolean bigTableWork) { + this.isBigTableWork = bigTableWork; + } +} diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/JoinUtil.java ql/src/java/org/apache/hadoop/hive/ql/exec/JoinUtil.java index 4013b7f..8bf1777 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/JoinUtil.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/JoinUtil.java @@ -54,8 +54,13 @@ int posBigTableAlias, int tagLen) throws HiveException { List[] result = new List[tagLen]; + LOG.info("VIKRAM> expr entries: " + exprEntries.length + " input obj inspector len: " + + inputObjInspector.length); int iterate = Math.min(exprEntries.length, inputObjInspector.length); for (byte alias = 0; alias < iterate; alias++) { + LOG.info("VIKRAM> alias is " + alias + " pos of big table: " + posBigTableAlias); + LOG.info("VIKRAM> exprEntries[" + alias + "] is " + exprEntries[alias] + + " input obj inspector[" + alias + "] = " + inputObjInspector[alias]); if (alias == (byte) posBigTableAlias || exprEntries[alias] == null || inputObjInspector[alias] == null) { // skip the driver and directly loadable tables @@ -68,6 +73,8 @@ fieldOIList.add(exprList.get(i).initialize(inputObjInspector[alias])); } result[alias] = fieldOIList; + LOG.info("VIKRAM> we just added the field OI list to result: " + alias + + " fieldOI list len: " + fieldOIList.size()); } return result; } @@ -79,12 +86,15 @@ List[] result = new List[tagLen]; for (byte alias = 0; alias < aliasToObjectInspectors.length; alias++) { //get big table + LOG.info("VIKRAM> alias is " + alias + " big table pos: " + posBigTableAlias + + " alias to obj inspectors " + aliasToObjectInspectors[alias]); if(alias == (byte) posBigTableAlias || aliasToObjectInspectors[alias] == null){ //skip the big tables continue; } List oiList = aliasToObjectInspectors[alias]; + LOG.info("VIKRAM> oilist len: " + oiList.size()); ArrayList fieldOIList = new ArrayList( oiList.size()); for (int i = 0; i < oiList.size(); i++) { @@ -106,14 +116,21 @@ public static int populateJoinKeyValue(List[] outMap, Map> inputMap, Byte[] order, int posBigTableAlias) throws HiveException { + try { + throw new Exception("We are here"); + } catch (Exception e) { + e.printStackTrace(); + } int total = 0; for (Entry> e : inputMap.entrySet()) { + LOG.info("VIKRAM> e key is " + e.getKey() + " order is " + order); if (e.getValue() == null) { continue; } Byte key = order == null ? e.getKey() : order[e.getKey()]; List valueFields = new ArrayList(); for (ExprNodeDesc expr : e.getValue()) { + LOG.info("VIKRAM> key is " + key); if (key == (byte) posBigTableAlias) { valueFields.add(null); } else { @@ -247,9 +264,11 @@ protected static boolean hasAnyFiltered(short tag) { public static TableDesc getSpillTableDesc(Byte alias, TableDesc[] spillTableDesc, JoinDesc conf, boolean noFilter) { + LOG.info("VIKRAM> Spill table desc is " + spillTableDesc); if (spillTableDesc == null || spillTableDesc.length == 0) { spillTableDesc = initSpillTables(conf,noFilter); } + LOG.info("VIKRAM> spill table desc inited: " + spillTableDesc[alias]); return spillTableDesc[alias]; } @@ -279,6 +298,7 @@ public static SerDe getSpillSerDe(byte alias, TableDesc[] spillTableDesc, int columnSize = valueCols.size(); StringBuilder colNames = new StringBuilder(); StringBuilder colTypes = new StringBuilder(); + LOG.info("VIKRAM> column size is " + columnSize); if (columnSize <= 0) { continue; } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/MapOperator.java ql/src/java/org/apache/hadoop/hive/ql/exec/MapOperator.java index d5de58e..97e730a 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/MapOperator.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/MapOperator.java @@ -33,8 +33,9 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; -import org.apache.hadoop.hive.ql.io.IOContext; import org.apache.hadoop.hive.ql.exec.mr.ExecMapperContext; +import org.apache.hadoop.hive.ql.exec.tez.MapRecordProcessor; +import org.apache.hadoop.hive.ql.io.IOContext; import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.metadata.VirtualColumn; import org.apache.hadoop.hive.ql.plan.MapWork; @@ -180,7 +181,8 @@ private MapOpCtx initObjectInspector(Configuration hconf, MapInputPath ctx, PartitionDesc pd = ctx.partDesc; TableDesc td = pd.getTableDesc(); - + + LOG.info("VIKRAM: td comparator: " + td.getTableName()); MapOpCtx opCtx = new MapOpCtx(); // Use table properties in case of unpartitioned tables, // and the union of table properties and partition properties, with partition @@ -201,51 +203,56 @@ private MapOpCtx initObjectInspector(Configuration hconf, MapInputPath ctx, (StructObjectInspector) opCtx.deserializer.getObjectInspector(); opCtx.tblRawRowObjectInspector = convertedOI.get(td); + LOG.info("VIKRAM: converted OI found an object? " + convertedOI.get(td)); opCtx.partTblObjectInspectorConverter = ObjectInspectorConverters.getConverter( partRawRowObjectInspector, opCtx.tblRawRowObjectInspector); - + LOG.info("VIKRAM: partTblOI converter - " + opCtx.partTblObjectInspectorConverter.toString()); + // Next check if this table has partitions and if so // get the list of partition names as well as allocate // the serdes for the partition columns String pcols = overlayedProps.getProperty(hive_metastoreConstants.META_TABLE_PARTITION_COLUMNS); - + LOG.info("VIKRAM: pcols is - " + pcols.length()); + if (pcols != null && pcols.length() > 0) { String[] partKeys = pcols.trim().split("/"); String pcolTypes = overlayedProps .getProperty(hive_metastoreConstants.META_TABLE_PARTITION_COLUMN_TYPES); String[] partKeyTypes = pcolTypes.trim().split(":"); - + if (partKeys.length > partKeyTypes.length) { throw new HiveException("Internal error : partKeys length, " +partKeys.length + " greater than partKeyTypes length, " + partKeyTypes.length); } - + List partNames = new ArrayList(partKeys.length); Object[] partValues = new Object[partKeys.length]; List partObjectInspectors = new ArrayList(partKeys.length); - + for (int i = 0; i < partKeys.length; i++) { String key = partKeys[i]; + LOG.info("VIKRAM> part key[" + i + "] = " + key); partNames.add(key); ObjectInspector oi = PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector (TypeInfoFactory.getPrimitiveTypeInfo(partKeyTypes[i])); - + // Partitions do not exist for this table if (partSpec == null) { // for partitionless table, initialize partValue to null partValues[i] = null; } else { - partValues[i] = + partValues[i] = ObjectInspectorConverters. getConverter(PrimitiveObjectInspectorFactory. - javaStringObjectInspector, oi).convert(partSpec.get(key)); + javaStringObjectInspector, oi).convert(partSpec.get(key)); } partObjectInspectors.add(oi); } opCtx.rowWithPart = new Object[] {null, partValues}; opCtx.partObjectInspector = ObjectInspectorFactory .getStandardStructObjectInspector(partNames, partObjectInspectors); + LOG.info("VIKRAM> partNames = " + Arrays.toString(partNames.toArray())); } // The op may not be a TableScan for mapjoins @@ -266,17 +273,26 @@ private MapOpCtx initObjectInspector(Configuration hconf, MapInputPath ctx, } } if (!opCtx.hasVC() && !opCtx.isPartitioned()) { + LOG.info("VIKRAM> opctx has vc or is paritioned is false: " + opCtx.hasVC() + " " + + opCtx.isPartitioned()); + LOG.info("VIKRAM> using the tbl raw row object inspector: " + + opCtx.tblRawRowObjectInspector.getTypeName()); + opCtx.rowObjectInspector = opCtx.tblRawRowObjectInspector; return opCtx; } List inspectors = new ArrayList(); inspectors.add(opCtx.tblRawRowObjectInspector); if (opCtx.isPartitioned()) { + LOG.info("VIKRAM> inspectors added for paritioned case."); inspectors.add(opCtx.partObjectInspector); } if (opCtx.hasVC()) { + LOG.info("VIKRAM> inspectors added for virtucal cols"); inspectors.add(opCtx.vcsObjectInspector); } + LOG.info("VIKRAM> row object inspector creating a union struct inspector using inspectors:" + + inspectors.size()); opCtx.rowObjectInspector = ObjectInspectorFactory.getUnionStructObjectInspector(inspectors); return opCtx; } @@ -340,6 +356,57 @@ private boolean isPartitioned(PartitionDesc pd) { return pd.getPartSpec() != null && !pd.getPartSpec().isEmpty(); } + // merge join related only. FIXME + public void setChildrenSecondary(Configuration hconf) throws HiveException { + + List> children = + new ArrayList>(); + + Map convertedOI = getConvertedOI(hconf); + try { + for (Map.Entry> entry : conf.getPathToAliases().entrySet()) { + String onefile = entry.getKey(); + List aliases = entry.getValue(); + + Path onepath = new Path(onefile); + + PartitionDesc partDesc = conf.getPathToPartitionInfo().get(onefile); + + for (String onealias : aliases) { + Operator op = conf.getAliasToWork().get(onealias); + MapInputPath inp = new MapInputPath(onefile, onealias, op, partDesc); + if (opCtxMap.containsKey(inp)) { + continue; + } + MapOpCtx opCtx = initObjectInspector(hconf, inp, convertedOI); + opCtxMap.put(inp, opCtx); + + op.setParentOperators(new ArrayList>()); + op.getParentOperators().add(this); + LOG.info("MapOPerator: operator is " + op); + // check for the operators who will process rows coming to this Map + // Operator + children.add(op); + childrenOpToOpCtxMap.put(op, opCtx); + LOG.info("dump " + op + " " + + opCtxMap.get(inp).rowObjectInspector.getTypeName()); + current = opCtx; // just need for TestOperators.testMapOperator + } + } + + if (children.size() == 0) { + // didn't find match for input file path in configuration! + // serious problem .. + throw new HiveException("Unable to set up children"); + } + + // we found all the operators that we are supposed to process. + setChildOperators(children); + } catch (Exception e) { + throw new HiveException(e); + } + } + public void setChildren(Configuration hconf) throws HiveException { Path fpath = IOContext.get().getInputPath(); @@ -350,9 +417,16 @@ public void setChildren(Configuration hconf) throws HiveException { new ArrayList>(); Map convertedOI = getConvertedOI(hconf); + LOG.info("We have converted OI: " + convertedOI.size()); + for (Entry entry : convertedOI.entrySet()) { + LOG.info("VIKRAM: tableDesc: " + entry.getKey().getTableName() + + " struct obj inspector of type: " + entry.getValue().getClass().getName()); + } try { for (Map.Entry> entry : conf.getPathToAliases().entrySet()) { + LOG.info("Path to aliases: " + entry.getKey() + " values: " + + Arrays.toString(entry.getValue().toArray())); String onefile = entry.getKey(); List aliases = entry.getValue(); @@ -366,7 +440,7 @@ public void setChildren(Configuration hconf) throws HiveException { for (String onealias : aliases) { Operator op = conf.getAliasToWork().get(onealias); if (LOG.isDebugEnabled()) { - LOG.debug("Adding alias " + onealias + " to work list for file " + LOG.info("Adding alias " + onealias + " to work list for file " + onefile); } MapInputPath inp = new MapInputPath(onefile, onealias, op, partDesc); @@ -421,6 +495,7 @@ public void initializeOp(Configuration hconf) throws HiveException { // children will inherit these HiveConf.setVar(hconf, HiveConf.ConfVars.HIVETABLENAME, mapOpCtx.tableName); HiveConf.setVar(hconf, HiveConf.ConfVars.HIVEPARTITIONNAME, mapOpCtx.partName); + LOG.info("VIKRAM> initializing OIs : " + mapOpCtx.rowObjectInspector); child.initialize(hconf, new ObjectInspector[] {mapOpCtx.rowObjectInspector}); } @@ -625,4 +700,15 @@ public OperatorType getType() { return null; } + @Override + public Object getNextRow(String inputName) throws Exception { + // This map operator definitely belongs to merge work. + Object nextRow = MapRecordProcessor.getNextRow(inputName); + return nextRow; + } + + @Override + public List getConnectOps() { + return MapRecordProcessor.getConnectOps(); + } } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java index db94271..43c8825 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java @@ -146,6 +146,7 @@ public int getNumChild() { /** * Implements the getChildren function for the Node Interface. */ + @Override public ArrayList getChildren() { if (getChildOperators() == null) { @@ -343,6 +344,7 @@ public void initialize(Configuration hconf, ObjectInspector[] inputOIs) if (inputOIs != null) { inputObjInspectors = inputOIs; } + LOG.info("VIKRAM> input obj inspectors: " + Arrays.toString(inputObjInspectors)); // initialize structure to maintain child op info. operator tree changes // while @@ -450,6 +452,8 @@ public void passExecContext(ExecMapperContext execContext) { */ protected void initialize(Configuration hconf, ObjectInspector inputOI, int parentId) throws HiveException { + LOG.info("id is " + id); + LOG.info("name is " + getName()); LOG.info("Initializing child " + id + " " + getName()); // Double the size of the array if needed if (parentId >= inputObjInspectors.length) { @@ -851,6 +855,7 @@ public void logStats() { * * @return the name of the operator */ + @Override public String getName() { return getOperatorName(); } @@ -1061,7 +1066,7 @@ public boolean supportSkewJoinOptimization() { if (parents != null) { for (Operator parent : parents) { - parentClones.add((Operator)(parent.clone())); + parentClones.add((parent.clone())); } } @@ -1082,8 +1087,8 @@ public boolean supportSkewJoinOptimization() { public Operator cloneOp() throws CloneNotSupportedException { T descClone = (T) conf.clone(); Operator ret = - (Operator) OperatorFactory.getAndMakeChild( - descClone, getSchema()); + OperatorFactory.getAndMakeChild( + descClone, getSchema()); return ret; } @@ -1254,15 +1259,15 @@ public Statistics getStatistics() { } return null; } - + public OpTraits getOpTraits() { if (conf != null) { return conf.getOpTraits(); } - + return null; } - + public void setOpTraits(OpTraits metaInfo) { if (LOG.isDebugEnabled()) { LOG.debug("Setting traits ("+metaInfo+") on "+this); @@ -1299,7 +1304,26 @@ public static Operator createDummy() { private static class DummyOperator extends Operator { public DummyOperator() { super("dummy"); } + @Override public void processOp(Object row, int tag) { } + @Override public OperatorType getType() { return null; } } + + public Object getNextRow(String string) throws Exception { + // all operators except the MapOperator just fetch and process the row. + assert (getParentOperators().size() == 1); + Object row = getParentOperators().get(0).getNextRow(string); + return row; + } + + public List getConnectOps() { + if ((parentOperators == null) || (parentOperators.size() == 0)) { + LOG.info("VIKRAM> Parent ops is null? " + parentOperators + " for operator: " + this); + return null; + } + LOG.info("VIKRAM> returning: " + parentOperators.get(0)); + List dummyOps = parentOperators.get(0).getConnectOps(); + return dummyOps; + } } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java index 5d41fa1..1d9bccc 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java @@ -28,11 +28,12 @@ import org.apache.hadoop.hive.ql.exec.vector.VectorLimitOperator; import org.apache.hadoop.hive.ql.exec.vector.VectorMapJoinOperator; import org.apache.hadoop.hive.ql.exec.vector.VectorReduceSinkOperator; -import org.apache.hadoop.hive.ql.exec.vector.VectorSelectOperator; import org.apache.hadoop.hive.ql.exec.vector.VectorSMBMapJoinOperator; +import org.apache.hadoop.hive.ql.exec.vector.VectorSelectOperator; import org.apache.hadoop.hive.ql.exec.vector.VectorizationContext; import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.plan.CollectDesc; +import org.apache.hadoop.hive.ql.plan.CommonMergeJoinDesc; import org.apache.hadoop.hive.ql.plan.DemuxDesc; import org.apache.hadoop.hive.ql.plan.DummyStoreDesc; import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; @@ -114,6 +115,8 @@ public OpTuple(Class descClass, Class> opClass) { DemuxOperator.class)); opvec.add(new OpTuple(MuxDesc.class, MuxOperator.class)); + opvec.add(new OpTuple(CommonMergeJoinDesc.class, + CommonMergeJoinOperator.class)); } public static ArrayList vectorOpvec; diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java index 4450ad3..c8b22b2 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java @@ -197,6 +197,7 @@ public static String HADOOP_LOCAL_FS = "file:///"; public static String MAP_PLAN_NAME = "map.xml"; public static String REDUCE_PLAN_NAME = "reduce.xml"; + public static String MERGE_PLAN_NAME = "merge.xml"; public static final String MAPRED_MAPPER_CLASS = "mapred.mapper.class"; public static final String MAPRED_REDUCER_CLASS = "mapred.reducer.class"; @@ -330,7 +331,7 @@ private static BaseWork getBaseWork(Configuration conf, String name) { } if (HiveConf.getBoolVar(conf, ConfVars.HIVE_RPC_QUERY_PLAN)) { - LOG.debug("Loading plan from string: "+path.toUri().getPath()); + LOG.info("Loading plan from string: " + path.toUri().getPath()); String planString = conf.get(path.toUri().getPath()); if (planString == null) { LOG.info("Could not find plan string in conf"); @@ -343,7 +344,7 @@ private static BaseWork getBaseWork(Configuration conf, String name) { in = new FileInputStream(localPath.toUri().getPath()); } - if(MAP_PLAN_NAME.equals(name)){ + if (MAP_PLAN_NAME.equals(name)) { if (ExecMapper.class.getName().equals(conf.get(MAPRED_MAPPER_CLASS))){ gWork = deserializePlan(in, MapWork.class, conf); } else if(RCFileMergeMapper.class.getName().equals(conf.get(MAPRED_MAPPER_CLASS))) { @@ -363,6 +364,8 @@ private static BaseWork getBaseWork(Configuration conf, String name) { throw new RuntimeException("unable to determine work from configuration ." + MAPRED_REDUCER_CLASS +" was "+ conf.get(MAPRED_REDUCER_CLASS)) ; } + } else if (MERGE_PLAN_NAME.equals(name)) { + gWork = deserializePlan(in, MapWork.class, conf); } gWorkMap.put(path, gWork); } else { @@ -586,14 +589,19 @@ public static void setMapRedWork(Configuration conf, MapredWork w, Path hiveScra } public static Path setMapWork(Configuration conf, MapWork w, Path hiveScratchDir, boolean useCache) { - return setBaseWork(conf, w, hiveScratchDir, MAP_PLAN_NAME, useCache); + List baseWorkList = new ArrayList(); + baseWorkList.add(w); + return setBaseWork(conf, baseWorkList, hiveScratchDir, MAP_PLAN_NAME, useCache); } public static Path setReduceWork(Configuration conf, ReduceWork w, Path hiveScratchDir, boolean useCache) { - return setBaseWork(conf, w, hiveScratchDir, REDUCE_PLAN_NAME, useCache); + List baseWorkList = new ArrayList(); + baseWorkList.add(w); + return setBaseWork(conf, baseWorkList, hiveScratchDir, REDUCE_PLAN_NAME, useCache); } - private static Path setBaseWork(Configuration conf, BaseWork w, Path hiveScratchDir, String name, boolean useCache) { + private static Path setBaseWork(Configuration conf, List workList, Path hiveScratchDir, + String name, boolean useCache) { try { setPlanPath(conf, hiveScratchDir); @@ -605,7 +613,9 @@ private static Path setBaseWork(Configuration conf, BaseWork w, Path hiveScratch // add it to the conf ByteArrayOutputStream byteOut = new ByteArrayOutputStream(); out = new DeflaterOutputStream(byteOut, new Deflater(Deflater.BEST_SPEED)); - serializePlan(w, out, conf); + for (BaseWork w : workList) { + serializePlan(w, out, conf); + } LOG.info("Setting plan: "+planPath.toUri().getPath()); conf.set(planPath.toUri().getPath(), Base64.encodeBase64String(byteOut.toByteArray())); @@ -613,7 +623,9 @@ private static Path setBaseWork(Configuration conf, BaseWork w, Path hiveScratch // use the default file system of the conf FileSystem fs = planPath.getFileSystem(conf); out = fs.create(planPath); - serializePlan(w, out, conf); + for (BaseWork w : workList) { + serializePlan(w, out, conf); + } // Serialize the plan to the default hdfs instance // Except for hadoop local mode execution where we should be @@ -634,7 +646,9 @@ private static Path setBaseWork(Configuration conf, BaseWork w, Path hiveScratch } // Cache the plan in this process - gWorkMap.put(planPath, w); + if (workList.size() == 1) { + gWorkMap.put(planPath, workList.get(0)); + } return planPath; } catch (Exception e) { @@ -3424,7 +3438,7 @@ public static boolean createDirsWithPermission(Configuration conf, Path mkdir, return createDirsWithPermission(conf, mkdir, fsPermission, recursive); } - private static void resetConfAndCloseFS (Configuration conf, boolean unsetUmask, + private static void resetConfAndCloseFS (Configuration conf, boolean unsetUmask, String origUmask, FileSystem fs) throws IOException { if (unsetUmask) { if (origUmask != null) { @@ -3499,4 +3513,13 @@ public static String getQualifiedPath(HiveConf conf, Path path) throws HiveExcep public static boolean isDefaultNameNode(HiveConf conf) { return !conf.getChangedProperties().containsKey(HiveConf.ConfVars.HADOOPFS.varname); } + + public static Path setMergeWork(JobConf conf, List workList, Path mrScratchDir, + boolean b) { + return setBaseWork(conf, workList, mrScratchDir, MERGE_PLAN_NAME, b); + } + + public static MapWork getMergeWork(JobConf jconf) { + return (MapWork) getBaseWork(jconf, MERGE_PLAN_NAME); + } } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/tez/CustomPartitionVertex.java ql/src/java/org/apache/hadoop/hive/ql/exec/tez/CustomPartitionVertex.java index 0aa80f0..07b91e6 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/tez/CustomPartitionVertex.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/tez/CustomPartitionVertex.java @@ -19,7 +19,6 @@ package org.apache.hadoop.hive.ql.exec.tez; import java.io.IOException; -import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; @@ -31,7 +30,9 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.ql.plan.BaseWork.VertexType; import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.hadoop.io.DataInputBuffer; import org.apache.hadoop.io.DataOutputBuffer; import org.apache.hadoop.io.serializer.SerializationFactory; import org.apache.hadoop.mapred.FileSplit; @@ -78,9 +79,10 @@ private List dataInformationEvents; private int numBuckets = -1; private Configuration conf = null; - private boolean rootVertexInitialized = false; private final SplitGrouper grouper = new SplitGrouper(); private int taskCount = 0; + private VertexType vertexType; + private String inputNameDecidingParallelism; public CustomPartitionVertex() { } @@ -88,8 +90,19 @@ public CustomPartitionVertex() { @Override public void initialize(VertexManagerPluginContext context) { this.context = context; - ByteBuffer byteBuf = ByteBuffer.wrap(context.getUserPayload()); - this.numBuckets = byteBuf.getInt(); + DataInputBuffer dib = new DataInputBuffer(); + byte[] payload = context.getUserPayload(); + dib.reset(payload, payload.length); + CustomVertexConfiguration customConf = new CustomVertexConfiguration(); + try { + customConf.readFields(dib); + } catch (IOException e) { + throw new RuntimeException(e); + } + + this.numBuckets = customConf.getNumBuckets(); + this.vertexType = customConf.getVertexType(); + this.inputNameDecidingParallelism = customConf.getInputName(); } @Override @@ -116,12 +129,6 @@ public void onVertexManagerEventReceived(VertexManagerEvent vmEvent) { public void onRootVertexInitialized(String inputName, InputDescriptor inputDescriptor, List events) { - // Ideally, since there's only 1 Input expected at the moment - - // ensure this method is called only once. Tez will call it once per Root - // Input. - Preconditions.checkState(rootVertexInitialized == false); - LOG.info("Root vertex not initialized"); - rootVertexInitialized = true; try { // This is using the payload from the RootVertexInitializer corresponding // to InputName. Ideally it should be using it's own configuration class - @@ -131,6 +138,11 @@ public void onRootVertexInitialized(String inputName, InputDescriptor inputDescr MRHelpers.parseMRInputPayload(inputDescriptor.getUserPayload()); this.conf = MRHelpers.createConfFromByteString(protoPayload.getConfigurationBytes()); + if ((vertexType == VertexType.MULTI_INPUT_INITIALIZED_EDGES) + || (vertexType == VertexType.MULTI_INPUT_UNINITIALIZED_EDGES)) { + //createRoutingTable(); + } + /* * Currently in tez, the flow of events is thus: * "Generate Splits -> Initialize Vertex" (with parallelism info obtained @@ -165,9 +177,6 @@ public void onRootVertexInitialized(String inputName, InputDescriptor inputDescr // No tasks should have been started yet. Checked by initial state // check. Preconditions.checkState(dataInformationEventSeen == false); - Preconditions - .checkState(context.getVertexNumTasks(context.getVertexName()) == -1, - "Parallelism for the vertex should be set to -1 if the InputInitializer is setting parallelism"); RootInputConfigureVertexTasksEvent cEvent = (RootInputConfigureVertexTasksEvent) event; // The vertex cannot be configured until all DataEvents are seen - to @@ -249,11 +258,13 @@ private void processAllEvents(String inputName, // Construct the EdgeManager descriptor to be used by all edges which need // the routing table. - EdgeManagerDescriptor hiveEdgeManagerDesc = - new EdgeManagerDescriptor(CustomPartitionEdge.class.getName()); - byte[] payload = getBytePayload(bucketToTaskMap); - hiveEdgeManagerDesc.setUserPayload(payload); - + EdgeManagerDescriptor hiveEdgeManagerDesc = null; + if ((vertexType == VertexType.MULTI_INPUT_INITIALIZED_EDGES) + || (vertexType == VertexType.INITIALIZED_EDGES)) { + hiveEdgeManagerDesc = new EdgeManagerDescriptor(CustomPartitionEdge.class.getName()); + byte[] payload = getBytePayload(bucketToTaskMap); + hiveEdgeManagerDesc.setUserPayload(payload); + } Map emMap = Maps.newHashMap(); // Replace the edge manager for all vertices which have routing type custom. @@ -286,10 +297,13 @@ private void processAllEvents(String inputName, rootInputSpecUpdate.put( inputName, RootInputSpecUpdate.getDefaultSinglePhysicalInputSpecUpdate()); - context.setVertexParallelism( - taskCount, - new VertexLocationHint(grouper.createTaskLocationHints(finalSplits - .toArray(new InputSplit[finalSplits.size()]))), emMap, rootInputSpecUpdate); + if (inputNameDecidingParallelism.compareTo(inputName) == 0) { + LOG.info("ZZZ: input name deciding parallelism is " + inputName); + context.setVertexParallelism( + taskCount, + new VertexLocationHint(grouper.createTaskLocationHints(finalSplits + .toArray(new InputSplit[finalSplits.size()]))), emMap, rootInputSpecUpdate); + } // Set the actual events for the tasks. context.addRootInputEvents(inputName, taskEvents); @@ -317,7 +331,8 @@ private FileSplit getFileSplitFromEvent(RootInputDataInformationEvent event) thr if (!(inputSplit instanceof FileSplit)) { throw new UnsupportedOperationException( - "Cannot handle splits other than FileSplit for the moment"); + "Cannot handle splits other than FileSplit for the moment. Current input split type: " + + inputSplit.getClass().getSimpleName()); } return (FileSplit) inputSplit; } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/tez/CustomVertexConfiguration.java ql/src/java/org/apache/hadoop/hive/ql/exec/tez/CustomVertexConfiguration.java new file mode 100644 index 0000000..d923fa9 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/exec/tez/CustomVertexConfiguration.java @@ -0,0 +1,50 @@ +package org.apache.hadoop.hive.ql.exec.tez; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + +import org.apache.hadoop.hive.ql.plan.BaseWork.VertexType; +import org.apache.hadoop.io.Writable; + +public class CustomVertexConfiguration implements Writable { + + private int numBuckets; + private VertexType vertexType; + private String inputName; + + public CustomVertexConfiguration() { + } + + public CustomVertexConfiguration(int numBuckets, VertexType vertexType, String inputName) { + this.numBuckets = numBuckets; + this.vertexType = vertexType; + this.inputName = inputName; + } + + @Override + public void write(DataOutput out) throws IOException { + out.writeInt(this.vertexType.ordinal()); + out.writeInt(this.numBuckets); + out.writeUTF(inputName); + } + + @Override + public void readFields(DataInput in) throws IOException { + this.vertexType = VertexType.values()[in.readInt()]; + this.numBuckets = in.readInt(); + this.inputName = in.readUTF(); + } + + public int getNumBuckets() { + return numBuckets; + } + + public VertexType getVertexType() { + return vertexType; + } + + public String getInputName() { + return inputName; + } +} diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/tez/MapRecordProcessor.java ql/src/java/org/apache/hadoop/hive/ql/exec/tez/MapRecordProcessor.java index d964eb1..3e90c00 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/tez/MapRecordProcessor.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/tez/MapRecordProcessor.java @@ -18,6 +18,8 @@ package org.apache.hadoop.hive.ql.exec.tez; import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; import java.util.List; import java.util.Map; import java.util.Map.Entry; @@ -25,6 +27,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.ql.exec.DummyStoreOperator; import org.apache.hadoop.hive.ql.exec.HashTableDummyOperator; import org.apache.hadoop.hive.ql.exec.MapOperator; import org.apache.hadoop.hive.ql.exec.MapredContext; @@ -36,6 +39,7 @@ import org.apache.hadoop.hive.ql.exec.mr.ExecMapper.reportStats; import org.apache.hadoop.hive.ql.exec.mr.ExecMapperContext; import org.apache.hadoop.hive.ql.exec.tez.TezProcessor.TezKVOutputCollector; +import org.apache.hadoop.hive.ql.exec.tez.tools.KeyValueInputMerger; import org.apache.hadoop.hive.ql.exec.vector.VectorMapOperator; import org.apache.hadoop.hive.ql.log.PerfLogger; import org.apache.hadoop.hive.ql.plan.MapWork; @@ -44,6 +48,7 @@ import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.util.StringUtils; import org.apache.tez.mapreduce.input.MRInputLegacy; +import org.apache.tez.mapreduce.input.MultiMRInput; import org.apache.tez.mapreduce.processor.MRTaskReporter; import org.apache.tez.runtime.api.LogicalInput; import org.apache.tez.runtime.api.LogicalOutput; @@ -57,12 +62,15 @@ public class MapRecordProcessor extends RecordProcessor { + private static KeyValueReader mergeQueue; private MapOperator mapOp; + private MapOperator mergeMapOp; public static final Log l4j = LogFactory.getLog(MapRecordProcessor.class); private final ExecMapperContext execContext = new ExecMapperContext(); private boolean abort = false; protected static final String MAP_PLAN_KEY = "__MAP_PLAN__"; private MapWork mapWork; + private static List connectOps = new ArrayList(); @Override void init(JobConf jconf, TezProcessorContext processorContext, MRTaskReporter mrReporter, @@ -90,11 +98,13 @@ void init(JobConf jconf, TezProcessorContext processorContext, MRTaskReporter mr ObjectCache cache = ObjectCacheFactory.getCache(jconf); try { + MapWork mergeMapWork = null; execContext.setJc(jconf); // create map and fetch operators mapWork = (MapWork) cache.retrieve(MAP_PLAN_KEY); if (mapWork == null) { mapWork = Utilities.getMapWork(jconf); + mergeMapWork = Utilities.getMergeWork(jconf); cache.cache(MAP_PLAN_KEY, mapWork); l4j.info("Plan: "+mapWork); for (String s: mapWork.getAliases()) { @@ -109,16 +119,39 @@ void init(JobConf jconf, TezProcessorContext processorContext, MRTaskReporter mr mapOp = new MapOperator(); } + if (mergeMapWork != null) { + if (mergeMapWork.getVectorMode()) { + mergeMapOp = new VectorMapOperator(); + } else { + mergeMapOp = new MapOperator(); + } + } + + // initialize the merge operators first. + if (mergeMapOp != null) { + mergeMapOp.setConf(mergeMapWork); + mergeMapOp.setChildrenSecondary(jconf); + DummyStoreOperator dummyOp = getJoinParentOp(mergeMapOp); + l4j.info("VIKRAM> connect ops added: " + dummyOp); + connectOps.add(dummyOp); + mergeMapOp.setExecContext(execContext); + mergeMapOp.initializeLocalWork(jconf); + } + // initialize map operator mapOp.setConf(mapWork); mapOp.setChildren(jconf); - l4j.info(mapOp.dump(0)); + // l4j.info(mapOp.dump(0)); MapredContext.init(true, new JobConf(jconf)); ((TezContext)MapredContext.get()).setInputs(inputs); mapOp.setExecContext(execContext); mapOp.initializeLocalWork(jconf); + mapOp.initialize(jconf, null); + if (mergeMapOp != null) { + mergeMapOp.initialize(jconf, null); + } // Initialization isn't finished until all parents of all operators // are initialized. For broadcast joins that means initializing the @@ -148,8 +181,20 @@ void init(JobConf jconf, TezProcessorContext processorContext, MRTaskReporter mr perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.TEZ_INIT_OPERATORS); } + private DummyStoreOperator getJoinParentOp(Operator mergeMapOp) { + for (Operator childOp : mergeMapOp.getChildOperators()) { + l4j.info("Child op we are walking: " + childOp); + if ((childOp.getChildOperators() == null) || (childOp.getChildOperators().isEmpty())) { + return (DummyStoreOperator) childOp; + } else { + return getJoinParentOp(childOp); + } + } + return null; + } + @Override - void run() throws IOException{ + void run() throws IOException, InterruptedException { MRInputLegacy in = TezProcessor.getMRInput(inputs); KeyValueReader reader = in.getReader(); @@ -239,4 +284,25 @@ void close(){ MapredContext.close(); } } + + public static Object getNextRow(String inputName) throws Exception { + // get the next row from the priority queue corresponding to this input name + if (mergeQueue == null) { + MultiMRInput multiMRInput = TezProcessor.getInput(inputName); + Collection kvReaders = multiMRInput.getKeyValueReaders(); + List kvReaderList = new ArrayList(kvReaders); + mergeQueue = new KeyValueInputMerger(kvReaderList); + } + + if (mergeQueue.next()) { + return mergeQueue.getCurrentValue(); + } else { + return null; + } + } + + public static List getConnectOps() { + l4j.info("VIKRAM> connect ops length: " + connectOps.size()); + return connectOps; + } } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/tez/ReduceRecordProcessor.java ql/src/java/org/apache/hadoop/hive/ql/exec/tez/ReduceRecordProcessor.java index e884afd..ba8923e 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/tez/ReduceRecordProcessor.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/tez/ReduceRecordProcessor.java @@ -20,7 +20,6 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; -import java.util.Iterator; import java.util.LinkedList; import java.util.List; import java.util.Map; @@ -38,7 +37,7 @@ import org.apache.hadoop.hive.ql.exec.mr.ExecMapper.reportStats; import org.apache.hadoop.hive.ql.exec.mr.ExecMapperContext; import org.apache.hadoop.hive.ql.exec.tez.TezProcessor.TezKVOutputCollector; -import org.apache.hadoop.hive.ql.exec.tez.tools.InputMerger; +import org.apache.hadoop.hive.ql.exec.tez.tools.KeyValuesInputMerger; import org.apache.hadoop.hive.ql.exec.vector.VectorizedBatchUtil; import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch; import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpressionWriter; @@ -59,7 +58,6 @@ import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.DataOutputBuffer; import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.mapred.OutputCollector; import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.util.StringUtils; import org.apache.tez.mapreduce.processor.MRTaskReporter; @@ -120,6 +118,10 @@ void init(JobConf jconf, TezProcessorContext processorContext, MRTaskReporter mr ObjectCache cache = ObjectCacheFactory.getCache(jconf); + for (Entry entry : inputs.entrySet()) { + l4j.info("REDUCER name : " + entry.getKey() + " Logical input type: " + entry.getValue()); + } + rowObjectInspector = new ObjectInspector[Byte.MAX_VALUE]; ObjectInspector[] valueObjectInspector = new ObjectInspector[Byte.MAX_VALUE]; ObjectInspector keyObjectInspector; @@ -140,7 +142,7 @@ void init(JobConf jconf, TezProcessorContext processorContext, MRTaskReporter mr try { keyTableDesc = redWork.getKeyDesc(); - inputKeyDeserializer = (SerDe) ReflectionUtils.newInstance(keyTableDesc + inputKeyDeserializer = ReflectionUtils.newInstance(keyTableDesc .getDeserializerClass(), null); SerDeUtils.initializeSerDe(inputKeyDeserializer, null, keyTableDesc.getProperties(), null); keyObjectInspector = inputKeyDeserializer.getObjectInspector(); @@ -152,7 +154,7 @@ void init(JobConf jconf, TezProcessorContext processorContext, MRTaskReporter mr keyStructInspector = (StructObjectInspector)keyObjectInspector; batches = new VectorizedRowBatch[maxTags]; valueStructInspectors = new StructObjectInspector[maxTags]; - valueStringWriters = (List[])new List[maxTags]; + valueStringWriters = new List[maxTags]; keysColumnOffset = keyStructInspector.getAllStructFieldRefs().size(); buffer = new DataOutputBuffer(); } @@ -279,7 +281,7 @@ void run() throws Exception { kvsReader = (KeyValuesReader) shuffleInputs.get(0).getReader(); }else { //get a sort merged input - kvsReader = new InputMerger(shuffleInputs); + kvsReader = new KeyValuesInputMerger(shuffleInputs); } } catch (Exception e) { throw new IOException(e); @@ -293,7 +295,6 @@ void run() throws Exception { break; } } - } /** @@ -306,7 +307,7 @@ void run() throws Exception { Map tag2input = redWork.getTagToInput(); ArrayList shuffleInputs = new ArrayList(); for(String inpStr : tag2input.values()){ - shuffleInputs.add((LogicalInput)inputs.get(inpStr)); + shuffleInputs.add(inputs.get(inpStr)); } return shuffleInputs; } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezProcessor.java ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezProcessor.java index 6839e34..112c902 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezProcessor.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/tez/TezProcessor.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hive.ql.exec.tez; import java.io.IOException; import java.text.NumberFormat; +import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -32,6 +33,7 @@ import org.apache.hadoop.util.StringUtils; import org.apache.tez.common.TezUtils; import org.apache.tez.mapreduce.input.MRInputLegacy; +import org.apache.tez.mapreduce.input.MultiMRInput; import org.apache.tez.mapreduce.processor.MRTaskReporter; import org.apache.tez.runtime.api.Event; import org.apache.tez.runtime.api.LogicalIOProcessor; @@ -59,6 +61,7 @@ private final PerfLogger perfLogger = PerfLogger.getPerfLogger(); private TezProcessorContext processorContext; + private static Map multiMRInputMap; protected static final NumberFormat taskIdFormat = NumberFormat.getInstance(); protected static final NumberFormat jobIdFormat = NumberFormat.getInstance(); @@ -95,6 +98,7 @@ public void initialize(TezProcessorContext processorContext) Configuration conf = TezUtils.createConfFromUserPayload(userPayload); this.jobConf = new JobConf(conf); setupMRLegacyConfigs(processorContext); + multiMRInputMap = new HashMap(); perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.TEZ_INITIALIZE_PROCESSOR); } @@ -138,10 +142,12 @@ public void run(Map inputs, Map out if (isMap) { rproc = new MapRecordProcessor(); MRInputLegacy mrInput = getMRInput(inputs); - try { - mrInput.init(); - } catch (IOException e) { - throw new RuntimeException("Failed while initializing MRInput", e); + if (mrInput != null) { + try { + mrInput.init(); + } catch (IOException e) { + throw new RuntimeException("Failed while initializing MRInput", e); + } } } else { rproc = new ReduceRecordProcessor(); @@ -207,23 +213,38 @@ void initialize() throws Exception { this.writer = (KeyValueWriter) output.getWriter(); } + @Override public void collect(Object key, Object value) throws IOException { writer.write(key, value); } } - static MRInputLegacy getMRInput(Map inputs) { + static MRInputLegacy getMRInput(Map inputs) throws InterruptedException, + IOException { //there should be only one MRInput MRInputLegacy theMRInput = null; - for(LogicalInput inp : inputs.values()){ - if(inp instanceof MRInputLegacy){ + LOG.info("The input names are: " + Arrays.toString(inputs.keySet().toArray())); + for (Entry inp : inputs.entrySet()) { + if (inp.getValue() instanceof MRInputLegacy) { if(theMRInput != null){ throw new IllegalArgumentException("Only one MRInput is expected"); } //a better logic would be to find the alias - theMRInput = (MRInputLegacy)inp; + theMRInput = (MRInputLegacy) inp.getValue(); + } else if (inp.getValue() instanceof MultiMRInput) { + LOG.info("Found input type MultiMRInput"); + multiMRInputMap.put(inp.getKey(), (MultiMRInput) inp.getValue()); + // Iterator mrInputKeyValueIterator = + // multiMRInput.getKeyValueReaders().iterator(); + // while (mrInputKeyValueIterator.hasNext()) { + // KeyValueReader kvReader = mrInputKeyValueIterator.next(); + // } } } return theMRInput; } + + public static MultiMRInput getInput(String inputName) { + return multiMRInputMap.get(inputName); + } } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/tez/tools/KeyValueInputMerger.java ql/src/java/org/apache/hadoop/hive/ql/exec/tez/tools/KeyValueInputMerger.java new file mode 100644 index 0000000..30cdd51 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/exec/tez/tools/KeyValueInputMerger.java @@ -0,0 +1,109 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.ql.exec.tez.tools; + +import java.io.IOException; +import java.util.Comparator; +import java.util.List; +import java.util.PriorityQueue; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.ql.exec.tez.ReduceRecordProcessor; +import org.apache.hadoop.io.BinaryComparable; +import org.apache.tez.runtime.library.api.KeyValueReader; + +/** + * A KeyValuesReader implementation that returns a sorted stream of key-values + * by doing a sorted merge of the key-value in LogicalInputs. + * Tags are in the last byte of the key, so no special handling for tags is required. + * Uses a priority queue to pick the KeyValuesReader of the input that is next in + * sort order. + */ +public class KeyValueInputMerger implements KeyValueReader { + + public static final Log l4j = LogFactory.getLog(ReduceRecordProcessor.class); + private PriorityQueue pQueue = null; + private KeyValueReader nextKVReader = null; + + public KeyValueInputMerger(List multiMRInputs) throws Exception { + //get KeyValuesReaders from the LogicalInput and add them to priority queue + int initialCapacity = multiMRInputs.size(); + pQueue = new PriorityQueue(initialCapacity, new KVReaderComparator()); + for (KeyValueReader input : multiMRInputs) { + addToQueue(input); + } + } + + /** + * Add KeyValueReader to queue if it has more key-value + * + * @param kvReader + * @throws IOException + */ + private void addToQueue(KeyValueReader kvReader) throws IOException { + if (kvReader.next()) { + pQueue.add(kvReader); + } + } + + /** + * @return true if there are more key-values and advances to next key-values + * @throws IOException + */ + @Override + public boolean next() throws IOException { + //add the previous nextKVReader back to queue + if(nextKVReader != null){ + addToQueue(nextKVReader); + } + + //get the new nextKVReader with lowest key + nextKVReader = pQueue.poll(); + return nextKVReader != null; + } + + @Override + public Object getCurrentKey() throws IOException { + return nextKVReader.getCurrentKey(); + } + + @Override + public Object getCurrentValue() throws IOException { + return nextKVReader.getCurrentValue(); + } + + /** + * Comparator that compares KeyValuesReader on their current key + */ + class KVReaderComparator implements Comparator { + + @Override + public int compare(KeyValueReader kvReadr1, KeyValueReader kvReadr2) { + try { + BinaryComparable key1 = (BinaryComparable) kvReadr1.getCurrentValue(); + BinaryComparable key2 = (BinaryComparable) kvReadr2.getCurrentValue(); + return key1.compareTo(key2); + } catch (IOException e) { + l4j.error("Caught exception while reading shuffle input", e); + //die! + throw new RuntimeException(e); + } + } + } +} diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/tez/tools/KeyValuesInputMerger.java ql/src/java/org/apache/hadoop/hive/ql/exec/tez/tools/KeyValuesInputMerger.java new file mode 100644 index 0000000..f62bedb --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/exec/tez/tools/KeyValuesInputMerger.java @@ -0,0 +1,109 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hive.ql.exec.tez.tools; + +import java.io.IOException; +import java.util.Comparator; +import java.util.List; +import java.util.PriorityQueue; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.ql.exec.tez.ReduceRecordProcessor; +import org.apache.hadoop.io.BinaryComparable; +import org.apache.tez.runtime.api.Input; +import org.apache.tez.runtime.api.LogicalInput; +import org.apache.tez.runtime.library.api.KeyValuesReader; + +/** + * A KeyValuesReader implementation that returns a sorted stream of key-values + * by doing a sorted merge of the key-value in LogicalInputs. + * Tags are in the last byte of the key, so no special handling for tags is required. + * Uses a priority queue to pick the KeyValuesReader of the input that is next in + * sort order. + */ +public class KeyValuesInputMerger implements KeyValuesReader { + + public static final Log l4j = LogFactory.getLog(ReduceRecordProcessor.class); + private PriorityQueue pQueue = null; + private KeyValuesReader nextKVReader = null; + + public KeyValuesInputMerger(List shuffleInputs) throws Exception { + //get KeyValuesReaders from the LogicalInput and add them to priority queue + int initialCapacity = shuffleInputs.size(); + pQueue = new PriorityQueue(initialCapacity, new KVReaderComparator()); + for(Input input : shuffleInputs){ + addToQueue((KeyValuesReader)input.getReader()); + } + } + + /** + * Add KeyValuesReader to queue if it has more key-values + * @param kvsReadr + * @throws IOException + */ + private void addToQueue(KeyValuesReader kvsReadr) throws IOException{ + if(kvsReadr.next()){ + pQueue.add(kvsReadr); + } + } + + /** + * @return true if there are more key-values and advances to next key-values + * @throws IOException + */ + public boolean next() throws IOException { + //add the previous nextKVReader back to queue + if(nextKVReader != null){ + addToQueue(nextKVReader); + } + + //get the new nextKVReader with lowest key + nextKVReader = pQueue.poll(); + return nextKVReader != null; + } + + public Object getCurrentKey() throws IOException { + return nextKVReader.getCurrentKey(); + } + + public Iterable getCurrentValues() throws IOException { + return nextKVReader.getCurrentValues(); + } + + /** + * Comparator that compares KeyValuesReader on their current key + */ + class KVReaderComparator implements Comparator { + + @Override + public int compare(KeyValuesReader kvReadr1, KeyValuesReader kvReadr2) { + try { + BinaryComparable key1 = (BinaryComparable) kvReadr1.getCurrentKey(); + BinaryComparable key2 = (BinaryComparable) kvReadr2.getCurrentKey(); + return key1.compareTo(key2); + } catch (IOException e) { + l4j.error("Caught exception while reading shuffle input", e); + //die! + throw new RuntimeException(e); + } + } + } + + +} diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/tez/tools/TezMergedLogicalInput.java ql/src/java/org/apache/hadoop/hive/ql/exec/tez/tools/TezMergedLogicalInput.java index cc4477f..f036c6c 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/tez/tools/TezMergedLogicalInput.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/tez/tools/TezMergedLogicalInput.java @@ -34,7 +34,7 @@ @Override public Reader getReader() throws Exception { - return new InputMerger(getInputs()); + return new KeyValuesInputMerger(getInputs()); } @Override diff --git ql/src/java/org/apache/hadoop/hive/ql/plan/BaseWork.java ql/src/java/org/apache/hadoop/hive/ql/plan/BaseWork.java index 6d9739f..d9610bb 100644 --- ql/src/java/org/apache/hadoop/hive/ql/plan/BaseWork.java +++ ql/src/java/org/apache/hadoop/hive/ql/plan/BaseWork.java @@ -18,8 +18,8 @@ package org.apache.hadoop.hive.ql.plan; -import java.util.LinkedList; import java.util.LinkedHashSet; +import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Set; @@ -36,6 +36,13 @@ @SuppressWarnings({"serial", "deprecation"}) public abstract class BaseWork extends AbstractOperatorDesc { + public enum VertexType { + AUTO_INITIALIZED_EDGES, // no custom vertex or edge + INITIALIZED_EDGES, // custom vertex and custom edge but single MR Input + MULTI_INPUT_INITIALIZED_EDGES, // custom vertex, custom edge and multi MR Input + MULTI_INPUT_UNINITIALIZED_EDGES // custom vertex, no custom edge, multi MR Input + } + // dummyOps is a reference to all the HashTableDummy operators in the // plan. These have to be separately initialized when we setup a task. // Their function is mainly as root ops to give the mapjoin the correct @@ -95,7 +102,7 @@ public void addDummyOp(HashTableDummyOperator dummyOp) { // add all children opStack.addAll(opSet); - + while(!opStack.empty()) { Operator op = opStack.pop(); returnSet.add(op); diff --git ql/src/java/org/apache/hadoop/hive/ql/plan/CommonMergeJoinDesc.java ql/src/java/org/apache/hadoop/hive/ql/plan/CommonMergeJoinDesc.java new file mode 100644 index 0000000..6b9555b --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/plan/CommonMergeJoinDesc.java @@ -0,0 +1,71 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hive.ql.plan; + +import java.io.Serializable; +import java.util.Map; + +import org.apache.hadoop.hive.ql.exec.Operator; + +@Explain(displayName = "Common Merge Join Operator") +public class CommonMergeJoinDesc extends MapJoinDesc implements Serializable { + private static final long serialVersionUID = 1L; + private int numBuckets; + private boolean isSubQuery; + private int mapJoinConversionPos; + private JoinDesc bigTableJoinDesc; + private Map> aliasToSinkMap; + + CommonMergeJoinDesc() { + } + + public CommonMergeJoinDesc(int numBuckets, boolean isSubQuery, int mapJoinConversionPos, + MapJoinDesc joinDesc) { + super(joinDesc); + this.numBuckets = numBuckets; + this.isSubQuery = isSubQuery; + this.mapJoinConversionPos = mapJoinConversionPos; + } + + public boolean getCustomMerge() { + return isSubQuery; + } + + public int getNumBuckets() { + return numBuckets; + } + + public int getBigTablePosition() { + return mapJoinConversionPos; + } + + public Map getTagToAlias() { + // FIXME this is supposed to be populated in the planning phase. Has a + // parent index to input name mapping. + return null; + } + + public Map> getAliasToSinkMap() { + return aliasToSinkMap; + } + + public void setAliasToSinkMap(Map> aliasToSinkMap) { + this.aliasToSinkMap = aliasToSinkMap; + } +} diff --git ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java index 1d96c5d..a2e40db 100644 --- ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java +++ ql/src/java/org/apache/hadoop/hive/ql/plan/MapWork.java @@ -26,9 +26,9 @@ import java.util.LinkedHashMap; import java.util.LinkedHashSet; import java.util.List; -import java.util.Set; import java.util.Map; import java.util.Map.Entry; +import java.util.Set; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -120,6 +120,10 @@ private Map> scratchColumnMap = null; private boolean vectorMode = false; + private boolean doSplitsGrouping = true; + + private VertexType vertexType = VertexType.AUTO_INITIALIZED_EDGES; + public MapWork() {} public MapWork(String name) { @@ -555,4 +559,45 @@ public void logPathToAliases() { } } } + + public void setDoSplitsGrouping(boolean doSplitsGrouping) { + this.doSplitsGrouping = doSplitsGrouping; + } + + public boolean getDoSplitsGrouping() { + return this.doSplitsGrouping; + } + + public void setVertexType(VertexType incomingVertexType) { + switch (this.vertexType) { + case INITIALIZED_EDGES: + if (incomingVertexType == VertexType.MULTI_INPUT_UNINITIALIZED_EDGES) { + vertexType = VertexType.MULTI_INPUT_INITIALIZED_EDGES; + } + break; + + case MULTI_INPUT_INITIALIZED_EDGES: + // nothing to do + break; + + case MULTI_INPUT_UNINITIALIZED_EDGES: + if (incomingVertexType == VertexType.INITIALIZED_EDGES) { + vertexType = VertexType.MULTI_INPUT_INITIALIZED_EDGES; + } + break; + + case AUTO_INITIALIZED_EDGES: + vertexType = incomingVertexType; + break; + + default: + break; + } + this.vertexType = vertexType; + } + + public VertexType getVertexType() { + // TODO Auto-generated method stub + return this.vertexType; + } } diff --git ql/src/java/org/apache/hadoop/hive/ql/plan/MergeJoinWork.java ql/src/java/org/apache/hadoop/hive/ql/plan/MergeJoinWork.java new file mode 100644 index 0000000..59dd752 --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/plan/MergeJoinWork.java @@ -0,0 +1,90 @@ +package org.apache.hadoop.hive.ql.plan; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.hadoop.hive.ql.exec.CommonMergeJoinOperator; +import org.apache.hadoop.hive.ql.exec.Operator; +import org.apache.hadoop.mapred.JobConf; + +public class MergeJoinWork extends BaseWork { + + private final List mergeJoinOpList = + new ArrayList(); + private final List mergeWorkList = new ArrayList(); + private int bigTableInputId; + private boolean isBigTableWork; + private BaseWork bigTableWork; + + public MergeJoinWork() { + super(); + } + + public MergeJoinWork(BaseWork work) { + super(work.getName()); + } + + @Explain(displayName = "Vertex") + @Override + public String getName() { + return super.getName(); + } + + @Override + public void replaceRoots(Map, Operator> replacementMap) { + } + + @Override + public Set> getAllRootOperators() { + return new HashSet>(); + } + + @Override + public void configureJobConf(JobConf job) { + } + + public List getMergeJoinOperator() { + return this.mergeJoinOpList; + } + + private void addMergeJoinOperator(CommonMergeJoinOperator mergeJoinOp) { + this.mergeJoinOpList.add(mergeJoinOp); + this.bigTableInputId = mergeJoinOp.getConf().getBigTablePosition(); + } + + public void + addMergedWork(BaseWork work, BaseWork connectWork, CommonMergeJoinOperator mergeJoinOp) { + addMergeJoinOperator(mergeJoinOp); + if (work != null) { + if ((bigTableWork != null) && (bigTableWork != work)) { + assert false; + } + this.bigTableWork = work; + setName(work.getName()); + if (work instanceof MapWork) { + MapWork mapWork = (MapWork) work; + mapWork.setVertexType(VertexType.MULTI_INPUT_UNINITIALIZED_EDGES); + mapWork.setDoSplitsGrouping(false); + } + } + + if (connectWork != null) { + this.mergeWorkList.add(connectWork); + } + } + + public List getBaseWorkList() { + return mergeWorkList; + } + + public String getBigTableAlias() { + return ((MapWork) bigTableWork).getAliasToWork().keySet().iterator().next(); + } + + public BaseWork getMainWork() { + return bigTableWork; + } +} diff --git ql/src/test/queries/clientpositive/auto_sortmerge_join_1.q ql/src/test/queries/clientpositive/auto_sortmerge_join_1.q index ddd2c18..9ec2935 100644 --- ql/src/test/queries/clientpositive/auto_sortmerge_join_1.q +++ ql/src/test/queries/clientpositive/auto_sortmerge_join_1.q @@ -1,36 +1,39 @@ --- small 1 part, 2 bucket & big 2 part, 4 bucket +set hive.auto.convert.join=true; +set hive.auto.convert.join.noconditionaltask=true; +set hive.auto.convert.join.noconditionaltask.size=10000; -CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) -CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +CREATE TABLE srcbucket_mapjoin(key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; +CREATE TABLE tab_part (key int, value string) PARTITIONED BY(ds STRING) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; +CREATE TABLE srcbucket_mapjoin_part (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; -CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; -load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin partition(ds='2008-04-08'); -load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); -set hive.auto.convert.join=true; -set hive.auto.convert.sortmerge.join=true; +set hive.enforce.bucketing=true; +set hive.enforce.sorting = true; +set hive.optimize.bucketingsorting=false; set hive.optimize.bucketmapjoin = true; set hive.optimize.bucketmapjoin.sortedmerge = true; - -set hive.auto.convert.sortmerge.join.to.mapjoin=false; set hive.auto.convert.sortmerge.join.bigtable.selection.policy = org.apache.hadoop.hive.ql.optimizer.TableSizeBasedBigTableSelectorForAutoSMJ; --- Since size is being used to find the big table, the order of the tables in the join does not matter -explain extended select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; -select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; +insert overwrite table tab_part partition (ds='2008-04-08') +select key,value from srcbucket_mapjoin_part; + +CREATE TABLE tab(key int, value string) PARTITIONED BY(ds STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; +insert overwrite table tab partition (ds='2008-04-08') +select key,value from srcbucket_mapjoin; + +set hive.convert.join.bucket.mapjoin.tez = false; +set hive.auto.convert.sortmerge.join = true; -explain extended select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key; -select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key; +explain +select a.key, a.value, b.value +from tab a join tab_part b on a.key = b.key; -set hive.auto.convert.sortmerge.join.to.mapjoin=true; -explain extended select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key; -select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key; +select a.key, a.value, b.value +from tab a join tab_part b on a.key = b.key; diff --git ql/src/test/queries/clientpositive/tez_bmj_schema_evolution.q ql/src/test/queries/clientpositive/tez_bmj_schema_evolution.q index 996ea88..697881e 100644 --- ql/src/test/queries/clientpositive/tez_bmj_schema_evolution.q +++ ql/src/test/queries/clientpositive/tez_bmj_schema_evolution.q @@ -21,3 +21,26 @@ explain select test.key, test.value from test join test1 on (test.key = test1.ke select test.key, test.value from test join test1 on (test.key = test1.key) order by test.key; +set hive.enforce.bucketing=true; +set hive.enforce.sorting = true; +set hive.optimize.bucketingsorting=false; +set hive.auto.convert.join.noconditionaltask.size=10000; + +create table test (key int, value string) partitioned by (p int) clustered by (key) into 2 buckets stored as textfile; +create table test1 (key int, value string) stored as textfile; + +insert into table test partition (p=1) select * from src; + +alter table test set fileformat orc; + +insert into table test partition (p=2) select * from src; +insert into table test1 select * from src; + +describe test; +set hive.auto.convert.join = true; +set hive.convert.join.bucket.mapjoin.tez = true; + +explain select test.key, test.value from test join test1 on (test.key = test1.key) order by test.key; + +select test.key, test.value from test join test1 on (test.key = test1.key) order by test.key; + diff --git ql/src/test/queries/clientpositive/tez_smb_1.q ql/src/test/queries/clientpositive/tez_smb_1.q new file mode 100644 index 0000000..dc62e31 --- /dev/null +++ ql/src/test/queries/clientpositive/tez_smb_1.q @@ -0,0 +1,39 @@ +explain +select a.key, a.value from src a join src b where a.key = b.key; + +set hive.auto.convert.join=true; +set hive.auto.convert.join.noconditionaltask=true; +set hive.auto.convert.join.noconditionaltask.size=10000; +set hive.auto.convert.sortmerge.join.bigtable.selection.policy = org.apache.hadoop.hive.ql.optimizer.TableSizeBasedBigTableSelectorForAutoSMJ; + +CREATE TABLE srcbucket_mapjoin(key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; +CREATE TABLE tab_part (key int, value string) PARTITIONED BY(ds STRING) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; +CREATE TABLE srcbucket_mapjoin_part (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; + +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin partition(ds='2008-04-08'); + +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); + +set hive.enforce.bucketing=true; +set hive.enforce.sorting = true; +set hive.optimize.bucketingsorting=false; +insert overwrite table tab_part partition (ds='2008-04-08') +select key,value from srcbucket_mapjoin_part; + +CREATE TABLE tab(key int, value string) PARTITIONED BY(ds STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; +insert overwrite table tab partition (ds='2008-04-08') +select key,value from srcbucket_mapjoin; + +set hive.convert.join.bucket.mapjoin.tez = false; +set hive.auto.convert.sortmerge.join = true; + +explain +select a.key, a.value, b.value +from tab a join tab_part b on a.key = b.key; + +select a.key, a.value, b.value +from tab a join tab_part b on a.key = b.key; diff --git ql/src/test/results/clientpositive/auto_sortmerge_join_1.q.out ql/src/test/results/clientpositive/auto_sortmerge_join_1.q.out index 3f36792..1c15757 100644 --- ql/src/test/results/clientpositive/auto_sortmerge_join_1.q.out +++ ql/src/test/results/clientpositive/auto_sortmerge_join_1.q.out @@ -1,371 +1,117 @@ -PREHOOK: query: -- small 1 part, 2 bucket & big 2 part, 4 bucket - -CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) -CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE +PREHOOK: query: CREATE TABLE srcbucket_mapjoin(key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE PREHOOK: type: CREATETABLE PREHOOK: Output: database:default -POSTHOOK: query: -- small 1 part, 2 bucket & big 2 part, 4 bucket - -CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) -CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE +POSTHOOK: query: CREATE TABLE srcbucket_mapjoin(key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default -POSTHOOK: Output: default@bucket_small -PREHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08') -PREHOOK: type: LOAD -#### A masked pattern was here #### -PREHOOK: Output: default@bucket_small -POSTHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08') -POSTHOOK: type: LOAD -#### A masked pattern was here #### -POSTHOOK: Output: default@bucket_small -POSTHOOK: Output: default@bucket_small@ds=2008-04-08 -PREHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08') -PREHOOK: type: LOAD -#### A masked pattern was here #### -PREHOOK: Output: default@bucket_small@ds=2008-04-08 -POSTHOOK: query: load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08') -POSTHOOK: type: LOAD -#### A masked pattern was here #### -POSTHOOK: Output: default@bucket_small@ds=2008-04-08 -PREHOOK: query: CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE +POSTHOOK: Output: default@srcbucket_mapjoin +PREHOOK: query: CREATE TABLE tab_part (key int, value string) PARTITIONED BY(ds STRING) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE PREHOOK: type: CREATETABLE PREHOOK: Output: database:default -POSTHOOK: query: CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE +POSTHOOK: query: CREATE TABLE tab_part (key int, value string) PARTITIONED BY(ds STRING) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default -POSTHOOK: Output: default@bucket_big -PREHOOK: query: load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') -PREHOOK: type: LOAD -#### A masked pattern was here #### -PREHOOK: Output: default@bucket_big -POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') -POSTHOOK: type: LOAD -#### A masked pattern was here #### -POSTHOOK: Output: default@bucket_big -POSTHOOK: Output: default@bucket_big@ds=2008-04-08 -PREHOOK: query: load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') -PREHOOK: type: LOAD -#### A masked pattern was here #### -PREHOOK: Output: default@bucket_big@ds=2008-04-08 -POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') -POSTHOOK: type: LOAD -#### A masked pattern was here #### -POSTHOOK: Output: default@bucket_big@ds=2008-04-08 -PREHOOK: query: load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +POSTHOOK: Output: default@tab_part +PREHOOK: query: CREATE TABLE srcbucket_mapjoin_part (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +POSTHOOK: query: CREATE TABLE srcbucket_mapjoin_part (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@srcbucket_mapjoin_part +PREHOOK: query: load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin partition(ds='2008-04-08') PREHOOK: type: LOAD #### A masked pattern was here #### -PREHOOK: Output: default@bucket_big@ds=2008-04-08 -POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +PREHOOK: Output: default@srcbucket_mapjoin +POSTHOOK: query: load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin partition(ds='2008-04-08') POSTHOOK: type: LOAD #### A masked pattern was here #### -POSTHOOK: Output: default@bucket_big@ds=2008-04-08 -PREHOOK: query: load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +POSTHOOK: Output: default@srcbucket_mapjoin +POSTHOOK: Output: default@srcbucket_mapjoin@ds=2008-04-08 +PREHOOK: query: load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin partition(ds='2008-04-08') PREHOOK: type: LOAD #### A masked pattern was here #### -PREHOOK: Output: default@bucket_big@ds=2008-04-08 -POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08') +PREHOOK: Output: default@srcbucket_mapjoin@ds=2008-04-08 +POSTHOOK: query: load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin partition(ds='2008-04-08') POSTHOOK: type: LOAD #### A masked pattern was here #### -POSTHOOK: Output: default@bucket_big@ds=2008-04-08 -PREHOOK: query: load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09') +POSTHOOK: Output: default@srcbucket_mapjoin@ds=2008-04-08 +PREHOOK: query: load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08') PREHOOK: type: LOAD #### A masked pattern was here #### -PREHOOK: Output: default@bucket_big -POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09') +PREHOOK: Output: default@srcbucket_mapjoin_part +POSTHOOK: query: load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08') POSTHOOK: type: LOAD #### A masked pattern was here #### -POSTHOOK: Output: default@bucket_big -POSTHOOK: Output: default@bucket_big@ds=2008-04-09 -PREHOOK: query: load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09') +POSTHOOK: Output: default@srcbucket_mapjoin_part +POSTHOOK: Output: default@srcbucket_mapjoin_part@ds=2008-04-08 +PREHOOK: query: load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08') PREHOOK: type: LOAD #### A masked pattern was here #### -PREHOOK: Output: default@bucket_big@ds=2008-04-09 -POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09') +PREHOOK: Output: default@srcbucket_mapjoin_part@ds=2008-04-08 +POSTHOOK: query: load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08') POSTHOOK: type: LOAD #### A masked pattern was here #### -POSTHOOK: Output: default@bucket_big@ds=2008-04-09 -PREHOOK: query: load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09') +POSTHOOK: Output: default@srcbucket_mapjoin_part@ds=2008-04-08 +PREHOOK: query: load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08') PREHOOK: type: LOAD #### A masked pattern was here #### -PREHOOK: Output: default@bucket_big@ds=2008-04-09 -POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09') +PREHOOK: Output: default@srcbucket_mapjoin_part@ds=2008-04-08 +POSTHOOK: query: load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08') POSTHOOK: type: LOAD #### A masked pattern was here #### -POSTHOOK: Output: default@bucket_big@ds=2008-04-09 -PREHOOK: query: load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09') +POSTHOOK: Output: default@srcbucket_mapjoin_part@ds=2008-04-08 +PREHOOK: query: load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08') PREHOOK: type: LOAD #### A masked pattern was here #### -PREHOOK: Output: default@bucket_big@ds=2008-04-09 -POSTHOOK: query: load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09') +PREHOOK: Output: default@srcbucket_mapjoin_part@ds=2008-04-08 +POSTHOOK: query: load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08') POSTHOOK: type: LOAD #### A masked pattern was here #### -POSTHOOK: Output: default@bucket_big@ds=2008-04-09 -PREHOOK: query: -- Since size is being used to find the big table, the order of the tables in the join does not matter -explain extended select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +POSTHOOK: Output: default@srcbucket_mapjoin_part@ds=2008-04-08 +PREHOOK: query: insert overwrite table tab_part partition (ds='2008-04-08') +select key,value from srcbucket_mapjoin_part PREHOOK: type: QUERY -POSTHOOK: query: -- Since size is being used to find the big table, the order of the tables in the join does not matter -explain extended select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +PREHOOK: Input: default@srcbucket_mapjoin_part +PREHOOK: Input: default@srcbucket_mapjoin_part@ds=2008-04-08 +PREHOOK: Output: default@tab_part@ds=2008-04-08 +POSTHOOK: query: insert overwrite table tab_part partition (ds='2008-04-08') +select key,value from srcbucket_mapjoin_part POSTHOOK: type: QUERY -ABSTRACT SYNTAX TREE: - -TOK_QUERY - TOK_FROM - TOK_JOIN - TOK_TABREF - TOK_TABNAME - bucket_small - a - TOK_TABREF - TOK_TABNAME - bucket_big - b - = - . - TOK_TABLE_OR_COL - a - key - . - TOK_TABLE_OR_COL - b - key - TOK_INSERT - TOK_DESTINATION - TOK_DIR - TOK_TMP_FILE - TOK_SELECT - TOK_SELEXPR - TOK_FUNCTIONSTAR - count - - -STAGE DEPENDENCIES: - Stage-1 is a root stage - Stage-0 depends on stages: Stage-1 - -STAGE PLANS: - Stage: Stage-1 - Map Reduce - Map Operator Tree: - TableScan - alias: b - Statistics: Num rows: 116 Data size: 11624 Basic stats: COMPLETE Column stats: NONE - GatherStats: false - Filter Operator - isSamplingPred: false - predicate: key is not null (type: boolean) - Statistics: Num rows: 58 Data size: 5812 Basic stats: COMPLETE Column stats: NONE - Sorted Merge Bucket Map Join Operator - condition map: - Inner Join 0 to 1 - condition expressions: - 0 - 1 - keys: - 0 key (type: string) - 1 key (type: string) - Position of Big Table: 1 - Select Operator - Group By Operator - aggregations: count() - mode: hash - outputColumnNames: _col0 - Reduce Output Operator - sort order: - tag: -1 - value expressions: _col0 (type: bigint) - auto parallelism: false - Path -> Alias: -#### A masked pattern was here #### - Path -> Partition: -#### A masked pattern was here #### - Partition - base file name: ds=2008-04-08 - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 4 - bucket_field_name key - columns key,value - columns.comments - columns.types string:string -#### A masked pattern was here #### - name default.bucket_big - numFiles 4 - numRows 0 - partition_columns ds - partition_columns.types string - rawDataSize 0 - serialization.ddl struct bucket_big { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 5812 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - SORTBUCKETCOLSPREFIX TRUE - bucket_count 4 - bucket_field_name key - columns key,value - columns.comments - columns.types string:string -#### A masked pattern was here #### - name default.bucket_big - partition_columns ds - partition_columns.types string - serialization.ddl struct bucket_big { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_big - name: default.bucket_big -#### A masked pattern was here #### - Partition - base file name: ds=2008-04-09 - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-09 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 4 - bucket_field_name key - columns key,value - columns.comments - columns.types string:string -#### A masked pattern was here #### - name default.bucket_big - numFiles 4 - numRows 0 - partition_columns ds - partition_columns.types string - rawDataSize 0 - serialization.ddl struct bucket_big { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 5812 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - SORTBUCKETCOLSPREFIX TRUE - bucket_count 4 - bucket_field_name key - columns key,value - columns.comments - columns.types string:string -#### A masked pattern was here #### - name default.bucket_big - partition_columns ds - partition_columns.types string - serialization.ddl struct bucket_big { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_big - name: default.bucket_big - Truncated Path -> Alias: - /bucket_big/ds=2008-04-08 [b] - /bucket_big/ds=2008-04-09 [b] - Needs Tagging: false - Reduce Operator Tree: - Group By Operator - aggregations: count(VALUE._col0) - mode: mergepartial - outputColumnNames: _col0 - Select Operator - expressions: _col0 (type: bigint) - outputColumnNames: _col0 - File Output Operator - compressed: false - GlobalTableId: 0 -#### A masked pattern was here #### - NumFilesPerFileSink: 1 -#### 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 - columns.types bigint - escape.delim \ - hive.serialization.extend.nesting.levels true - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - TotalFiles: 1 - GatherStats: false - MultiFileSpray: false - - Stage: Stage-0 - Fetch Operator - limit: -1 - Processor Tree: - ListSink - -PREHOOK: query: select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +POSTHOOK: Input: default@srcbucket_mapjoin_part +POSTHOOK: Input: default@srcbucket_mapjoin_part@ds=2008-04-08 +POSTHOOK: Output: default@tab_part@ds=2008-04-08 +POSTHOOK: Lineage: tab_part PARTITION(ds=2008-04-08).key SIMPLE [(srcbucket_mapjoin_part)srcbucket_mapjoin_part.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: tab_part PARTITION(ds=2008-04-08).value SIMPLE [(srcbucket_mapjoin_part)srcbucket_mapjoin_part.FieldSchema(name:value, type:string, comment:null), ] +PREHOOK: query: CREATE TABLE tab(key int, value string) PARTITIONED BY(ds STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +POSTHOOK: query: CREATE TABLE tab(key int, value string) PARTITIONED BY(ds STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@tab +PREHOOK: query: insert overwrite table tab partition (ds='2008-04-08') +select key,value from srcbucket_mapjoin PREHOOK: type: QUERY -PREHOOK: Input: default@bucket_big -PREHOOK: Input: default@bucket_big@ds=2008-04-08 -PREHOOK: Input: default@bucket_big@ds=2008-04-09 -PREHOOK: Input: default@bucket_small -PREHOOK: Input: default@bucket_small@ds=2008-04-08 -#### A masked pattern was here #### -POSTHOOK: query: select count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key +PREHOOK: Input: default@srcbucket_mapjoin +PREHOOK: Input: default@srcbucket_mapjoin@ds=2008-04-08 +PREHOOK: Output: default@tab@ds=2008-04-08 +POSTHOOK: query: insert overwrite table tab partition (ds='2008-04-08') +select key,value from srcbucket_mapjoin POSTHOOK: type: QUERY -POSTHOOK: Input: default@bucket_big -POSTHOOK: Input: default@bucket_big@ds=2008-04-08 -POSTHOOK: Input: default@bucket_big@ds=2008-04-09 -POSTHOOK: Input: default@bucket_small -POSTHOOK: Input: default@bucket_small@ds=2008-04-08 -#### A masked pattern was here #### -38 -PREHOOK: query: explain extended select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +POSTHOOK: Input: default@srcbucket_mapjoin +POSTHOOK: Input: default@srcbucket_mapjoin@ds=2008-04-08 +POSTHOOK: Output: default@tab@ds=2008-04-08 +POSTHOOK: Lineage: tab PARTITION(ds=2008-04-08).key SIMPLE [(srcbucket_mapjoin)srcbucket_mapjoin.FieldSchema(name:key, type:int, comment:null), ] +POSTHOOK: Lineage: tab PARTITION(ds=2008-04-08).value SIMPLE [(srcbucket_mapjoin)srcbucket_mapjoin.FieldSchema(name:value, type:string, comment:null), ] +PREHOOK: query: explain +select a.key, a.value, b.value +from tab a join tab_part b on a.key = b.key PREHOOK: type: QUERY -POSTHOOK: query: explain extended select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +POSTHOOK: query: explain +select a.key, a.value, b.value +from tab a join tab_part b on a.key = b.key POSTHOOK: type: QUERY -ABSTRACT SYNTAX TREE: - -TOK_QUERY - TOK_FROM - TOK_JOIN - TOK_TABREF - TOK_TABNAME - bucket_big - a - TOK_TABREF - TOK_TABNAME - bucket_small - b - = - . - TOK_TABLE_OR_COL - a - key - . - TOK_TABLE_OR_COL - b - key - TOK_INSERT - TOK_DESTINATION - TOK_DIR - TOK_TMP_FILE - TOK_SELECT - TOK_SELEXPR - TOK_FUNCTIONSTAR - count - - STAGE DEPENDENCIES: Stage-1 is a root stage Stage-0 depends on stages: Stage-1 @@ -375,1006 +121,30 @@ STAGE PLANS: Map Reduce Map Operator Tree: TableScan - alias: a - Statistics: Num rows: 116 Data size: 11624 Basic stats: COMPLETE Column stats: NONE - GatherStats: false - Filter Operator - isSamplingPred: false - predicate: key is not null (type: boolean) - Statistics: Num rows: 58 Data size: 5812 Basic stats: COMPLETE Column stats: NONE - Sorted Merge Bucket Map Join Operator - condition map: - Inner Join 0 to 1 - condition expressions: - 0 - 1 - keys: - 0 key (type: string) - 1 key (type: string) - Position of Big Table: 0 - Select Operator - Group By Operator - aggregations: count() - mode: hash - outputColumnNames: _col0 - Reduce Output Operator - sort order: - tag: -1 - value expressions: _col0 (type: bigint) - auto parallelism: false - Path -> Alias: -#### A masked pattern was here #### - Path -> Partition: -#### A masked pattern was here #### - Partition - base file name: ds=2008-04-08 - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 4 - bucket_field_name key - columns key,value - columns.comments - columns.types string:string -#### A masked pattern was here #### - name default.bucket_big - numFiles 4 - numRows 0 - partition_columns ds - partition_columns.types string - rawDataSize 0 - serialization.ddl struct bucket_big { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 5812 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - SORTBUCKETCOLSPREFIX TRUE - bucket_count 4 - bucket_field_name key - columns key,value - columns.comments - columns.types string:string -#### A masked pattern was here #### - name default.bucket_big - partition_columns ds - partition_columns.types string - serialization.ddl struct bucket_big { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_big - name: default.bucket_big -#### A masked pattern was here #### - Partition - base file name: ds=2008-04-09 - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-09 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 4 - bucket_field_name key - columns key,value - columns.comments - columns.types string:string -#### A masked pattern was here #### - name default.bucket_big - numFiles 4 - numRows 0 - partition_columns ds - partition_columns.types string - rawDataSize 0 - serialization.ddl struct bucket_big { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 5812 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - SORTBUCKETCOLSPREFIX TRUE - bucket_count 4 - bucket_field_name key - columns key,value - columns.comments - columns.types string:string -#### A masked pattern was here #### - name default.bucket_big - partition_columns ds - partition_columns.types string - serialization.ddl struct bucket_big { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_big - name: default.bucket_big - Truncated Path -> Alias: - /bucket_big/ds=2008-04-08 [a] - /bucket_big/ds=2008-04-09 [a] - Needs Tagging: false - Reduce Operator Tree: - Group By Operator - aggregations: count(VALUE._col0) - mode: mergepartial - outputColumnNames: _col0 - Select Operator - expressions: _col0 (type: bigint) - outputColumnNames: _col0 - File Output Operator - compressed: false - GlobalTableId: 0 -#### A masked pattern was here #### - NumFilesPerFileSink: 1 -#### 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 - columns.types bigint - escape.delim \ - hive.serialization.extend.nesting.levels true - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - TotalFiles: 1 - GatherStats: false - MultiFileSpray: false - - Stage: Stage-0 - Fetch Operator - limit: -1 - Processor Tree: - ListSink - -PREHOOK: query: select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key -PREHOOK: type: QUERY -PREHOOK: Input: default@bucket_big -PREHOOK: Input: default@bucket_big@ds=2008-04-08 -PREHOOK: Input: default@bucket_big@ds=2008-04-09 -PREHOOK: Input: default@bucket_small -PREHOOK: Input: default@bucket_small@ds=2008-04-08 -#### A masked pattern was here #### -POSTHOOK: query: select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key -POSTHOOK: type: QUERY -POSTHOOK: Input: default@bucket_big -POSTHOOK: Input: default@bucket_big@ds=2008-04-08 -POSTHOOK: Input: default@bucket_big@ds=2008-04-09 -POSTHOOK: Input: default@bucket_small -POSTHOOK: Input: default@bucket_small@ds=2008-04-08 -#### A masked pattern was here #### -38 -PREHOOK: query: explain extended select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key -PREHOOK: type: QUERY -POSTHOOK: query: explain extended select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key -POSTHOOK: type: QUERY -ABSTRACT SYNTAX TREE: - -TOK_QUERY - TOK_FROM - TOK_JOIN - TOK_TABREF - TOK_TABNAME - bucket_big - a - TOK_TABREF - TOK_TABNAME - bucket_small - b - = - . - TOK_TABLE_OR_COL - a - key - . - TOK_TABLE_OR_COL - b - key - TOK_INSERT - TOK_DESTINATION - TOK_DIR - TOK_TMP_FILE - TOK_SELECT - TOK_SELEXPR - TOK_FUNCTIONSTAR - count - - -STAGE DEPENDENCIES: - Stage-5 is a root stage , consists of Stage-6, Stage-7, Stage-1 - Stage-6 has a backup stage: Stage-1 - Stage-3 depends on stages: Stage-6 - Stage-7 has a backup stage: Stage-1 - Stage-4 depends on stages: Stage-7 - Stage-1 - Stage-0 depends on stages: Stage-3, Stage-4, Stage-1 - -STAGE PLANS: - Stage: Stage-5 - Conditional Operator - - Stage: Stage-6 - Map Reduce Local Work - Alias -> Map Local Tables: - b - Fetch Operator - limit: -1 - Partition Description: - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 2 - bucket_field_name key - columns key,value - columns.comments - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - numFiles 2 - numRows 0 - partition_columns ds - partition_columns.types string - rawDataSize 0 - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 114 -#### 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: - SORTBUCKETCOLSPREFIX TRUE - bucket_count 2 - bucket_field_name key - columns key,value - columns.comments - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - partition_columns ds - partition_columns.types string - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_small - name: default.bucket_small - Alias -> Map Local Operator Tree: - b - TableScan alias: b - GatherStats: false + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Filter Operator - isSamplingPred: false predicate: key is not null (type: boolean) - HashTable Sink Operator - condition expressions: - 0 - 1 - keys: - 0 key (type: string) - 1 key (type: string) - Position of Big Table: 0 - - Stage: Stage-3 - Map Reduce - Map Operator Tree: - TableScan - alias: a - GatherStats: false - Filter Operator - isSamplingPred: false - predicate: key is not null (type: boolean) - Map Join Operator - condition map: - Inner Join 0 to 1 - condition expressions: - 0 - 1 - keys: - 0 key (type: string) - 1 key (type: string) - Position of Big Table: 0 - Select Operator - Group By Operator - aggregations: count() - mode: hash - outputColumnNames: _col0 - Reduce Output Operator - sort order: - tag: -1 - value expressions: _col0 (type: bigint) - auto parallelism: 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: ds=2008-04-08 - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 4 - bucket_field_name key - columns key,value - columns.comments - columns.types string:string -#### A masked pattern was here #### - name default.bucket_big - numFiles 4 - numRows 0 - partition_columns ds - partition_columns.types string - rawDataSize 0 - serialization.ddl struct bucket_big { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 5812 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - SORTBUCKETCOLSPREFIX TRUE - bucket_count 4 - bucket_field_name key - columns key,value - columns.comments - columns.types string:string -#### A masked pattern was here #### - name default.bucket_big - partition_columns ds - partition_columns.types string - serialization.ddl struct bucket_big { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_big - name: default.bucket_big -#### A masked pattern was here #### - Partition - base file name: ds=2008-04-09 - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-09 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 4 - bucket_field_name key - columns key,value - columns.comments - columns.types string:string -#### A masked pattern was here #### - name default.bucket_big - numFiles 4 - numRows 0 - partition_columns ds - partition_columns.types string - rawDataSize 0 - serialization.ddl struct bucket_big { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 5812 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - SORTBUCKETCOLSPREFIX TRUE - bucket_count 4 - bucket_field_name key - columns key,value - columns.comments - columns.types string:string -#### A masked pattern was here #### - name default.bucket_big - partition_columns ds - partition_columns.types string - serialization.ddl struct bucket_big { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_big - name: default.bucket_big -#### A masked pattern was here #### - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 2 - bucket_field_name key - columns key,value - columns.comments - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - numFiles 2 - numRows 0 - partition_columns ds - partition_columns.types string - rawDataSize 0 - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 114 -#### 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: - SORTBUCKETCOLSPREFIX TRUE - bucket_count 2 - bucket_field_name key - columns key,value - columns.comments - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - partition_columns ds - partition_columns.types string - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_small - name: default.bucket_small - Truncated Path -> Alias: - /bucket_big/ds=2008-04-08 [a] - /bucket_big/ds=2008-04-09 [a] - Needs Tagging: false - Reduce Operator Tree: - Group By Operator - aggregations: count(VALUE._col0) - mode: mergepartial - outputColumnNames: _col0 - Select Operator - expressions: _col0 (type: bigint) - outputColumnNames: _col0 - File Output Operator - compressed: false - GlobalTableId: 0 -#### A masked pattern was here #### - NumFilesPerFileSink: 1 -#### 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 - columns.types bigint - escape.delim \ - hive.serialization.extend.nesting.levels true - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - TotalFiles: 1 - GatherStats: false - MultiFileSpray: false - - Stage: Stage-7 - Map Reduce Local Work - Alias -> Map Local Tables: - a - Fetch Operator - limit: -1 - Partition Description: - Partition - base file name: ds=2008-04-08 - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 4 - bucket_field_name key - columns key,value - columns.comments - columns.types string:string -#### A masked pattern was here #### - name default.bucket_big - numFiles 4 - numRows 0 - partition_columns ds - partition_columns.types string - rawDataSize 0 - serialization.ddl struct bucket_big { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 5812 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - SORTBUCKETCOLSPREFIX TRUE - bucket_count 4 - bucket_field_name key - columns key,value - columns.comments - columns.types string:string -#### A masked pattern was here #### - name default.bucket_big - partition_columns ds - partition_columns.types string - serialization.ddl struct bucket_big { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_big - name: default.bucket_big - Partition - base file name: ds=2008-04-09 - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-09 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 4 - bucket_field_name key - columns key,value - columns.comments - columns.types string:string -#### A masked pattern was here #### - name default.bucket_big - numFiles 4 - numRows 0 - partition_columns ds - partition_columns.types string - rawDataSize 0 - serialization.ddl struct bucket_big { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 5812 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - SORTBUCKETCOLSPREFIX TRUE - bucket_count 4 - bucket_field_name key - columns key,value - columns.comments - columns.types string:string -#### A masked pattern was here #### - name default.bucket_big - partition_columns ds - partition_columns.types string - serialization.ddl struct bucket_big { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_big - name: default.bucket_big - Alias -> Map Local Operator Tree: - a - TableScan - alias: a - GatherStats: false - Filter Operator - isSamplingPred: false - predicate: key is not null (type: boolean) - HashTable Sink Operator - condition expressions: - 0 - 1 - keys: - 0 key (type: string) - 1 key (type: string) - Position of Big Table: 1 - - Stage: Stage-4 - Map Reduce - Map Operator Tree: - TableScan - alias: b - GatherStats: false - Filter Operator - isSamplingPred: false - predicate: key is not null (type: boolean) - Map Join Operator - condition map: - Inner Join 0 to 1 - condition expressions: - 0 - 1 - keys: - 0 key (type: string) - 1 key (type: string) - Position of Big Table: 1 - Select Operator - Group By Operator - aggregations: count() - mode: hash - outputColumnNames: _col0 - Reduce Output Operator - sort order: - tag: -1 - value expressions: _col0 (type: bigint) - auto parallelism: 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: ds=2008-04-08 - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 4 - bucket_field_name key - columns key,value - columns.comments - columns.types string:string -#### A masked pattern was here #### - name default.bucket_big - numFiles 4 - numRows 0 - partition_columns ds - partition_columns.types string - rawDataSize 0 - serialization.ddl struct bucket_big { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 5812 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - SORTBUCKETCOLSPREFIX TRUE - bucket_count 4 - bucket_field_name key - columns key,value - columns.comments - columns.types string:string -#### A masked pattern was here #### - name default.bucket_big - partition_columns ds - partition_columns.types string - serialization.ddl struct bucket_big { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_big - name: default.bucket_big -#### A masked pattern was here #### - Partition - base file name: ds=2008-04-09 - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-09 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 4 - bucket_field_name key - columns key,value - columns.comments - columns.types string:string -#### A masked pattern was here #### - name default.bucket_big - numFiles 4 - numRows 0 - partition_columns ds - partition_columns.types string - rawDataSize 0 - serialization.ddl struct bucket_big { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 5812 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - SORTBUCKETCOLSPREFIX TRUE - bucket_count 4 - bucket_field_name key - columns key,value - columns.comments - columns.types string:string -#### A masked pattern was here #### - name default.bucket_big - partition_columns ds - partition_columns.types string - serialization.ddl struct bucket_big { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_big - name: default.bucket_big -#### A masked pattern was here #### - Partition - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 2 - bucket_field_name key - columns key,value - columns.comments - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - numFiles 2 - numRows 0 - partition_columns ds - partition_columns.types string - rawDataSize 0 - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 114 -#### 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: - SORTBUCKETCOLSPREFIX TRUE - bucket_count 2 - bucket_field_name key - columns key,value - columns.comments - columns.types string:string -#### A masked pattern was here #### - name default.bucket_small - partition_columns ds - partition_columns.types string - serialization.ddl struct bucket_small { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_small - name: default.bucket_small - Truncated Path -> Alias: - /bucket_small/ds=2008-04-08 [b] - Needs Tagging: false - Reduce Operator Tree: - Group By Operator - aggregations: count(VALUE._col0) - mode: mergepartial - outputColumnNames: _col0 - Select Operator - expressions: _col0 (type: bigint) - outputColumnNames: _col0 - File Output Operator - compressed: false - GlobalTableId: 0 -#### A masked pattern was here #### - NumFilesPerFileSink: 1 -#### 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 - columns.types bigint - escape.delim \ - hive.serialization.extend.nesting.levels true - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - TotalFiles: 1 - GatherStats: false - MultiFileSpray: false - - Stage: Stage-1 - Map Reduce - Map Operator Tree: - TableScan - alias: a - Statistics: Num rows: 116 Data size: 11624 Basic stats: COMPLETE Column stats: NONE - GatherStats: false - Filter Operator - isSamplingPred: false - predicate: key is not null (type: boolean) - Statistics: Num rows: 58 Data size: 5812 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE Sorted Merge Bucket Map Join Operator condition map: Inner Join 0 to 1 condition expressions: - 0 - 1 + 0 {key} {value} + 1 {value} keys: - 0 key (type: string) - 1 key (type: string) - Position of Big Table: 0 + 0 key (type: int) + 1 key (type: int) + outputColumnNames: _col0, _col1, _col6 Select Operator - Group By Operator - aggregations: count() - mode: hash - outputColumnNames: _col0 - Reduce Output Operator - sort order: - tag: -1 - value expressions: _col0 (type: bigint) - auto parallelism: false - Path -> Alias: -#### A masked pattern was here #### - Path -> Partition: -#### A masked pattern was here #### - Partition - base file name: ds=2008-04-08 - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-08 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 4 - bucket_field_name key - columns key,value - columns.comments - columns.types string:string -#### A masked pattern was here #### - name default.bucket_big - numFiles 4 - numRows 0 - partition_columns ds - partition_columns.types string - rawDataSize 0 - serialization.ddl struct bucket_big { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 5812 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - SORTBUCKETCOLSPREFIX TRUE - bucket_count 4 - bucket_field_name key - columns key,value - columns.comments - columns.types string:string -#### A masked pattern was here #### - name default.bucket_big - partition_columns ds - partition_columns.types string - serialization.ddl struct bucket_big { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_big - name: default.bucket_big -#### A masked pattern was here #### - Partition - base file name: ds=2008-04-09 - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 2008-04-09 - properties: - COLUMN_STATS_ACCURATE true - bucket_count 4 - bucket_field_name key - columns key,value - columns.comments - columns.types string:string -#### A masked pattern was here #### - name default.bucket_big - numFiles 4 - numRows 0 - partition_columns ds - partition_columns.types string - rawDataSize 0 - serialization.ddl struct bucket_big { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 5812 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - SORTBUCKETCOLSPREFIX TRUE - bucket_count 4 - bucket_field_name key - columns key,value - columns.comments - columns.types string:string -#### A masked pattern was here #### - name default.bucket_big - partition_columns ds - partition_columns.types string - serialization.ddl struct bucket_big { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.bucket_big - name: default.bucket_big - Truncated Path -> Alias: - /bucket_big/ds=2008-04-08 [a] - /bucket_big/ds=2008-04-09 [a] - Needs Tagging: false - Reduce Operator Tree: - Group By Operator - aggregations: count(VALUE._col0) - mode: mergepartial - outputColumnNames: _col0 - Select Operator - expressions: _col0 (type: bigint) - outputColumnNames: _col0 - File Output Operator - compressed: false - GlobalTableId: 0 -#### A masked pattern was here #### - NumFilesPerFileSink: 1 -#### 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 - columns.types bigint - 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 + expressions: _col0 (type: int), _col1 (type: string), _col6 (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 Stage: Stage-0 Fetch Operator @@ -1382,20 +152,499 @@ STAGE PLANS: Processor Tree: ListSink -PREHOOK: query: select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +PREHOOK: query: select a.key, a.value, b.value +from tab a join tab_part b on a.key = b.key PREHOOK: type: QUERY -PREHOOK: Input: default@bucket_big -PREHOOK: Input: default@bucket_big@ds=2008-04-08 -PREHOOK: Input: default@bucket_big@ds=2008-04-09 -PREHOOK: Input: default@bucket_small -PREHOOK: Input: default@bucket_small@ds=2008-04-08 +PREHOOK: Input: default@tab +PREHOOK: Input: default@tab@ds=2008-04-08 +PREHOOK: Input: default@tab_part +PREHOOK: Input: default@tab_part@ds=2008-04-08 #### A masked pattern was here #### -POSTHOOK: query: select count(*) FROM bucket_big a JOIN bucket_small b ON a.key = b.key +POSTHOOK: query: select a.key, a.value, b.value +from tab a join tab_part b on a.key = b.key POSTHOOK: type: QUERY -POSTHOOK: Input: default@bucket_big -POSTHOOK: Input: default@bucket_big@ds=2008-04-08 -POSTHOOK: Input: default@bucket_big@ds=2008-04-09 -POSTHOOK: Input: default@bucket_small -POSTHOOK: Input: default@bucket_small@ds=2008-04-08 -#### A masked pattern was here #### -38 +POSTHOOK: Input: default@tab +POSTHOOK: Input: default@tab@ds=2008-04-08 +POSTHOOK: Input: default@tab_part +POSTHOOK: Input: default@tab_part@ds=2008-04-08 +#### A masked pattern was here #### +0 val_0 val_0 +0 val_0 val_0 +0 val_0 val_0 +0 val_0 val_0 +0 val_0 val_0 +0 val_0 val_0 +0 val_0 val_0 +0 val_0 val_0 +0 val_0 val_0 +4 val_4 val_4 +8 val_8 val_8 +20 val_20 val_20 +24 val_24 val_24 +24 val_24 val_24 +24 val_24 val_24 +24 val_24 val_24 +28 val_28 val_28 +44 val_44 val_44 +64 val_64 val_64 +80 val_80 val_80 +84 val_84 val_84 +84 val_84 val_84 +84 val_84 val_84 +84 val_84 val_84 +116 val_116 val_116 +136 val_136 val_136 +152 val_152 val_152 +152 val_152 val_152 +152 val_152 val_152 +152 val_152 val_152 +156 val_156 val_156 +172 val_172 val_172 +172 val_172 val_172 +172 val_172 val_172 +172 val_172 val_172 +176 val_176 val_176 +176 val_176 val_176 +176 val_176 val_176 +176 val_176 val_176 +192 val_192 val_192 +196 val_196 val_196 +200 val_200 val_200 +200 val_200 val_200 +200 val_200 val_200 +200 val_200 val_200 +208 val_208 val_208 +208 val_208 val_208 +208 val_208 val_208 +208 val_208 val_208 +208 val_208 val_208 +208 val_208 val_208 +208 val_208 val_208 +208 val_208 val_208 +208 val_208 val_208 +224 val_224 val_224 +224 val_224 val_224 +224 val_224 val_224 +224 val_224 val_224 +228 val_228 val_228 +244 val_244 val_244 +248 val_248 val_248 +260 val_260 val_260 +280 val_280 val_280 +280 val_280 val_280 +280 val_280 val_280 +280 val_280 val_280 +284 val_284 val_284 +288 val_288 val_288 +288 val_288 val_288 +288 val_288 val_288 +288 val_288 val_288 +316 val_316 val_316 +316 val_316 val_316 +316 val_316 val_316 +316 val_316 val_316 +316 val_316 val_316 +316 val_316 val_316 +316 val_316 val_316 +316 val_316 val_316 +316 val_316 val_316 +332 val_332 val_332 +336 val_336 val_336 +356 val_356 val_356 +392 val_392 val_392 +396 val_396 val_396 +396 val_396 val_396 +396 val_396 val_396 +396 val_396 val_396 +396 val_396 val_396 +396 val_396 val_396 +396 val_396 val_396 +396 val_396 val_396 +396 val_396 val_396 +400 val_400 val_400 +404 val_404 val_404 +404 val_404 val_404 +404 val_404 val_404 +404 val_404 val_404 +424 val_424 val_424 +424 val_424 val_424 +424 val_424 val_424 +424 val_424 val_424 +444 val_444 val_444 +448 val_448 val_448 +460 val_460 val_460 +468 val_468 val_468 +468 val_468 val_468 +468 val_468 val_468 +468 val_468 val_468 +468 val_468 val_468 +468 val_468 val_468 +468 val_468 val_468 +468 val_468 val_468 +468 val_468 val_468 +468 val_468 val_468 +468 val_468 val_468 +468 val_468 val_468 +468 val_468 val_468 +468 val_468 val_468 +468 val_468 val_468 +468 val_468 val_468 +480 val_480 val_480 +480 val_480 val_480 +480 val_480 val_480 +480 val_480 val_480 +480 val_480 val_480 +480 val_480 val_480 +480 val_480 val_480 +480 val_480 val_480 +480 val_480 val_480 +484 val_484 val_484 +17 val_17 val_17 +33 val_33 val_33 +37 val_37 val_37 +37 val_37 val_37 +37 val_37 val_37 +37 val_37 val_37 +53 val_53 val_53 +57 val_57 val_57 +77 val_77 val_77 +97 val_97 val_97 +97 val_97 val_97 +97 val_97 val_97 +97 val_97 val_97 +105 val_105 val_105 +125 val_125 val_125 +125 val_125 val_125 +125 val_125 val_125 +125 val_125 val_125 +129 val_129 val_129 +129 val_129 val_129 +129 val_129 val_129 +129 val_129 val_129 +145 val_145 val_145 +149 val_149 val_149 +149 val_149 val_149 +149 val_149 val_149 +149 val_149 val_149 +165 val_165 val_165 +165 val_165 val_165 +165 val_165 val_165 +165 val_165 val_165 +169 val_169 val_169 +169 val_169 val_169 +169 val_169 val_169 +169 val_169 val_169 +169 val_169 val_169 +169 val_169 val_169 +169 val_169 val_169 +169 val_169 val_169 +169 val_169 val_169 +169 val_169 val_169 +169 val_169 val_169 +169 val_169 val_169 +169 val_169 val_169 +169 val_169 val_169 +169 val_169 val_169 +169 val_169 val_169 +181 val_181 val_181 +189 val_189 val_189 +213 val_213 val_213 +213 val_213 val_213 +213 val_213 val_213 +213 val_213 val_213 +217 val_217 val_217 +217 val_217 val_217 +217 val_217 val_217 +217 val_217 val_217 +233 val_233 val_233 +233 val_233 val_233 +233 val_233 val_233 +233 val_233 val_233 +237 val_237 val_237 +237 val_237 val_237 +237 val_237 val_237 +237 val_237 val_237 +257 val_257 val_257 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +273 val_273 val_273 +277 val_277 val_277 +277 val_277 val_277 +277 val_277 val_277 +277 val_277 val_277 +277 val_277 val_277 +277 val_277 val_277 +277 val_277 val_277 +277 val_277 val_277 +277 val_277 val_277 +277 val_277 val_277 +277 val_277 val_277 +277 val_277 val_277 +277 val_277 val_277 +277 val_277 val_277 +277 val_277 val_277 +277 val_277 val_277 +305 val_305 val_305 +309 val_309 val_309 +309 val_309 val_309 +309 val_309 val_309 +309 val_309 val_309 +321 val_321 val_321 +321 val_321 val_321 +321 val_321 val_321 +321 val_321 val_321 +325 val_325 val_325 +325 val_325 val_325 +325 val_325 val_325 +325 val_325 val_325 +341 val_341 val_341 +345 val_345 val_345 +365 val_365 val_365 +369 val_369 val_369 +369 val_369 val_369 +369 val_369 val_369 +369 val_369 val_369 +369 val_369 val_369 +369 val_369 val_369 +369 val_369 val_369 +369 val_369 val_369 +369 val_369 val_369 +389 val_389 val_389 +413 val_413 val_413 +413 val_413 val_413 +413 val_413 val_413 +413 val_413 val_413 +417 val_417 val_417 +417 val_417 val_417 +417 val_417 val_417 +417 val_417 val_417 +417 val_417 val_417 +417 val_417 val_417 +417 val_417 val_417 +417 val_417 val_417 +417 val_417 val_417 +437 val_437 val_437 +453 val_453 val_453 +457 val_457 val_457 +477 val_477 val_477 +493 val_493 val_493 +497 val_497 val_497 +2 val_2 val_2 +26 val_26 val_26 +26 val_26 val_26 +26 val_26 val_26 +26 val_26 val_26 +42 val_42 val_42 +42 val_42 val_42 +42 val_42 val_42 +42 val_42 val_42 +66 val_66 val_66 +82 val_82 val_82 +86 val_86 val_86 +114 val_114 val_114 +118 val_118 val_118 +118 val_118 val_118 +118 val_118 val_118 +118 val_118 val_118 +134 val_134 val_134 +134 val_134 val_134 +134 val_134 val_134 +134 val_134 val_134 +138 val_138 val_138 +138 val_138 val_138 +138 val_138 val_138 +138 val_138 val_138 +138 val_138 val_138 +138 val_138 val_138 +138 val_138 val_138 +138 val_138 val_138 +138 val_138 val_138 +138 val_138 val_138 +138 val_138 val_138 +138 val_138 val_138 +138 val_138 val_138 +138 val_138 val_138 +138 val_138 val_138 +138 val_138 val_138 +150 val_150 val_150 +158 val_158 val_158 +170 val_170 val_170 +174 val_174 val_174 +174 val_174 val_174 +174 val_174 val_174 +174 val_174 val_174 +178 val_178 val_178 +190 val_190 val_190 +194 val_194 val_194 +202 val_202 val_202 +222 val_222 val_222 +226 val_226 val_226 +242 val_242 val_242 +242 val_242 val_242 +242 val_242 val_242 +242 val_242 val_242 +262 val_262 val_262 +266 val_266 val_266 +282 val_282 val_282 +282 val_282 val_282 +282 val_282 val_282 +282 val_282 val_282 +286 val_286 val_286 +310 val_310 val_310 +318 val_318 val_318 +318 val_318 val_318 +318 val_318 val_318 +318 val_318 val_318 +318 val_318 val_318 +318 val_318 val_318 +318 val_318 val_318 +318 val_318 val_318 +318 val_318 val_318 +338 val_338 val_338 +374 val_374 val_374 +378 val_378 val_378 +394 val_394 val_394 +402 val_402 val_402 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +406 val_406 val_406 +446 val_446 val_446 +462 val_462 val_462 +462 val_462 val_462 +462 val_462 val_462 +462 val_462 val_462 +466 val_466 val_466 +466 val_466 val_466 +466 val_466 val_466 +466 val_466 val_466 +466 val_466 val_466 +466 val_466 val_466 +466 val_466 val_466 +466 val_466 val_466 +466 val_466 val_466 +482 val_482 val_482 +11 val_11 val_11 +15 val_15 val_15 +15 val_15 val_15 +15 val_15 val_15 +15 val_15 val_15 +19 val_19 val_19 +35 val_35 val_35 +35 val_35 val_35 +35 val_35 val_35 +35 val_35 val_35 +35 val_35 val_35 +35 val_35 val_35 +35 val_35 val_35 +35 val_35 val_35 +35 val_35 val_35 +51 val_51 val_51 +51 val_51 val_51 +51 val_51 val_51 +51 val_51 val_51 +95 val_95 val_95 +95 val_95 val_95 +95 val_95 val_95 +95 val_95 val_95 +103 val_103 val_103 +103 val_103 val_103 +103 val_103 val_103 +103 val_103 val_103 +143 val_143 val_143 +163 val_163 val_163 +167 val_167 val_167 +167 val_167 val_167 +167 val_167 val_167 +167 val_167 val_167 +167 val_167 val_167 +167 val_167 val_167 +167 val_167 val_167 +167 val_167 val_167 +167 val_167 val_167 +183 val_183 val_183 +187 val_187 val_187 +187 val_187 val_187 +187 val_187 val_187 +187 val_187 val_187 +187 val_187 val_187 +187 val_187 val_187 +187 val_187 val_187 +187 val_187 val_187 +187 val_187 val_187 +219 val_219 val_219 +219 val_219 val_219 +219 val_219 val_219 +219 val_219 val_219 +235 val_235 val_235 +239 val_239 val_239 +239 val_239 val_239 +239 val_239 val_239 +239 val_239 val_239 +255 val_255 val_255 +255 val_255 val_255 +255 val_255 val_255 +255 val_255 val_255 +275 val_275 val_275 +291 val_291 val_291 +307 val_307 val_307 +307 val_307 val_307 +307 val_307 val_307 +307 val_307 val_307 +323 val_323 val_323 +327 val_327 val_327 +327 val_327 val_327 +327 val_327 val_327 +327 val_327 val_327 +327 val_327 val_327 +327 val_327 val_327 +327 val_327 val_327 +327 val_327 val_327 +327 val_327 val_327 +367 val_367 val_367 +367 val_367 val_367 +367 val_367 val_367 +367 val_367 val_367 +411 val_411 val_411 +419 val_419 val_419 +431 val_431 val_431 +431 val_431 val_431 +431 val_431 val_431 +431 val_431 val_431 +431 val_431 val_431 +431 val_431 val_431 +431 val_431 val_431 +431 val_431 val_431 +431 val_431 val_431 +435 val_435 val_435 +439 val_439 val_439 +439 val_439 val_439 +439 val_439 val_439 +439 val_439 val_439 +455 val_455 val_455 +459 val_459 val_459 +459 val_459 val_459 +459 val_459 val_459 +459 val_459 val_459 +475 val_475 val_475 +479 val_479 val_479 +491 val_491 val_491 +495 val_495 val_495 diff --git ql/src/test/results/clientpositive/tez/tez_bmj_schema_evolution.q.out ql/src/test/results/clientpositive/tez/tez_bmj_schema_evolution.q.out index 047ce3d..2a1e6f8 100644 --- ql/src/test/results/clientpositive/tez/tez_bmj_schema_evolution.q.out +++ ql/src/test/results/clientpositive/tez/tez_bmj_schema_evolution.q.out @@ -2211,3 +2211,2216 @@ POSTHOOK: Input: default@test@p=2 498 val_498 498 val_498 498 val_498 +PREHOOK: query: create table test (key int, value string) partitioned by (p int) clustered by (key) into 2 buckets stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +POSTHOOK: query: create table test (key int, value string) partitioned by (p int) clustered by (key) into 2 buckets stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@test +PREHOOK: query: create table test1 (key int, value string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +POSTHOOK: query: create table test1 (key int, value string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@test1 +PREHOOK: query: insert into table test partition (p=1) select * from src +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@test@p=1 +POSTHOOK: query: insert into table test partition (p=1) select * from src +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@test@p=1 +POSTHOOK: Lineage: test PARTITION(p=1).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: test PARTITION(p=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: alter table test set fileformat orc +PREHOOK: type: ALTERTABLE_FILEFORMAT +PREHOOK: Input: default@test +PREHOOK: Output: default@test +POSTHOOK: query: alter table test set fileformat orc +POSTHOOK: type: ALTERTABLE_FILEFORMAT +POSTHOOK: Input: default@test +POSTHOOK: Output: default@test +PREHOOK: query: insert into table test partition (p=2) select * from src +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@test@p=2 +POSTHOOK: query: insert into table test partition (p=2) select * from src +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@test@p=2 +POSTHOOK: Lineage: test PARTITION(p=2).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: test PARTITION(p=2).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: insert into table test1 select * from src +PREHOOK: type: QUERY +PREHOOK: Input: default@src +PREHOOK: Output: default@test1 +POSTHOOK: query: insert into table test1 select * from src +POSTHOOK: type: QUERY +POSTHOOK: Input: default@src +POSTHOOK: Output: default@test1 +POSTHOOK: Lineage: test1.key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: test1.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: describe test +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@test +POSTHOOK: query: describe test +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@test +key int +value string +p int + +# Partition Information +# col_name data_type comment + +p int +PREHOOK: query: explain select test.key, test.value from test join test1 on (test.key = test1.key) order by test.key +PREHOOK: type: QUERY +POSTHOOK: query: explain select test.key, test.value from test join test1 on (test.key = test1.key) order by test.key +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Tez + Edges: + Map 2 <- Map 1 (CUSTOM_EDGE) + Reducer 3 <- Map 2 (SIMPLE_EDGE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: test1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: key (type: int) + sort order: + + Map-reduce partition columns: key (type: int) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Map 2 + Map Operator Tree: + TableScan + alias: test + Statistics: Num rows: 1000 Data size: 52312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: key is not null (type: boolean) + Statistics: Num rows: 500 Data size: 26156 Basic stats: COMPLETE Column stats: NONE + Map Join Operator + condition map: + Inner Join 0 to 1 + condition expressions: + 0 {key} {value} + 1 + keys: + 0 key (type: int) + 1 key (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 550 Data size: 28771 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: int), _col1 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 550 Data size: 28771 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Statistics: Num rows: 550 Data size: 28771 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: string) + Reducer 3 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 550 Data size: 28771 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 550 Data size: 28771 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 + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select test.key, test.value from test join test1 on (test.key = test1.key) order by test.key +PREHOOK: type: QUERY +PREHOOK: Input: default@test +PREHOOK: Input: default@test1 +PREHOOK: Input: default@test@p=1 +PREHOOK: Input: default@test@p=2 +#### A masked pattern was here #### +POSTHOOK: query: select test.key, test.value from test join test1 on (test.key = test1.key) order by test.key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@test +POSTHOOK: Input: default@test1 +POSTHOOK: Input: default@test@p=1 +POSTHOOK: Input: default@test@p=2 +#### A masked pattern was here #### +0 val_0 +0 val_0 +0 val_0 +0 val_0 +0 val_0 +0 val_0 +0 val_0 +0 val_0 +0 val_0 +0 val_0 +0 val_0 +0 val_0 +0 val_0 +0 val_0 +0 val_0 +0 val_0 +0 val_0 +0 val_0 +2 val_2 +2 val_2 +4 val_4 +4 val_4 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +5 val_5 +8 val_8 +8 val_8 +9 val_9 +9 val_9 +10 val_10 +10 val_10 +11 val_11 +11 val_11 +12 val_12 +12 val_12 +12 val_12 +12 val_12 +12 val_12 +12 val_12 +12 val_12 +12 val_12 +15 val_15 +15 val_15 +15 val_15 +15 val_15 +15 val_15 +15 val_15 +15 val_15 +15 val_15 +17 val_17 +17 val_17 +18 val_18 +18 val_18 +18 val_18 +18 val_18 +18 val_18 +18 val_18 +18 val_18 +18 val_18 +19 val_19 +19 val_19 +20 val_20 +20 val_20 +24 val_24 +24 val_24 +24 val_24 +24 val_24 +24 val_24 +24 val_24 +24 val_24 +24 val_24 +26 val_26 +26 val_26 +26 val_26 +26 val_26 +26 val_26 +26 val_26 +26 val_26 +26 val_26 +27 val_27 +27 val_27 +28 val_28 +28 val_28 +30 val_30 +30 val_30 +33 val_33 +33 val_33 +34 val_34 +34 val_34 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +35 val_35 +37 val_37 +37 val_37 +37 val_37 +37 val_37 +37 val_37 +37 val_37 +37 val_37 +37 val_37 +41 val_41 +41 val_41 +42 val_42 +42 val_42 +42 val_42 +42 val_42 +42 val_42 +42 val_42 +42 val_42 +42 val_42 +43 val_43 +43 val_43 +44 val_44 +44 val_44 +47 val_47 +47 val_47 +51 val_51 +51 val_51 +51 val_51 +51 val_51 +51 val_51 +51 val_51 +51 val_51 +51 val_51 +53 val_53 +53 val_53 +54 val_54 +54 val_54 +57 val_57 +57 val_57 +58 val_58 +58 val_58 +58 val_58 +58 val_58 +58 val_58 +58 val_58 +58 val_58 +58 val_58 +64 val_64 +64 val_64 +65 val_65 +65 val_65 +66 val_66 +66 val_66 +67 val_67 +67 val_67 +67 val_67 +67 val_67 +67 val_67 +67 val_67 +67 val_67 +67 val_67 +69 val_69 +69 val_69 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +70 val_70 +72 val_72 +72 val_72 +72 val_72 +72 val_72 +72 val_72 +72 val_72 +72 val_72 +72 val_72 +74 val_74 +74 val_74 +76 val_76 +76 val_76 +76 val_76 +76 val_76 +76 val_76 +76 val_76 +76 val_76 +76 val_76 +77 val_77 +77 val_77 +78 val_78 +78 val_78 +80 val_80 +80 val_80 +82 val_82 +82 val_82 +83 val_83 +83 val_83 +83 val_83 +83 val_83 +83 val_83 +83 val_83 +83 val_83 +83 val_83 +84 val_84 +84 val_84 +84 val_84 +84 val_84 +84 val_84 +84 val_84 +84 val_84 +84 val_84 +85 val_85 +85 val_85 +86 val_86 +86 val_86 +87 val_87 +87 val_87 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +90 val_90 +92 val_92 +92 val_92 +95 val_95 +95 val_95 +95 val_95 +95 val_95 +95 val_95 +95 val_95 +95 val_95 +95 val_95 +96 val_96 +96 val_96 +97 val_97 +97 val_97 +97 val_97 +97 val_97 +97 val_97 +97 val_97 +97 val_97 +97 val_97 +98 val_98 +98 val_98 +98 val_98 +98 val_98 +98 val_98 +98 val_98 +98 val_98 +98 val_98 +100 val_100 +100 val_100 +100 val_100 +100 val_100 +100 val_100 +100 val_100 +100 val_100 +100 val_100 +103 val_103 +103 val_103 +103 val_103 +103 val_103 +103 val_103 +103 val_103 +103 val_103 +103 val_103 +104 val_104 +104 val_104 +104 val_104 +104 val_104 +104 val_104 +104 val_104 +104 val_104 +104 val_104 +105 val_105 +105 val_105 +111 val_111 +111 val_111 +113 val_113 +113 val_113 +113 val_113 +113 val_113 +113 val_113 +113 val_113 +113 val_113 +113 val_113 +114 val_114 +114 val_114 +116 val_116 +116 val_116 +118 val_118 +118 val_118 +118 val_118 +118 val_118 +118 val_118 +118 val_118 +118 val_118 +118 val_118 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +119 val_119 +120 val_120 +120 val_120 +120 val_120 +120 val_120 +120 val_120 +120 val_120 +120 val_120 +120 val_120 +125 val_125 +125 val_125 +125 val_125 +125 val_125 +125 val_125 +125 val_125 +125 val_125 +125 val_125 +126 val_126 +126 val_126 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +128 val_128 +129 val_129 +129 val_129 +129 val_129 +129 val_129 +129 val_129 +129 val_129 +129 val_129 +129 val_129 +131 val_131 +131 val_131 +133 val_133 +133 val_133 +134 val_134 +134 val_134 +134 val_134 +134 val_134 +134 val_134 +134 val_134 +134 val_134 +134 val_134 +136 val_136 +136 val_136 +137 val_137 +137 val_137 +137 val_137 +137 val_137 +137 val_137 +137 val_137 +137 val_137 +137 val_137 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +138 val_138 +143 val_143 +143 val_143 +145 val_145 +145 val_145 +146 val_146 +146 val_146 +146 val_146 +146 val_146 +146 val_146 +146 val_146 +146 val_146 +146 val_146 +149 val_149 +149 val_149 +149 val_149 +149 val_149 +149 val_149 +149 val_149 +149 val_149 +149 val_149 +150 val_150 +150 val_150 +152 val_152 +152 val_152 +152 val_152 +152 val_152 +152 val_152 +152 val_152 +152 val_152 +152 val_152 +153 val_153 +153 val_153 +155 val_155 +155 val_155 +156 val_156 +156 val_156 +157 val_157 +157 val_157 +158 val_158 +158 val_158 +160 val_160 +160 val_160 +162 val_162 +162 val_162 +163 val_163 +163 val_163 +164 val_164 +164 val_164 +164 val_164 +164 val_164 +164 val_164 +164 val_164 +164 val_164 +164 val_164 +165 val_165 +165 val_165 +165 val_165 +165 val_165 +165 val_165 +165 val_165 +165 val_165 +165 val_165 +166 val_166 +166 val_166 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +167 val_167 +168 val_168 +168 val_168 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +169 val_169 +170 val_170 +170 val_170 +172 val_172 +172 val_172 +172 val_172 +172 val_172 +172 val_172 +172 val_172 +172 val_172 +172 val_172 +174 val_174 +174 val_174 +174 val_174 +174 val_174 +174 val_174 +174 val_174 +174 val_174 +174 val_174 +175 val_175 +175 val_175 +175 val_175 +175 val_175 +175 val_175 +175 val_175 +175 val_175 +175 val_175 +176 val_176 +176 val_176 +176 val_176 +176 val_176 +176 val_176 +176 val_176 +176 val_176 +176 val_176 +177 val_177 +177 val_177 +178 val_178 +178 val_178 +179 val_179 +179 val_179 +179 val_179 +179 val_179 +179 val_179 +179 val_179 +179 val_179 +179 val_179 +180 val_180 +180 val_180 +181 val_181 +181 val_181 +183 val_183 +183 val_183 +186 val_186 +186 val_186 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +187 val_187 +189 val_189 +189 val_189 +190 val_190 +190 val_190 +191 val_191 +191 val_191 +191 val_191 +191 val_191 +191 val_191 +191 val_191 +191 val_191 +191 val_191 +192 val_192 +192 val_192 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +193 val_193 +194 val_194 +194 val_194 +195 val_195 +195 val_195 +195 val_195 +195 val_195 +195 val_195 +195 val_195 +195 val_195 +195 val_195 +196 val_196 +196 val_196 +197 val_197 +197 val_197 +197 val_197 +197 val_197 +197 val_197 +197 val_197 +197 val_197 +197 val_197 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +199 val_199 +200 val_200 +200 val_200 +200 val_200 +200 val_200 +200 val_200 +200 val_200 +200 val_200 +200 val_200 +201 val_201 +201 val_201 +202 val_202 +202 val_202 +203 val_203 +203 val_203 +203 val_203 +203 val_203 +203 val_203 +203 val_203 +203 val_203 +203 val_203 +205 val_205 +205 val_205 +205 val_205 +205 val_205 +205 val_205 +205 val_205 +205 val_205 +205 val_205 +207 val_207 +207 val_207 +207 val_207 +207 val_207 +207 val_207 +207 val_207 +207 val_207 +207 val_207 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +208 val_208 +209 val_209 +209 val_209 +209 val_209 +209 val_209 +209 val_209 +209 val_209 +209 val_209 +209 val_209 +213 val_213 +213 val_213 +213 val_213 +213 val_213 +213 val_213 +213 val_213 +213 val_213 +213 val_213 +214 val_214 +214 val_214 +216 val_216 +216 val_216 +216 val_216 +216 val_216 +216 val_216 +216 val_216 +216 val_216 +216 val_216 +217 val_217 +217 val_217 +217 val_217 +217 val_217 +217 val_217 +217 val_217 +217 val_217 +217 val_217 +218 val_218 +218 val_218 +219 val_219 +219 val_219 +219 val_219 +219 val_219 +219 val_219 +219 val_219 +219 val_219 +219 val_219 +221 val_221 +221 val_221 +221 val_221 +221 val_221 +221 val_221 +221 val_221 +221 val_221 +221 val_221 +222 val_222 +222 val_222 +223 val_223 +223 val_223 +223 val_223 +223 val_223 +223 val_223 +223 val_223 +223 val_223 +223 val_223 +224 val_224 +224 val_224 +224 val_224 +224 val_224 +224 val_224 +224 val_224 +224 val_224 +224 val_224 +226 val_226 +226 val_226 +228 val_228 +228 val_228 +229 val_229 +229 val_229 +229 val_229 +229 val_229 +229 val_229 +229 val_229 +229 val_229 +229 val_229 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +230 val_230 +233 val_233 +233 val_233 +233 val_233 +233 val_233 +233 val_233 +233 val_233 +233 val_233 +233 val_233 +235 val_235 +235 val_235 +237 val_237 +237 val_237 +237 val_237 +237 val_237 +237 val_237 +237 val_237 +237 val_237 +237 val_237 +238 val_238 +238 val_238 +238 val_238 +238 val_238 +238 val_238 +238 val_238 +238 val_238 +238 val_238 +239 val_239 +239 val_239 +239 val_239 +239 val_239 +239 val_239 +239 val_239 +239 val_239 +239 val_239 +241 val_241 +241 val_241 +242 val_242 +242 val_242 +242 val_242 +242 val_242 +242 val_242 +242 val_242 +242 val_242 +242 val_242 +244 val_244 +244 val_244 +247 val_247 +247 val_247 +248 val_248 +248 val_248 +249 val_249 +249 val_249 +252 val_252 +252 val_252 +255 val_255 +255 val_255 +255 val_255 +255 val_255 +255 val_255 +255 val_255 +255 val_255 +255 val_255 +256 val_256 +256 val_256 +256 val_256 +256 val_256 +256 val_256 +256 val_256 +256 val_256 +256 val_256 +257 val_257 +257 val_257 +258 val_258 +258 val_258 +260 val_260 +260 val_260 +262 val_262 +262 val_262 +263 val_263 +263 val_263 +265 val_265 +265 val_265 +265 val_265 +265 val_265 +265 val_265 +265 val_265 +265 val_265 +265 val_265 +266 val_266 +266 val_266 +272 val_272 +272 val_272 +272 val_272 +272 val_272 +272 val_272 +272 val_272 +272 val_272 +272 val_272 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +273 val_273 +274 val_274 +274 val_274 +275 val_275 +275 val_275 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +277 val_277 +278 val_278 +278 val_278 +278 val_278 +278 val_278 +278 val_278 +278 val_278 +278 val_278 +278 val_278 +280 val_280 +280 val_280 +280 val_280 +280 val_280 +280 val_280 +280 val_280 +280 val_280 +280 val_280 +281 val_281 +281 val_281 +281 val_281 +281 val_281 +281 val_281 +281 val_281 +281 val_281 +281 val_281 +282 val_282 +282 val_282 +282 val_282 +282 val_282 +282 val_282 +282 val_282 +282 val_282 +282 val_282 +283 val_283 +283 val_283 +284 val_284 +284 val_284 +285 val_285 +285 val_285 +286 val_286 +286 val_286 +287 val_287 +287 val_287 +288 val_288 +288 val_288 +288 val_288 +288 val_288 +288 val_288 +288 val_288 +288 val_288 +288 val_288 +289 val_289 +289 val_289 +291 val_291 +291 val_291 +292 val_292 +292 val_292 +296 val_296 +296 val_296 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +298 val_298 +302 val_302 +302 val_302 +305 val_305 +305 val_305 +306 val_306 +306 val_306 +307 val_307 +307 val_307 +307 val_307 +307 val_307 +307 val_307 +307 val_307 +307 val_307 +307 val_307 +308 val_308 +308 val_308 +309 val_309 +309 val_309 +309 val_309 +309 val_309 +309 val_309 +309 val_309 +309 val_309 +309 val_309 +310 val_310 +310 val_310 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +311 val_311 +315 val_315 +315 val_315 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +316 val_316 +317 val_317 +317 val_317 +317 val_317 +317 val_317 +317 val_317 +317 val_317 +317 val_317 +317 val_317 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +318 val_318 +321 val_321 +321 val_321 +321 val_321 +321 val_321 +321 val_321 +321 val_321 +321 val_321 +321 val_321 +322 val_322 +322 val_322 +322 val_322 +322 val_322 +322 val_322 +322 val_322 +322 val_322 +322 val_322 +323 val_323 +323 val_323 +325 val_325 +325 val_325 +325 val_325 +325 val_325 +325 val_325 +325 val_325 +325 val_325 +325 val_325 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +327 val_327 +331 val_331 +331 val_331 +331 val_331 +331 val_331 +331 val_331 +331 val_331 +331 val_331 +331 val_331 +332 val_332 +332 val_332 +333 val_333 +333 val_333 +333 val_333 +333 val_333 +333 val_333 +333 val_333 +333 val_333 +333 val_333 +335 val_335 +335 val_335 +336 val_336 +336 val_336 +338 val_338 +338 val_338 +339 val_339 +339 val_339 +341 val_341 +341 val_341 +342 val_342 +342 val_342 +342 val_342 +342 val_342 +342 val_342 +342 val_342 +342 val_342 +342 val_342 +344 val_344 +344 val_344 +344 val_344 +344 val_344 +344 val_344 +344 val_344 +344 val_344 +344 val_344 +345 val_345 +345 val_345 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +348 val_348 +351 val_351 +351 val_351 +353 val_353 +353 val_353 +353 val_353 +353 val_353 +353 val_353 +353 val_353 +353 val_353 +353 val_353 +356 val_356 +356 val_356 +360 val_360 +360 val_360 +362 val_362 +362 val_362 +364 val_364 +364 val_364 +365 val_365 +365 val_365 +366 val_366 +366 val_366 +367 val_367 +367 val_367 +367 val_367 +367 val_367 +367 val_367 +367 val_367 +367 val_367 +367 val_367 +368 val_368 +368 val_368 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +369 val_369 +373 val_373 +373 val_373 +374 val_374 +374 val_374 +375 val_375 +375 val_375 +377 val_377 +377 val_377 +378 val_378 +378 val_378 +379 val_379 +379 val_379 +382 val_382 +382 val_382 +382 val_382 +382 val_382 +382 val_382 +382 val_382 +382 val_382 +382 val_382 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +384 val_384 +386 val_386 +386 val_386 +389 val_389 +389 val_389 +392 val_392 +392 val_392 +393 val_393 +393 val_393 +394 val_394 +394 val_394 +395 val_395 +395 val_395 +395 val_395 +395 val_395 +395 val_395 +395 val_395 +395 val_395 +395 val_395 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +396 val_396 +397 val_397 +397 val_397 +397 val_397 +397 val_397 +397 val_397 +397 val_397 +397 val_397 +397 val_397 +399 val_399 +399 val_399 +399 val_399 +399 val_399 +399 val_399 +399 val_399 +399 val_399 +399 val_399 +400 val_400 +400 val_400 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +401 val_401 +402 val_402 +402 val_402 +403 val_403 +403 val_403 +403 val_403 +403 val_403 +403 val_403 +403 val_403 +403 val_403 +403 val_403 +403 val_403 +403 val_403 +403 val_403 +403 val_403 +403 val_403 +403 val_403 +403 val_403 +403 val_403 +403 val_403 +403 val_403 +404 val_404 +404 val_404 +404 val_404 +404 val_404 +404 val_404 +404 val_404 +404 val_404 +404 val_404 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +406 val_406 +407 val_407 +407 val_407 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +409 val_409 +411 val_411 +411 val_411 +413 val_413 +413 val_413 +413 val_413 +413 val_413 +413 val_413 +413 val_413 +413 val_413 +413 val_413 +414 val_414 +414 val_414 +414 val_414 +414 val_414 +414 val_414 +414 val_414 +414 val_414 +414 val_414 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +417 val_417 +418 val_418 +418 val_418 +419 val_419 +419 val_419 +421 val_421 +421 val_421 +424 val_424 +424 val_424 +424 val_424 +424 val_424 +424 val_424 +424 val_424 +424 val_424 +424 val_424 +427 val_427 +427 val_427 +429 val_429 +429 val_429 +429 val_429 +429 val_429 +429 val_429 +429 val_429 +429 val_429 +429 val_429 +430 val_430 +430 val_430 +430 val_430 +430 val_430 +430 val_430 +430 val_430 +430 val_430 +430 val_430 +430 val_430 +430 val_430 +430 val_430 +430 val_430 +430 val_430 +430 val_430 +430 val_430 +430 val_430 +430 val_430 +430 val_430 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +431 val_431 +432 val_432 +432 val_432 +435 val_435 +435 val_435 +436 val_436 +436 val_436 +437 val_437 +437 val_437 +438 val_438 +438 val_438 +438 val_438 +438 val_438 +438 val_438 +438 val_438 +438 val_438 +438 val_438 +438 val_438 +438 val_438 +438 val_438 +438 val_438 +438 val_438 +438 val_438 +438 val_438 +438 val_438 +438 val_438 +438 val_438 +439 val_439 +439 val_439 +439 val_439 +439 val_439 +439 val_439 +439 val_439 +439 val_439 +439 val_439 +443 val_443 +443 val_443 +444 val_444 +444 val_444 +446 val_446 +446 val_446 +448 val_448 +448 val_448 +449 val_449 +449 val_449 +452 val_452 +452 val_452 +453 val_453 +453 val_453 +454 val_454 +454 val_454 +454 val_454 +454 val_454 +454 val_454 +454 val_454 +454 val_454 +454 val_454 +454 val_454 +454 val_454 +454 val_454 +454 val_454 +454 val_454 +454 val_454 +454 val_454 +454 val_454 +454 val_454 +454 val_454 +455 val_455 +455 val_455 +457 val_457 +457 val_457 +458 val_458 +458 val_458 +458 val_458 +458 val_458 +458 val_458 +458 val_458 +458 val_458 +458 val_458 +459 val_459 +459 val_459 +459 val_459 +459 val_459 +459 val_459 +459 val_459 +459 val_459 +459 val_459 +460 val_460 +460 val_460 +462 val_462 +462 val_462 +462 val_462 +462 val_462 +462 val_462 +462 val_462 +462 val_462 +462 val_462 +463 val_463 +463 val_463 +463 val_463 +463 val_463 +463 val_463 +463 val_463 +463 val_463 +463 val_463 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +466 val_466 +467 val_467 +467 val_467 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +468 val_468 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +469 val_469 +470 val_470 +470 val_470 +472 val_472 +472 val_472 +475 val_475 +475 val_475 +477 val_477 +477 val_477 +478 val_478 +478 val_478 +478 val_478 +478 val_478 +478 val_478 +478 val_478 +478 val_478 +478 val_478 +479 val_479 +479 val_479 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +480 val_480 +481 val_481 +481 val_481 +482 val_482 +482 val_482 +483 val_483 +483 val_483 +484 val_484 +484 val_484 +485 val_485 +485 val_485 +487 val_487 +487 val_487 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +489 val_489 +490 val_490 +490 val_490 +491 val_491 +491 val_491 +492 val_492 +492 val_492 +492 val_492 +492 val_492 +492 val_492 +492 val_492 +492 val_492 +492 val_492 +493 val_493 +493 val_493 +494 val_494 +494 val_494 +495 val_495 +495 val_495 +496 val_496 +496 val_496 +497 val_497 +497 val_497 +498 val_498 +498 val_498 +498 val_498 +498 val_498 +498 val_498 +498 val_498 +498 val_498 +498 val_498 +498 val_498 +498 val_498 +498 val_498 +498 val_498 +498 val_498 +498 val_498 +498 val_498 +498 val_498 +498 val_498 +498 val_498 diff --git serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/StandardStructObjectInspector.java serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/StandardStructObjectInspector.java index c96fc2d..4dff98a 100644 --- serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/StandardStructObjectInspector.java +++ serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/StandardStructObjectInspector.java @@ -51,7 +51,7 @@ protected MyField() { super(); } - + public MyField(int fieldID, String fieldName, ObjectInspector fieldObjectInspector) { this.fieldID = fieldID; @@ -65,18 +65,22 @@ public MyField(int fieldID, String fieldName, this.fieldComment = fieldComment; } + @Override public int getFieldID() { return fieldID; } + @Override public String getFieldName() { return fieldName; } + @Override public ObjectInspector getFieldObjectInspector() { return fieldObjectInspector; } + @Override public String getFieldComment() { return fieldComment; } @@ -115,6 +119,9 @@ protected void init(List structFieldNames, fields = new ArrayList(structFieldNames.size()); for (int i = 0; i < structFieldNames.size(); i++) { + LOG.info("i = " + i + " structFieldNames: " + structFieldNames + + " structFieldObjectInspectors: " + structFieldObjectInspectors + + " structFieldComments: " + structFieldComments); fields.add(new MyField(i, structFieldNames.get(i), structFieldObjectInspectors.get(i), structFieldComments == null ? null : structFieldComments.get(i))); @@ -133,10 +140,12 @@ protected void init(List fields) { } } + @Override public String getTypeName() { return ObjectInspectorUtils.getStandardStructTypeName(this); } + @Override public final Category getCategory() { return Category.STRUCT; }